From b4f2391f9e6d7b8cd29a7d79365517c19799a9e7 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 20 May 2015 18:10:45 -0700 Subject: [PATCH 01/59] KAFKA-2190; Flush mirror maker before commiting offsets; abort the mirror-maker producer with close(0) on send error; reviewed by Joel Koshy --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 24 +++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 9548521..459aaec 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -18,7 +18,7 @@ package kafka.tools import java.util -import java.util.concurrent.CountDownLatch +import java.util.concurrent.{TimeUnit, CountDownLatch} import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import java.util.{Collections, Properties} @@ -222,7 +222,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { connector.setConsumerRebalanceListener(consumerRebalanceListener) } - // create Kafka streams + // create filters val filterSpec = if (options.has(whitelistOpt)) new Whitelist(options.valueOf(whitelistOpt)) else @@ -271,10 +271,6 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { } info("Closing producer.") producer.close() - connectors.foreach(commitOffsets) - // Connector should only be shutdown after offsets are committed. - info("Shutting down consumer connectors.") - connectors.foreach(_.shutdown()) info("Kafka mirror maker shutdown successfully") } } @@ -306,7 +302,6 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val stream = streams(0) val iter = stream.iterator() - // TODO: Need to be changed after KAFKA-1660 is available. while (!exitingOnSendFailure && !shuttingDown) { try { while (!exitingOnSendFailure && !shuttingDown && iter.hasNext()) { @@ -326,6 +321,12 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { case t: Throwable => fatal("Mirror maker thread failure due to ", t) } finally { + info("Flushing producer.") + producer.flush() + info("Committing consumer offsets.") + commitOffsets(connector) + info("Shutting down consumer connectors.") + connector.shutdown() shutdownLatch.countDown() info("Mirror maker thread stopped") // if it exits accidentally, stop the entire mirror maker @@ -388,6 +389,10 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { def close() { this.producer.close() } + + def close(timeout: Long) { + this.producer.close(timeout, TimeUnit.MILLISECONDS) + } } private class MirrorMakerProducerCallback (topic: String, key: Array[Byte], value: Array[Byte]) @@ -399,8 +404,11 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { // still could not be sent. super.onCompletion(metadata, exception) // If abort.on.send.failure is set, stop the mirror maker. Otherwise log skipped message and move on. - if (abortOnSendFailure) + if (abortOnSendFailure) { + info("Closing producer due to send failure.") exitingOnSendFailure = true + producer.close(0) + } numDroppedMessages.incrementAndGet() } } -- 1.7.12.4 From 23ff851f30bb55e794aefd2fae5367845d9230ee Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 20 May 2015 18:20:37 -0700 Subject: [PATCH 02/59] KAFKA-2186; Follow-up to KAFKA-1650 - add selective offset commit to consumer connector API; reviewed by Joel Koshy --- .../main/scala/kafka/javaapi/consumer/ConsumerConnector.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java index cc3400f..f340065 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java @@ -20,9 +20,12 @@ package kafka.javaapi.consumer; import java.util.List; import java.util.Map; + +import kafka.common.OffsetAndMetadata; import kafka.consumer.KafkaStream; import kafka.consumer.TopicFilter; import kafka.serializer.Decoder; +import org.apache.kafka.common.TopicPartition; public interface ConsumerConnector { /** @@ -65,6 +68,14 @@ public interface ConsumerConnector { public void commitOffsets(boolean retryOnFailure); /** + * Commit offsets using the provided offsets map + * + * @param offsetsToCommit a map containing the offset to commit for each partition. + * @param retryOnFailure enable retries on the offset commit if it fails. + */ + public void commitOffsets(Map offsetsToCommit, boolean retryOnFailure); + + /** * Shut down the connector */ public void shutdown(); -- 1.7.12.4 From 467736c7ad4a24e2eb1048b6fe4211bec02e4a05 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Thu, 21 May 2015 14:32:58 -0700 Subject: [PATCH 03/59] Revert "KAFKA-2186; Follow-up to KAFKA-1650 - add selective offset commit to" This reverts commit 23ff851f30bb55e794aefd2fae5367845d9230ee. --- .../main/scala/kafka/javaapi/consumer/ConsumerConnector.java | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java index f340065..cc3400f 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java @@ -20,12 +20,9 @@ package kafka.javaapi.consumer; import java.util.List; import java.util.Map; - -import kafka.common.OffsetAndMetadata; import kafka.consumer.KafkaStream; import kafka.consumer.TopicFilter; import kafka.serializer.Decoder; -import org.apache.kafka.common.TopicPartition; public interface ConsumerConnector { /** @@ -68,14 +65,6 @@ public interface ConsumerConnector { public void commitOffsets(boolean retryOnFailure); /** - * Commit offsets using the provided offsets map - * - * @param offsetsToCommit a map containing the offset to commit for each partition. - * @param retryOnFailure enable retries on the offset commit if it fails. - */ - public void commitOffsets(Map offsetsToCommit, boolean retryOnFailure); - - /** * Shut down the connector */ public void shutdown(); -- 1.7.12.4 From 43b92f8b1ce8140c432edf11b0c842f5fbe04120 Mon Sep 17 00:00:00 2001 From: Vivek Madani Date: Sun, 24 May 2015 12:02:32 -0700 Subject: [PATCH 04/59] KAFKA-1737; Enforce ZKSerializer while creating ZkClient; reviewed by Guozhang Wang --- .../scala/kafka/admin/ConsumerGroupCommand.scala | 2 +- .../PreferredReplicaLeaderElectionCommand.scala | 2 +- .../kafka/admin/ReassignPartitionsCommand.scala | 2 +- core/src/main/scala/kafka/admin/TopicCommand.scala | 2 +- .../consumer/ZookeeperConsumerConnector.scala | 2 +- .../consumer/ZookeeperTopicEventWatcher.scala | 2 +- core/src/main/scala/kafka/server/KafkaServer.scala | 4 +-- .../main/scala/kafka/tools/ConsoleConsumer.scala | 2 +- .../scala/kafka/tools/ConsumerOffsetChecker.scala | 2 +- .../main/scala/kafka/tools/ExportZkOffsets.scala | 4 +-- .../main/scala/kafka/tools/ImportZkOffsets.scala | 4 +-- .../main/scala/kafka/tools/UpdateOffsetsInZK.scala | 6 ++-- .../kafka/tools/VerifyConsumerRebalance.scala | 4 +-- core/src/main/scala/kafka/utils/ZkUtils.scala | 9 ++++-- core/src/test/scala/other/kafka/DeleteZKPath.scala | 5 ++-- .../test/scala/other/kafka/TestOffsetManager.scala | 4 +-- .../consumer/ZookeeperConsumerConnectorTest.scala | 2 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 2 +- .../test/scala/unit/kafka/zk/ZKEphemeralTest.scala | 8 ++--- core/src/test/scala/unit/kafka/zk/ZKPathTest.scala | 34 +++++++++------------- .../scala/unit/kafka/zk/ZooKeeperTestHarness.scala | 4 +-- 21 files changed, 50 insertions(+), 56 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 1c3b380..6d1c6ab 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -48,7 +48,7 @@ object ConsumerGroupCommand { opts.checkArgs() - val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000) try { if (opts.options.has(opts.listOpt)) diff --git a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala index 3b3cd67..2aa6e62 100755 --- a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala +++ b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala @@ -53,7 +53,7 @@ object PreferredReplicaLeaderElectionCommand extends Logging { var zkClient: ZkClient = null try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) val partitionsForPreferredReplicaElection = if (!options.has(jsonFileOpt)) ZkUtils.getAllPartitions(zkClient) diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index acaa611..912b718 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -38,7 +38,7 @@ object ReassignPartitionsCommand extends Logging { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt) val zkConnect = opts.options.valueOf(opts.zkConnectOpt) - var zkClient: ZkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + var zkClient: ZkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) try { if(opts.options.has(opts.verifyOpt)) verifyAssignment(zkClient, opts) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 8e6f186..dacbdd0 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -47,7 +47,7 @@ object TopicCommand { opts.checkArgs() - val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000) try { if(opts.options.has(opts.createOpt)) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 0b0dca1..a7f2acc 100755 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -178,7 +178,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def connectZk() { info("Connecting to zookeeper instance at " + config.zkConnect) - zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) } // Blocks until the offset manager is located and a channel is established to it. diff --git a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala index f2fa36f..f74823b 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala @@ -18,7 +18,7 @@ package kafka.consumer import scala.collection.JavaConversions._ -import kafka.utils.{ZkUtils, ZKStringSerializer, Logging} +import kafka.utils.{ZkUtils, Logging} import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} import org.apache.zookeeper.Watcher.Event.KeeperState diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index ea6d165..e66710d 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -196,13 +196,13 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (chroot.length > 1) { val zkConnForChrootCreation = config.zkConnect.substring(0, config.zkConnect.indexOf("/")) - val zkClientForChrootCreation = new ZkClient(zkConnForChrootCreation, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + val zkClientForChrootCreation = ZkUtils.createZkClient(zkConnForChrootCreation, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) ZkUtils.makeSurePersistentPathExists(zkClientForChrootCreation, chroot) info("Created zookeeper path " + chroot) zkClientForChrootCreation.close() } - val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) ZkUtils.setupCommonPaths(zkClient) zkClient } diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index bba3990..a3bee58 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -209,7 +209,7 @@ object ConsoleConsumer extends Logging { def checkZkPathExists(zkUrl: String, path: String): Boolean = { try { - val zk = new ZkClient(zkUrl, 30*1000,30*1000, ZKStringSerializer) + val zk = ZkUtils.createZkClient(zkUrl, 30*1000,30*1000); zk.exists(path) } catch { case _: Throwable => false diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index d2bac85..ad64cee 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -149,7 +149,7 @@ object ConsumerOffsetChecker extends Logging { var zkClient: ZkClient = null var channel: BlockingChannel = null try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) val topicList = topics match { case Some(x) => x.split(",").view.toList diff --git a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala index ce14bbc..7b52fe4 100644 --- a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala +++ b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala @@ -19,7 +19,7 @@ package kafka.tools import java.io.FileWriter import joptsimple._ -import kafka.utils.{Logging, ZkUtils, ZKStringSerializer, ZKGroupTopicDirs, CommandLineUtils} +import kafka.utils.{Logging, ZkUtils, ZKGroupTopicDirs, CommandLineUtils} import org.I0Itec.zkclient.ZkClient @@ -76,7 +76,7 @@ object ExportZkOffsets extends Logging { val fileWriter : FileWriter = new FileWriter(outfile) try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) var consumerGroups: Seq[String] = null diff --git a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala index 598350d..b56f587 100644 --- a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala +++ b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala @@ -20,7 +20,7 @@ package kafka.tools import java.io.BufferedReader import java.io.FileReader import joptsimple._ -import kafka.utils.{Logging, ZkUtils,ZKStringSerializer, CommandLineUtils} +import kafka.utils.{Logging, ZkUtils, CommandLineUtils} import org.I0Itec.zkclient.ZkClient @@ -68,7 +68,7 @@ object ImportZkOffsets extends Logging { val zkConnect = options.valueOf(zkConnectOpt) val partitionOffsetFile = options.valueOf(inFileOpt) - val zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) val partitionOffsets: Map[String,String] = getPartitionOffsetsFromFile(partitionOffsetFile) updateZkOffsets(zkClient, partitionOffsets) diff --git a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala index 9235ed9..9942686 100755 --- a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala +++ b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala @@ -21,7 +21,7 @@ import org.I0Itec.zkclient.ZkClient import kafka.consumer.{SimpleConsumer, ConsumerConfig} import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} import kafka.common.{TopicAndPartition, KafkaException} -import kafka.utils.{ZKGroupTopicDirs, ZkUtils, ZKStringSerializer, CoreUtils} +import kafka.utils.{ZKGroupTopicDirs, ZkUtils, CoreUtils} import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.utils.Utils @@ -36,8 +36,8 @@ object UpdateOffsetsInZK { if(args.length < 3) usage val config = new ConsumerConfig(Utils.loadProps(args(1))) - val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) args(0) match { case Earliest => getAndSetOffsets(zkClient, OffsetRequest.EarliestTime, config, args(2)) case Latest => getAndSetOffsets(zkClient, OffsetRequest.LatestTime, config, args(2)) diff --git a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala index 4fb519b..db2721f 100644 --- a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala +++ b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala @@ -19,7 +19,7 @@ package kafka.tools import joptsimple.OptionParser import org.I0Itec.zkclient.ZkClient -import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils, ZKStringSerializer, CommandLineUtils} +import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils, CommandLineUtils} object VerifyConsumerRebalance extends Logging { def main(args: Array[String]) { @@ -48,7 +48,7 @@ object VerifyConsumerRebalance extends Logging { var zkClient: ZkClient = null try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) debug("zkConnect = %s; group = %s".format(zkConnect, group)) diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 2618dd3..78475e3 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -460,7 +460,7 @@ object ZkUtils extends Logging { def maybeDeletePath(zkUrl: String, dir: String) { try { - val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) + val zk = createZkClient(zkUrl, 30*1000, 30*1000) zk.deleteRecursive(dir) zk.close() } catch { @@ -781,9 +781,14 @@ object ZkUtils extends Logging { } } } + + def createZkClient(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int): ZkClient = { + val zkClient = new ZkClient(zkUrl, sessionTimeout, connectionTimeout, ZKStringSerializer) + zkClient + } } -object ZKStringSerializer extends ZkSerializer { +private object ZKStringSerializer extends ZkSerializer { @throws(classOf[ZkMarshallingError]) def serialize(data : Object) : Array[Byte] = data.asInstanceOf[String].getBytes("UTF-8") diff --git a/core/src/test/scala/other/kafka/DeleteZKPath.scala b/core/src/test/scala/other/kafka/DeleteZKPath.scala index 33c3ef8..fb8ab9f 100755 --- a/core/src/test/scala/other/kafka/DeleteZKPath.scala +++ b/core/src/test/scala/other/kafka/DeleteZKPath.scala @@ -18,7 +18,7 @@ package kafka import consumer.ConsumerConfig -import utils.{ZKStringSerializer, ZkUtils} +import utils.ZkUtils import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.utils.Utils @@ -32,8 +32,7 @@ object DeleteZKPath { val config = new ConsumerConfig(Utils.loadProps(args(0))) val zkPath = args(1) - val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) try { ZkUtils.deletePathRecursive(zkClient, zkPath); diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala index 9881bd3..4e90534 100644 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -2,7 +2,7 @@ package other.kafka import org.I0Itec.zkclient.ZkClient import kafka.api._ -import kafka.utils.{ShutdownableThread, ZKStringSerializer} +import kafka.utils.{ZkUtils, ShutdownableThread} import org.apache.kafka.common.protocol.SecurityProtocol import scala.collection._ import kafka.client.ClientUtils @@ -238,7 +238,7 @@ object TestOffsetManager { var fetchThread: FetchThread = null var statsThread: StatsThread = null try { - zkClient = new ZkClient(zookeeper, 6000, 2000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zookeeper, 6000, 2000) commitThreads = (0 to (threadCount-1)).map { threadId => new CommitThread(threadId, partitionCount, commitIntervalMs, zkClient) } diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 7f9fca3..359b0f5 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -316,7 +316,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar } def testLeaderSelectionForPartition() { - val zkClient = new ZkClient(zkConnect, 6000, 30000, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(zkConnect, 6000, 30000) // create topic topic1 with 1 partition on broker 0 createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index faae0e9..17e9fe4 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -448,7 +448,7 @@ object TestUtils extends Logging { } def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = { - val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) ZkUtils.updatePersistentPath(zkClient, path, offset.toString) } diff --git a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala index 85eec6f..2be1619 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala @@ -19,7 +19,7 @@ package kafka.zk import kafka.consumer.ConsumerConfig import org.I0Itec.zkclient.ZkClient -import kafka.utils.{ZkUtils, ZKStringSerializer} +import kafka.utils.ZkUtils import kafka.utils.TestUtils import org.junit.Assert import org.scalatest.junit.JUnit3Suite @@ -29,8 +29,7 @@ class ZKEphemeralTest extends JUnit3Suite with ZooKeeperTestHarness { def testEphemeralNodeCleanup = { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) try { ZkUtils.createEphemeralPathExpectConflict(zkClient, "/tmp/zktest", "node created") @@ -42,8 +41,7 @@ class ZKEphemeralTest extends JUnit3Suite with ZooKeeperTestHarness { testData = ZkUtils.readData(zkClient, "/tmp/zktest")._1 Assert.assertNotNull(testData) zkClient.close - zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) val nodeExists = ZkUtils.pathExists(zkClient, "/tmp/zktest") Assert.assertFalse(nodeExists) } diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala index a2d062f..64c3ba2 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala @@ -19,7 +19,7 @@ package unit.kafka.zk import junit.framework.Assert import kafka.consumer.ConsumerConfig -import kafka.utils.{ZkPath, TestUtils, ZKStringSerializer, ZkUtils} +import kafka.utils.{ZkPath, TestUtils, ZkUtils} import kafka.zk.ZooKeeperTestHarness import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.config.ConfigException @@ -34,9 +34,8 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { def testCreatePersistentPathThrowsException { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) - var zkClient = new ZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) try { ZkPath.resetNamespaceCheckedState ZkUtils.createPersistentPath(zkClient, path) @@ -49,8 +48,7 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { def testCreatePersistentPath { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) try { ZkPath.resetNamespaceCheckedState ZkUtils.createPersistentPath(zkClient, path) @@ -64,9 +62,8 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { def testMakeSurePersistsPathExistsThrowsException { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) - var zkClient = new ZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) try { ZkPath.resetNamespaceCheckedState ZkUtils.makeSurePersistentPathExists(zkClient, path) @@ -79,8 +76,7 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { def testMakeSurePersistsPathExists { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) try { ZkPath.resetNamespaceCheckedState ZkUtils.makeSurePersistentPathExists(zkClient, path) @@ -94,9 +90,8 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { def testCreateEphemeralPathThrowsException { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) - var zkClient = new ZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) try { ZkPath.resetNamespaceCheckedState ZkUtils.createEphemeralPathExpectConflict(zkClient, path, "somedata") @@ -109,8 +104,7 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { def testCreateEphemeralPathExists { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) try { ZkPath.resetNamespaceCheckedState ZkUtils.createEphemeralPathExpectConflict(zkClient, path, "somedata") @@ -124,9 +118,8 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { def testCreatePersistentSequentialThrowsException { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, "test", "1")) - var zkClient = new ZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) try { ZkPath.resetNamespaceCheckedState ZkUtils.createSequentialPersistentPath(zkClient, path) @@ -139,8 +132,7 @@ class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { def testCreatePersistentSequentialExists { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) var actualPath: String = "" try { diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 86bddea..1f4d10d 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -19,7 +19,7 @@ package kafka.zk import org.scalatest.junit.JUnit3Suite import org.I0Itec.zkclient.ZkClient -import kafka.utils.{ZKStringSerializer, CoreUtils} +import kafka.utils.{ZkUtils, CoreUtils} trait ZooKeeperTestHarness extends JUnit3Suite { var zkPort: Int = -1 @@ -34,7 +34,7 @@ trait ZooKeeperTestHarness extends JUnit3Suite { super.setUp zookeeper = new EmbeddedZookeeper() zkPort = zookeeper.port - zkClient = new ZkClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeout, zkConnectionTimeout) } override def tearDown() { -- 1.7.12.4 From c60f325014b911b8dec04848d598717e49b09011 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 28 May 2015 10:16:20 -0700 Subject: [PATCH 05/59] kafka-2189; Snappy compression of message batches less efficient in 0.8.2.1; patched by Ismael Juma; reviewed by Jun Rao --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index cd2aa83..96b55db 100644 --- a/build.gradle +++ b/build.gradle @@ -352,7 +352,7 @@ project(':clients') { dependencies { compile "org.slf4j:slf4j-api:1.7.6" - compile 'org.xerial.snappy:snappy-java:1.1.1.6' + compile 'org.xerial.snappy:snappy-java:1.1.1.7' compile 'net.jpountz.lz4:lz4:1.2.0' testCompile 'com.novocode:junit-interface:0.9' -- 1.7.12.4 From 05e8a78145fc3db673bb46be8ba1788558c5a26d Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 28 May 2015 13:22:30 -0700 Subject: [PATCH 06/59] KAFKA-2186; Follow-up to KAFKA-1650 - add selective offset commit to consumer connector API; reviewed by Joel Koshy --- .../main/scala/kafka/javaapi/consumer/ConsumerConnector.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java index cc3400f..ca74ca8 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java @@ -20,6 +20,9 @@ package kafka.javaapi.consumer; import java.util.List; import java.util.Map; + +import kafka.common.OffsetAndMetadata; +import kafka.common.TopicAndPartition; import kafka.consumer.KafkaStream; import kafka.consumer.TopicFilter; import kafka.serializer.Decoder; @@ -65,6 +68,14 @@ public interface ConsumerConnector { public void commitOffsets(boolean retryOnFailure); /** + * Commit offsets using the provided offsets map + * + * @param offsetsToCommit a map containing the offset to commit for each partition. + * @param retryOnFailure enable retries on the offset commit if it fails. + */ + public void commitOffsets(Map offsetsToCommit, boolean retryOnFailure); + + /** * Shut down the connector */ public void shutdown(); -- 1.7.12.4 From d6c45c70fb9773043766446e88370db9709e7995 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 28 May 2015 13:27:05 -0700 Subject: [PATCH 07/59] KAFKA-2091; Expose a partitioner interface in the new producer (https://cwiki.apache.org/confluence/display/KAFKA/KIP-+22+-+Expose+a+Partitioner+interface+in+the+new+producer); reviewed by Joel Koshy and Jay Kreps --- .../kafka/clients/producer/KafkaProducer.java | 82 ++++++++++++-------- .../kafka/clients/producer/MockProducer.java | 40 +++++++--- .../apache/kafka/clients/producer/Partitioner.java | 46 +++++++++++ .../kafka/clients/producer/ProducerConfig.java | 14 +++- .../producer/internals/DefaultPartitioner.java | 89 ++++++++++++++++++++++ .../clients/producer/internals/Partitioner.java | 89 ---------------------- .../producer/internals/DefaultPartitionerTest.java | 63 +++++++++++++++ .../producer/internals/PartitionerTest.java | 68 ----------------- 8 files changed, 289 insertions(+), 202 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java delete mode 100644 clients/src/test/java/org/apache/kafka/clients/producer/internals/PartitionerTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 8e336a3..ded19d8 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 @@ -3,9 +3,9 @@ * 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. @@ -23,7 +23,6 @@ import java.util.concurrent.atomic.AtomicReference; 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.Partitioner; import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.Sender; import org.apache.kafka.common.Cluster; @@ -73,11 +72,11 @@ import org.slf4j.LoggerFactory; * 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(); * } *

@@ -92,25 +91,25 @@ import org.slf4j.LoggerFactory; * 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 + * 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 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 + * 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 + * 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 + * 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. *

@@ -207,7 +206,7 @@ public class KafkaProducer implements Producer { MetricsReporter.class); reporters.add(new JmxReporter(jmxPrefix)); this.metrics = new Metrics(metricConfig, reporters, time); - this.partitioner = new Partitioner(); + this.partitioner = config.getConfiguredInstance(ProducerConfig.PARTITIONER_CLASS_CONFIG, Partitioner.class); long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG); this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); this.metadata = new Metadata(retryBackoffMs, config.getLong(ProducerConfig.METADATA_MAX_AGE_CONFIG)); @@ -285,7 +284,7 @@ public class KafkaProducer implements Producer { } /** - * Asynchronously send a record to a topic. Equivalent to send(record, null). + * Asynchronously send a record to a topic. Equivalent to send(record, null). * See {@link #send(ProducerRecord, Callback)} for details. */ @Override @@ -309,7 +308,7 @@ public class KafkaProducer implements Producer { * or throw any exception that occurred while sending the record. *

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

      * {@code
      * byte[] key = "key".getBytes();
@@ -320,7 +319,7 @@ public class KafkaProducer implements Producer {
      * 

* 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, value);
@@ -334,10 +333,10 @@ public class KafkaProducer implements Producer {
      *               });
      * }
      * 
- * + * * 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
      * producer.send(new ProducerRecord(topic, partition, key1, value1), callback1);
@@ -349,15 +348,15 @@ public class KafkaProducer implements Producer {
      * 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.
-     * 
+     *
      * @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) {
@@ -380,7 +379,7 @@ public class KafkaProducer implements Producer {
                         " to class " + producerConfig.getClass(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName() +
                         " specified in value.serializer");
             }
-            int partition = partitioner.partition(record.topic(), serializedKey, record.partition(), metadata.fetch());
+            int partition = partition(record, serializedKey, serializedValue, metadata.fetch());
             int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue);
             ensureValidRecordSize(serializedSize);
             TopicPartition tp = new TopicPartition(record.topic(), partition);
@@ -452,12 +451,12 @@ public class KafkaProducer implements Producer {
                                               ProducerConfig.BUFFER_MEMORY_CONFIG +
                                               " configuration.");
     }
-    
+
     /**
-     * Invoking this method makes all buffered records immediately available to send (even if linger.ms is 
+     * 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 
+     * 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, @@ -475,10 +474,10 @@ public class KafkaProducer implements Producer { * 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 @@ -550,7 +549,7 @@ public class KafkaProducer implements Producer { public void close(long timeout, TimeUnit timeUnit) { close(timeout, timeUnit, false); } - + private void close(long timeout, TimeUnit timeUnit, boolean swallowException) { if (timeout < 0) throw new IllegalArgumentException("The timeout cannot be negative."); @@ -600,6 +599,27 @@ public class KafkaProducer implements Producer { throw new KafkaException("Failed to close kafka producer", firstException.get()); } + /** + * computes partition for given record. + * if the record has partition returns the value otherwise + * calls configured partitioner class to compute the partition. + */ + private int partition(ProducerRecord record, byte[] serializedKey , byte[] serializedValue, Cluster cluster) { + Integer partition = record.partition(); + if (partition != null) { + List partitions = cluster.partitionsForTopic(record.topic()); + int numPartitions = partitions.size(); + // they have given us a partition, use it + if (partition < 0 || partition >= numPartitions) + throw new IllegalArgumentException("Invalid partition given with record: " + partition + + " is not in the range [0..." + + numPartitions + + "]."); + return partition; + } + return this.partitioner.partition(record.topic(), record.key(), serializedKey, record.value(), serializedValue, cluster); + } + private static class FutureFailure implements Future { private final ExecutionException exception; 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 3c34610..e66491c 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 @@ -27,7 +27,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; -import org.apache.kafka.clients.producer.internals.Partitioner; +import org.apache.kafka.clients.producer.internals.DefaultPartitioner; import org.apache.kafka.clients.producer.internals.ProduceRequestResult; import org.apache.kafka.common.*; @@ -41,7 +41,7 @@ import org.apache.kafka.common.*; public class MockProducer implements Producer { private final Cluster cluster; - private final Partitioner partitioner = new Partitioner(); + private final Partitioner partitioner = new DefaultPartitioner(); private final List> sent; private final Deque completions; private boolean autoComplete; @@ -49,7 +49,7 @@ public class MockProducer implements Producer { /** * Create a mock producer - * + * * @param cluster The cluster holding metadata for this producer * @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise * the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after @@ -66,7 +66,7 @@ public class MockProducer implements Producer { /** * Create a new mock producer with invented metadata the given autoComplete setting. - * + * * Equivalent to {@link #MockProducer(Cluster, boolean) new MockProducer(null, autoComplete)} */ public MockProducer(boolean autoComplete) { @@ -75,7 +75,7 @@ public class MockProducer implements Producer { /** * Create a new auto completing mock producer - * + * * Equivalent to {@link #MockProducer(boolean) new MockProducer(true)} */ public MockProducer() { @@ -94,14 +94,14 @@ public class MockProducer implements Producer { /** * Adds the record to the list of sent records. - * + * * @see #history() */ @Override public synchronized Future send(ProducerRecord record, Callback callback) { int partition = 0; if (this.cluster.partitionsForTopic(record.topic()) != null) - partition = partitioner.partition(record.topic(), record.key(), record.partition(), this.cluster); + partition = partition(record, this.cluster); ProduceRequestResult result = new ProduceRequestResult(); FutureRecordMetadata future = new FutureRecordMetadata(result, 0); TopicPartition topicPartition = new TopicPartition(record.topic(), partition); @@ -129,7 +129,7 @@ public class MockProducer implements Producer { return offset; } } - + public synchronized void flush() { while (!this.completions.isEmpty()) completeNext(); @@ -168,7 +168,7 @@ public class MockProducer implements Producer { /** * Complete the earliest uncompleted call successfully. - * + * * @return true if there was an uncompleted call to complete */ public synchronized boolean completeNext() { @@ -177,7 +177,7 @@ public class MockProducer implements Producer { /** * Complete the earliest uncompleted call with the given error. - * + * * @return true if there was an uncompleted call to complete */ public synchronized boolean errorNext(RuntimeException e) { @@ -190,6 +190,26 @@ public class MockProducer implements Producer { } } + /** + * computes partition for given record. + */ + private int partition(ProducerRecord record, Cluster cluster) { + Integer partition = record.partition(); + if (partition != null) { + List partitions = cluster.partitionsForTopic(record.topic()); + int numPartitions = partitions.size(); + // they have given us a partition, use it + if (partition < 0 || partition >= numPartitions) + throw new IllegalArgumentException("Invalid partition given with record: " + partition + + " is not in the range [0..." + + numPartitions + + "]."); + return partition; + } + return this.partitioner.partition(record.topic(), null, record.key(), null, record.value(), cluster); + } + + private static class Completion { private final long offset; private final RecordMetadata metadata; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java new file mode 100644 index 0000000..383619d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.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.clients.producer; + +import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.Cluster; + +/** + * Partitioner Interface + */ + +public interface Partitioner extends Configurable { + + /** + * Compute the partition for the given record. + * + * @param topic The topic name + * @param key The key to partition on (or null if no key) + * @param keyBytes The serialized key to partition on( or null if no key) + * @param value The value to partition on or null + * @param valueBytes The serialized value to partition on or null + * @param cluster The current cluster metadata + */ + public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster); + + /** + * This is called when partitioner is closed. + */ + public void close(); + +} 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 187d000..023bd2e 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 @@ -3,9 +3,9 @@ * 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. @@ -51,7 +51,7 @@ public class ProducerConfig extends AbstractConfig { /** metadata.max.age.ms */ 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 " @@ -169,6 +169,11 @@ public class ProducerConfig extends AbstractConfig { 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."; + /** partitioner.class */ + public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class"; + private static final String PARTITIONER_CLASS_DOC = "Partitioner class that implements the Partitioner interface."; + + static { 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) @@ -217,7 +222,8 @@ public class ProducerConfig extends AbstractConfig { Importance.LOW, MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_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); + .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC) + .define(PARTITIONER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.producer.internals.DefaultPartitioner", Importance.MEDIUM, PARTITIONER_CLASS_DOC); } public static Map addSerializerToConfig(Map configs, diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java new file mode 100644 index 0000000..f81c496 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java @@ -0,0 +1,89 @@ +/** + * 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.internals; + +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.kafka.clients.producer.Partitioner; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.utils.Utils; + +/** + * The default partitioning strategy: + *
    + *
  • If a partition is specified in the record, use it + *
  • If no partition is specified but a key is present choose a partition based on a hash of the key + *
  • If no partition or key is present choose a partition in a round-robin fashion + */ +public class DefaultPartitioner implements Partitioner { + + private final AtomicInteger counter = new AtomicInteger(new Random().nextInt()); + + /** + * A cheap way to deterministically convert a number to a positive value. When the input is + * positive, the original value is returned. When the input number is negative, the returned + * positive value is the original value bit AND against 0x7fffffff which is not its absolutely + * value. + * + * Note: changing this method in the future will possibly cause partition selection not to be + * compatible with the existing messages already placed on a partition. + * + * @param number a given number + * @return a positive number. + */ + private static int toPositive(int number) { + return number & 0x7fffffff; + } + + public void configure(Map configs) {} + + /** + * Compute the partition for the given record. + * + * @param topic The topic name + * @param key The key to partition on (or null if no key) + * @param keyBytes serialized key to partition on (or null if no key) + * @param value The value to partition on or null + * @param valueBytes serialized value to partition on or null + * @param cluster The current cluster metadata + */ + public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster) { + List partitions = cluster.partitionsForTopic(topic); + int numPartitions = partitions.size(); + if (keyBytes == null) { + int nextValue = counter.getAndIncrement(); + List availablePartitions = cluster.availablePartitionsForTopic(topic); + if (availablePartitions.size() > 0) { + int part = DefaultPartitioner.toPositive(nextValue) % availablePartitions.size(); + return availablePartitions.get(part).partition(); + } else { + // no partitions are available, give a non-available partition + return DefaultPartitioner.toPositive(nextValue) % numPartitions; + } + } else { + // hash the keyBytes to choose a partition + return DefaultPartitioner.toPositive(Utils.murmur2(keyBytes)) % numPartitions; + } + } + + 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 deleted file mode 100644 index 93e7991..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java +++ /dev/null @@ -1,89 +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.producer.internals; - -import java.util.List; -import java.util.Random; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.utils.Utils; - -/** - * The default partitioning strategy: - *
      - *
    • If a partition is specified in the record, use it - *
    • If no partition is specified but a key is present choose a partition based on a hash of the key - *
    • If no partition or key is present choose a partition in a round-robin fashion - */ -public class Partitioner { - - private final AtomicInteger counter = new AtomicInteger(new Random().nextInt()); - - /** - * A cheap way to deterministically convert a number to a positive value. When the input is - * positive, the original value is returned. When the input number is negative, the returned - * positive value is the original value bit AND against 0x7fffffff which is not its absolutely - * value. - * - * Note: changing this method in the future will possibly cause partition selection not to be - * compatible with the existing messages already placed on a partition. - * - * @param number a given number - * @return a positive number. - */ - private static int toPositive(int number) { - return number & 0x7fffffff; - } - - /** - * Compute the partition for the given record. - * - * @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(String topic, byte[] key, Integer partition, Cluster cluster) { - List partitions = cluster.partitionsForTopic(topic); - int numPartitions = partitions.size(); - if (partition != null) { - // they have given us a partition, use it - if (partition < 0 || partition >= numPartitions) - throw new IllegalArgumentException("Invalid partition given with record: " + partition - + " is not in the range [0..." - + numPartitions - + "]."); - return partition; - } else if (key == null) { - int nextValue = counter.getAndIncrement(); - List availablePartitions = cluster.availablePartitionsForTopic(topic); - if (availablePartitions.size() > 0) { - int part = Partitioner.toPositive(nextValue) % availablePartitions.size(); - return availablePartitions.get(part).partition(); - } else { - // no partitions are available, give a non-available partition - return Partitioner.toPositive(nextValue) % numPartitions; - } - } else { - // hash the key to choose a partition - return Partitioner.toPositive(Utils.murmur2(key)) % numPartitions; - } - } - -} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java new file mode 100644 index 0000000..977fa93 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java @@ -0,0 +1,63 @@ +/** + * 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.internals; + +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.kafka.clients.producer.Partitioner; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.junit.Test; + +public class DefaultPartitionerTest { + private byte[] keyBytes = "key".getBytes(); + private Partitioner partitioner = new DefaultPartitioner(); + 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 String topic = "test"; + // 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 + public void testKeyPartitionIsStable() { + int partition = partitioner.partition("test", null, keyBytes, null, null, cluster); + assertEquals("Same key should yield same partition", partition, partitioner.partition("test", null, keyBytes, null, null, cluster)); + } + + @Test + 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 part = partitioner.partition("test", null, null, null, null, cluster); + 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); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/PartitionerTest.java deleted file mode 100644 index 5dadd0e..0000000 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/PartitionerTest.java +++ /dev/null @@ -1,68 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients.producer.internals; - -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.util.List; - -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; -import org.junit.Test; - -public class PartitionerTest { - - private byte[] key = "key".getBytes(); - private Partitioner partitioner = new Partitioner(); - 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 String topic = "test"; - // 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 - public void testUserSuppliedPartitioning() { - 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("test", key, null, cluster); - assertEquals("Same key should yield same partition", partition, partitioner.partition("test", key, null, cluster)); - } - - @Test - 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 part = partitioner.partition("test", null, null, cluster); - 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); - } -} -- 1.7.12.4 From 3836c57865fe9f75f193d9dbdc236c2b81133eac Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 28 May 2015 17:19:44 -0700 Subject: [PATCH 08/59] kafka-2185; Update to Gradle 2.4; patched by Ismael Juma; reviewed by Jun Rao --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 96b55db..3dca28e 100644 --- a/build.gradle +++ b/build.gradle @@ -30,7 +30,7 @@ allprojects { } ext { - gradleVersion = "2.0" + gradleVersion = "2.4" } def isVerificationRequired(project) { -- 1.7.12.4 From 3d8494bca3074f49567f352f6eedd1ed008eeabd Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 29 May 2015 14:50:45 -0700 Subject: [PATCH 09/59] KAFKA-2199 Make signing artifacts optional and disabled by default for SNAPSHOTs and allow remote Maven repository configuration from the command line. --- README.md | 3 +++ build.gradle | 50 ++++++++++++++++++++++++++------------------------ gradle.properties | 4 ---- 3 files changed, 29 insertions(+), 28 deletions(-) diff --git a/README.md b/README.md index 946ec62..a9a5d1e 100644 --- a/README.md +++ b/README.md @@ -93,6 +93,9 @@ Please note for this to work you should create/update `~/.gradle/gradle.properti signing.password= signing.secretKeyRingFile= +### Publishing the jars without signing to a local repository ### + ./gradlew -Dorg.gradle.project.skipSigning=true -Dorg.gradle.project.mavenUrl=file://path/to/repo uploadArchivesAll + ### Building the test jar ### ./gradlew testJar diff --git a/build.gradle b/build.gradle index 3dca28e..30d1cf2 100644 --- a/build.gradle +++ b/build.gradle @@ -31,10 +31,13 @@ allprojects { ext { gradleVersion = "2.4" -} -def isVerificationRequired(project) { - project.gradle.startParameter.taskNames.any { it.contains("upload") } + skipSigning = project.hasProperty('skipSigning') && skipSigning.toBoolean() + shouldSign = !skipSigning && !version.endsWith("SNAPSHOT") + + mavenUrl = project.hasProperty('mavenUrl') ? project.mavenUrl : '' + mavenUsername = project.hasProperty('mavenUsername') ? project.mavenUsername : '' + mavenPassword = project.hasProperty('mavenPassword') ? project.mavenPassword : '' } apply from: file('wrapper.gradle') @@ -54,27 +57,26 @@ subprojects { uploadArchives { repositories { signing { - if (isVerificationRequired(project)) { - sign configurations.archives - - // To test locally, replace mavenUrl in ~/.gradle/gradle.properties to file://localhost/tmp/myRepo/ - mavenDeployer { - beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) } - repository(url: "${mavenUrl}") { - authentication(userName: "${mavenUsername}", password: "${mavenPassword}") - } - afterEvaluate { - pom.artifactId = "${archivesBaseName}" - pom.project { - name 'Apache Kafka' - packaging 'jar' - url 'http://kafka.apache.org' - licenses { - license { - name 'The Apache Software License, Version 2.0' - url 'http://www.apache.org/licenses/LICENSE-2.0.txt' - distribution 'repo' - } + required { shouldSign } + sign configurations.archives + + // To test locally, replace mavenUrl in ~/.gradle/gradle.properties to file://localhost/tmp/myRepo/ + mavenDeployer { + beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) } + repository(url: "${mavenUrl}") { + authentication(userName: "${mavenUsername}", password: "${mavenPassword}") + } + afterEvaluate { + pom.artifactId = "${archivesBaseName}" + pom.project { + name 'Apache Kafka' + packaging 'jar' + url 'http://kafka.apache.org' + licenses { + license { + name 'The Apache Software License, Version 2.0' + url 'http://www.apache.org/licenses/LICENSE-2.0.txt' + distribution 'repo' } } } diff --git a/gradle.properties b/gradle.properties index 90b1945..cd4050b 100644 --- a/gradle.properties +++ b/gradle.properties @@ -18,7 +18,3 @@ version=0.8.3-SNAPSHOT scalaVersion=2.10.5 task=build org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m - -mavenUrl= -mavenUsername= -mavenPassword= -- 1.7.12.4 From 9e894aa0173b14d64a900bcf780d6b7809368384 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 1 Jun 2015 14:15:16 -0700 Subject: [PATCH 10/59] kafka-2226; NullPointerException in TestPurgatoryPerformance; patched by Yasuhiro Matsuda; reviewed by Onur Karaman, Guozhang Wang and Jun Rao --- core/src/main/scala/kafka/utils/timer/Timer.scala | 5 +- .../main/scala/kafka/utils/timer/TimerTask.scala | 8 ++- .../scala/kafka/utils/timer/TimerTaskList.scala | 65 ++++++++++++++++------ .../main/scala/kafka/utils/timer/TimingWheel.scala | 5 +- 4 files changed, 60 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/kafka/utils/timer/Timer.scala b/core/src/main/scala/kafka/utils/timer/Timer.scala index b8cde82..bdd0e75 100644 --- a/core/src/main/scala/kafka/utils/timer/Timer.scala +++ b/core/src/main/scala/kafka/utils/timer/Timer.scala @@ -51,8 +51,9 @@ class Timer(taskExecutor: ExecutorService, tickMs: Long = 1, wheelSize: Int = 20 private def addTimerTaskEntry(timerTaskEntry: TimerTaskEntry): Unit = { if (!timingWheel.add(timerTaskEntry)) { - // already expired - taskExecutor.submit(timerTaskEntry.timerTask) + // Already expired or cancelled + if (!timerTaskEntry.cancelled) + taskExecutor.submit(timerTaskEntry.timerTask) } } diff --git a/core/src/main/scala/kafka/utils/timer/TimerTask.scala b/core/src/main/scala/kafka/utils/timer/TimerTask.scala index 3407138..d6b3a2e 100644 --- a/core/src/main/scala/kafka/utils/timer/TimerTask.scala +++ b/core/src/main/scala/kafka/utils/timer/TimerTask.scala @@ -33,11 +33,15 @@ trait TimerTask extends Runnable { synchronized { // if this timerTask is already held by an existing timer task entry, // we will remove such an entry first. - if (timerTaskEntry != null && timerTaskEntry != entry) { + if (timerTaskEntry != null && timerTaskEntry != entry) timerTaskEntry.remove() - } + timerTaskEntry = entry } } + private[timer] def getTimerTaskEntry(): TimerTaskEntry = { + timerTaskEntry + } + } diff --git a/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala index e7a9657..c4aeb5d 100644 --- a/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala +++ b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala @@ -52,7 +52,9 @@ private[timer] class TimerTaskList(taskCounter: AtomicInteger) extends Delayed { var entry = root.next while (entry ne root) { val nextEntry = entry.next - f(entry.timerTask) + + if (!entry.cancelled) f(entry.timerTask) + entry = nextEntry } } @@ -60,28 +62,43 @@ private[timer] class TimerTaskList(taskCounter: AtomicInteger) extends Delayed { // Add a timer task entry to this list def add(timerTaskEntry: TimerTaskEntry): Unit = { - synchronized { - // put the timer task entry to the end of the list. (root.prev points to the tail entry) - val tail = root.prev - timerTaskEntry.next = root - timerTaskEntry.prev = tail - timerTaskEntry.list = this - tail.next = timerTaskEntry - root.prev = timerTaskEntry - taskCounter.incrementAndGet() + var done = false + while (!done) { + // Remove the timer task entry if it is already in any other list + // We do this outside of the sync block below to avoid deadlocking. + // We may retry until timerTaskEntry.list becomes null. + timerTaskEntry.remove() + + synchronized { + timerTaskEntry.synchronized { + if (timerTaskEntry.list == null) { + // put the timer task entry to the end of the list. (root.prev points to the tail entry) + val tail = root.prev + timerTaskEntry.next = root + timerTaskEntry.prev = tail + timerTaskEntry.list = this + tail.next = timerTaskEntry + root.prev = timerTaskEntry + taskCounter.incrementAndGet() + done = true + } + } + } } } // Remove the specified timer task entry from this list def remove(timerTaskEntry: TimerTaskEntry): Unit = { synchronized { - if (timerTaskEntry.list != null) { - timerTaskEntry.next.prev = timerTaskEntry.prev - timerTaskEntry.prev.next = timerTaskEntry.next - timerTaskEntry.next = null - timerTaskEntry.prev = null - timerTaskEntry.list = null - taskCounter.decrementAndGet() + timerTaskEntry.synchronized { + if (timerTaskEntry.list eq this) { + timerTaskEntry.next.prev = timerTaskEntry.prev + timerTaskEntry.prev.next = timerTaskEntry.next + timerTaskEntry.next = null + timerTaskEntry.prev = null + timerTaskEntry.list = null + taskCounter.decrementAndGet() + } } } } @@ -116,6 +133,7 @@ private[timer] class TimerTaskList(taskCounter: AtomicInteger) extends Delayed { private[timer] class TimerTaskEntry(val timerTask: TimerTask) { + @volatile var list: TimerTaskList = null var next: TimerTaskEntry = null var prev: TimerTaskEntry = null @@ -124,8 +142,19 @@ private[timer] class TimerTaskEntry(val timerTask: TimerTask) { // setTimerTaskEntry will remove it. if (timerTask != null) timerTask.setTimerTaskEntry(this) + def cancelled: Boolean = { + timerTask.getTimerTaskEntry != this + } + def remove(): Unit = { - if (list != null) list.remove(this) + var currentList = list + // If remove is called when another thread is moving the entry from a task entry list to another, + // this may fail to remove the entry due to the change of value of list. Thus, we retry until the list becomes null. + // In a rare case, this thread sees null and exits the loop, but the other thread insert the entry to another list later. + while (currentList != null) { + currentList.remove(this) + currentList = list + } } } diff --git a/core/src/main/scala/kafka/utils/timer/TimingWheel.scala b/core/src/main/scala/kafka/utils/timer/TimingWheel.scala index e92aba3..f5b6efe 100644 --- a/core/src/main/scala/kafka/utils/timer/TimingWheel.scala +++ b/core/src/main/scala/kafka/utils/timer/TimingWheel.scala @@ -125,7 +125,10 @@ private[timer] class TimingWheel(tickMs: Long, wheelSize: Int, startMs: Long, ta def add(timerTaskEntry: TimerTaskEntry): Boolean = { val expiration = timerTaskEntry.timerTask.expirationMs - if (expiration < currentTime + tickMs) { + if (timerTaskEntry.cancelled) { + // Cancelled + false + } else if (expiration < currentTime + tickMs) { // Already expired false } else if (expiration < currentTime + interval) { -- 1.7.12.4 From 48edeca33b4d47f65ddf47c46b30d0ae605295b8 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Tue, 2 Jun 2015 20:55:29 -0700 Subject: [PATCH 11/59] KAFKA-2161; Fix a few copyrights --- .../integration/kafka/api/ConsumerBounceTest.scala | 20 +++++++++----------- .../integration/kafka/api/FixedPortTestUtils.scala | 19 ++++++++----------- .../integration/kafka/api/ProducerBounceTest.scala | 19 ++++++++----------- 3 files changed, 25 insertions(+), 33 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 5c4cca6..fbc6706 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -1,18 +1,16 @@ /** - * Copyright 2015 Confluent Inc. - * - * Licensed 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 + * 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. - **/ + * 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 kafka.server.KafkaConfig diff --git a/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala b/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala index 1d31a43..b26b242 100644 --- a/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala +++ b/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala @@ -1,18 +1,15 @@ /** - * Copyright 2015 Confluent Inc. - * - * Licensed 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 + * 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. - **/ + * 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 diff --git a/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala index c9d16bb..ce70a0a 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala @@ -1,18 +1,15 @@ /** - * Copyright 2015 Confluent Inc. - * - * Licensed 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 + * 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. - **/ + * 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 -- 1.7.12.4 From d22987f01d50549d855ae092b69f520d75bfeb7b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 3 Jun 2015 13:47:01 -0700 Subject: [PATCH 12/59] KAFKA-2208; add consumer side error handling upon coordinator failure; reviewed by Onur Karaman --- .../clients/consumer/internals/Coordinator.java | 91 +++++++++++++++------- .../kafka/clients/consumer/internals/Fetcher.java | 7 +- .../kafka/common/requests/HeartbeatResponse.java | 5 +- .../kafka/common/requests/JoinGroupResponse.java | 7 +- .../kafka/coordinator/ConsumerCoordinator.scala | 27 ++++--- .../kafka/coordinator/ConsumerGroupMetadata.scala | 18 +++-- .../main/scala/kafka/network/RequestChannel.scala | 9 ++- core/src/main/scala/kafka/server/KafkaConfig.scala | 28 +++++++ .../integration/kafka/api/ConsumerBounceTest.scala | 10 ++- .../scala/integration/kafka/api/ConsumerTest.scala | 14 ++-- .../kafka/server/KafkaConfigConfigDefTest.scala | 5 ++ 11 files changed, 156 insertions(+), 65 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index b2764df..fac7995 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -120,30 +120,58 @@ public final class Coordinator { // send a join group request to the coordinator log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); - JoinGroupRequest request = new JoinGroupRequest(groupId, - (int) this.sessionTimeoutMs, - subscribedTopics, - this.consumerId, - this.assignmentStrategy); - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now); - - // process the response - JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); - // TODO: needs to handle disconnects and errors, should not just throw exceptions - Errors.forCode(response.errorCode()).maybeThrow(); - this.consumerId = response.consumerId(); - this.generation = response.generationId(); - - // set the flag to refresh last committed offsets - this.subscriptions.needRefreshCommits(); - - log.debug("Joined group: {}", response); - - // record re-assignment time - this.sensors.partitionReassignments.record(time.milliseconds() - now); - - // return assigned partitions - return response.assignedPartitions(); + // repeat processing the response until succeed or fatal error + do { + JoinGroupRequest request = new JoinGroupRequest(groupId, + (int) this.sessionTimeoutMs, + subscribedTopics, + this.consumerId, + this.assignmentStrategy); + + ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now); + JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); + short errorCode = response.errorCode(); + + if (errorCode == Errors.NONE.code()) { + this.consumerId = response.consumerId(); + this.generation = response.generationId(); + + // set the flag to refresh last committed offsets + this.subscriptions.needRefreshCommits(); + + log.debug("Joined group: {}", response); + + // record re-assignment time + this.sensors.partitionReassignments.record(time.milliseconds() - now); + + // return assigned partitions + return response.assignedPartitions(); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { + // reset the consumer id and retry immediately + this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", + groupId); + } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + // re-discover the coordinator and retry with backoff + coordinatorDead(); + Utils.sleep(this.retryBackoffMs); + + log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", + groupId); + } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code() + || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code() + || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) { + // log the error and re-throw the exception + log.error("Attempt to join group {} failed due to: {}", + groupId, Errors.forCode(errorCode).exception().getMessage()); + Errors.forCode(errorCode).maybeThrow(); + } else { + // unexpected error, throw the exception + throw new KafkaException("Unexpected error in join group response: " + + Errors.forCode(response.errorCode()).exception().getMessage()); + } + } while (true); } /** @@ -217,7 +245,6 @@ public final class Coordinator { // parse the response to get the offsets boolean offsetsReady = true; OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); - // TODO: needs to handle disconnects Map offsets = new HashMap(response.responseData().size()); for (Map.Entry entry : response.responseData().entrySet()) { TopicPartition tp = entry.getKey(); @@ -239,7 +266,8 @@ public final class Coordinator { // just ignore this partition log.debug("Unknown topic or partition for " + tp); } else { - throw new IllegalStateException("Unexpected error code " + data.errorCode + " while fetching offset"); + throw new KafkaException("Unexpected error in fetch offset response: " + + Errors.forCode(data.errorCode).exception().getMessage()); } } else if (data.offset >= 0) { // record the position with the offset (-1 indicates no committed offset to fetch) @@ -471,9 +499,15 @@ public final class Coordinator { if (response.errorCode() == Errors.NONE.code()) { log.debug("Received successful heartbeat response."); } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() - || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); coordinatorDead(); } else if (response.errorCode() == Errors.ILLEGAL_GENERATION.code()) { + log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group."); + subscriptions.needReassignment(); + } else if (response.errorCode() == Errors.UNKNOWN_CONSUMER_ID.code()) { + log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group."); + consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; subscriptions.needReassignment(); } else { throw new KafkaException("Unexpected error in heartbeat response: " @@ -506,9 +540,10 @@ public final class Coordinator { 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()) { + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { coordinatorDead(); } else { + // do not need to throw the exception but just log the error log.error("Error committing partition {} at offset {}: {}", tp, offset, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index ef9dd52..c5e577f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -231,13 +231,14 @@ public class Fetcher { log.debug("Fetched offset {} for partition {}", offset, topicPartition); return offset; } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() - || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { + || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", topicPartition); awaitMetadataUpdate(); } else { - // TODO: we should not just throw exceptions but should handle and log it. - Errors.forCode(errorCode).maybeThrow(); + log.error("Attempt to fetch offsets for partition {} failed due to: {}", + topicPartition, Errors.forCode(errorCode).exception().getMessage()); + awaitMetadataUpdate(); } } } else { 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 f548cd0..96e6ab0 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 @@ -27,7 +27,10 @@ public class HeartbeatResponse extends AbstractRequestResponse { /** * Possible error code: * - * TODO + * CONSUMER_COORDINATOR_NOT_AVAILABLE (15) + * NOT_COORDINATOR_FOR_CONSUMER (16) + * ILLEGAL_GENERATION (22) + * UNKNOWN_CONSUMER_ID (25) */ private final short errorCode; 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 fd9c545..8d418cd 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 @@ -30,7 +30,12 @@ public class JoinGroupResponse extends AbstractRequestResponse { /** * Possible error code: * - * TODO + * CONSUMER_COORDINATOR_NOT_AVAILABLE (15) + * NOT_COORDINATOR_FOR_CONSUMER (16) + * INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY (23) + * UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY (24) + * UNKNOWN_CONSUMER_ID (25) + * INVALID_SESSION_TIMEOUT (26) */ private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index af06ad4..51e89c8 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -25,11 +25,6 @@ import org.apache.kafka.common.requests.JoinGroupRequest import org.I0Itec.zkclient.ZkClient import java.util.concurrent.atomic.AtomicBoolean -// TODO: expose MinSessionTimeoutMs and MaxSessionTimeoutMs in broker configs -object ConsumerCoordinator { - private val MinSessionTimeoutMs = 6000 - private val MaxSessionTimeoutMs = 30000 -} /** * ConsumerCoordinator handles consumer group and consumer offset management. @@ -41,7 +36,6 @@ object ConsumerCoordinator { class ConsumerCoordinator(val config: KafkaConfig, val zkClient: ZkClient, val offsetManager: OffsetManager) extends Logging { - import ConsumerCoordinator._ this.logIdent = "[ConsumerCoordinator " + config.brokerId + "]: " @@ -93,15 +87,18 @@ class ConsumerCoordinator(val config: KafkaConfig, responseCallback(Set.empty, consumerId, 0, Errors.NOT_COORDINATOR_FOR_CONSUMER.code) } else if (!PartitionAssignor.strategies.contains(partitionAssignmentStrategy)) { responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code) - } else if (sessionTimeoutMs < MinSessionTimeoutMs || sessionTimeoutMs > MaxSessionTimeoutMs) { + } else if (sessionTimeoutMs < config.consumerMinSessionTimeoutMs || sessionTimeoutMs > config.consumerMaxSessionTimeoutMs) { responseCallback(Set.empty, consumerId, 0, Errors.INVALID_SESSION_TIMEOUT.code) } else { - val group = coordinatorMetadata.getGroup(groupId) + // only try to create the group if the group is not unknown AND + // the consumer id is UNKNOWN, if consumer is specified but group does not + // exist we should reject the request + var group = coordinatorMetadata.getGroup(groupId) if (group == null) { if (consumerId != JoinGroupRequest.UNKNOWN_CONSUMER_ID) { responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_CONSUMER_ID.code) } else { - val group = coordinatorMetadata.addGroup(groupId, partitionAssignmentStrategy) + group = coordinatorMetadata.addGroup(groupId, partitionAssignmentStrategy) doJoinGroup(group, consumerId, topics, sessionTimeoutMs, partitionAssignmentStrategy, responseCallback) } } else { @@ -118,10 +115,16 @@ class ConsumerCoordinator(val config: KafkaConfig, responseCallback:(Set[TopicAndPartition], String, Int, Short) => Unit) { group synchronized { if (group.is(Dead)) { + // if the group is marked as dead, it means some other thread has just removed the group + // from the coordinator metadata; this is likely that the group has migrated to some other + // coordinator OR the group is in a transient unstable phase. Let the consumer to retry + // joining without specified consumer id, responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_CONSUMER_ID.code) } else if (partitionAssignmentStrategy != group.partitionAssignmentStrategy) { responseCallback(Set.empty, consumerId, 0, Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code) } else if (consumerId != JoinGroupRequest.UNKNOWN_CONSUMER_ID && !group.has(consumerId)) { + // if the consumer trying to register with a un-recognized id, send the response to let + // it reset its consumer id and retry responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_CONSUMER_ID.code) } else if (group.has(consumerId) && group.is(Stable) && topics == group.get(consumerId).topics) { /* @@ -170,6 +173,10 @@ class ConsumerCoordinator(val config: KafkaConfig, } else { val group = coordinatorMetadata.getGroup(groupId) if (group == null) { + // if the group is marked as dead, it means some other thread has just removed the group + // from the coordinator metadata; this is likely that the group has migrated to some other + // coordinator OR the group is in a transient unstable phase. Let the consumer to retry + // joining without specified consumer id, responseCallback(Errors.UNKNOWN_CONSUMER_ID.code) } else { group synchronized { @@ -304,7 +311,7 @@ class ConsumerCoordinator(val config: KafkaConfig, if (group.isEmpty) { group.transitionTo(Dead) - info("Group %s generation %s is dead".format(group.groupId, group.generationId)) + info("Group %s generation %s is dead and removed".format(group.groupId, group.generationId)) coordinatorMetadata.removeGroup(group.groupId, group.topics) } } diff --git a/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala b/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala index 47bdfa7..0e3657f 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala @@ -62,6 +62,14 @@ private[coordinator] case object Stable extends GroupState { val state: Byte = 3 private[coordinator] case object Dead extends GroupState { val state: Byte = 4 } +private object ConsumerGroupMetadata { + private val validPreviousStates: Map[GroupState, Set[GroupState]] = + Map(Dead -> Set(PreparingRebalance), + Stable -> Set(Rebalancing), + PreparingRebalance -> Set(Stable), + Rebalancing -> Set(PreparingRebalance)) +} + /** * Group contains the following metadata: * @@ -77,12 +85,6 @@ private[coordinator] case object Dead extends GroupState { val state: Byte = 4 } private[coordinator] class ConsumerGroupMetadata(val groupId: String, val partitionAssignmentStrategy: String) { - private val validPreviousStates: Map[GroupState, Set[GroupState]] = - Map(Dead -> Set(PreparingRebalance), - Stable -> Set(Rebalancing), - PreparingRebalance -> Set(Stable), - Rebalancing -> Set(PreparingRebalance)) - private val consumers = new mutable.HashMap[String, ConsumerMetadata] private var state: GroupState = Stable var generationId = 0 @@ -124,8 +126,8 @@ private[coordinator] class ConsumerGroupMetadata(val groupId: String, } private def assertValidTransition(targetState: GroupState) { - if (!validPreviousStates(targetState).contains(state)) + if (!ConsumerGroupMetadata.validPreviousStates(targetState).contains(state)) throw new IllegalStateException("Group %s should be in the %s states before moving to %s state. Instead it is in %s state" - .format(groupId, validPreviousStates(targetState).mkString(","), targetState, state)) + .format(groupId, ConsumerGroupMetadata.validPreviousStates(targetState).mkString(","), targetState, state)) } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 1d0024c..24edb61 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -49,11 +49,14 @@ object RequestChannel extends Logging { @volatile var responseCompleteTimeMs = -1L @volatile var responseDequeueTimeMs = -1L val requestId = buffer.getShort() + // for server-side request / response format + // TODO: this will be removed once we migrated to client-side format val requestObj = if ( RequestKeys.keyToNameAndDeserializerMap.contains(requestId)) RequestKeys.deserializerForKey(requestId)(buffer) else null + // for client-side request / response format val header: RequestHeader = if (requestObj == null) { buffer.rewind @@ -68,7 +71,7 @@ object RequestChannel extends Logging { buffer = null private val requestLogger = Logger.getLogger("kafka.request.logger") - trace("Processor %d received request : %s".format(processor, requestObj)) + trace("Processor %d received request : %s".format(processor, if (requestObj != null) requestObj.describe(false) else header.toString + " : " + body.toString)) def updateRequestMetrics() { val endTimeMs = SystemTime.milliseconds @@ -101,10 +104,10 @@ object RequestChannel extends Logging { } if(requestLogger.isTraceEnabled) requestLogger.trace("Completed request:%s from client %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" - .format(requestObj.describe(true), remoteAddress, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) + .format(if (requestObj != null) requestObj.describe(true) else header.toString + " : " + body.toString, remoteAddress, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) else if(requestLogger.isDebugEnabled) { requestLogger.debug("Completed request:%s from client %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" - .format(requestObj.describe(false), remoteAddress, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) + .format(if (requestObj != null) requestObj.describe(false) else header.toString + " : " + body.toString, remoteAddress, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) } } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 9efa15c..6f25afd 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -111,6 +111,10 @@ object Defaults { val ControlledShutdownRetryBackoffMs = 5000 val ControlledShutdownEnable = true + /** ********* Consumer coordinator configuration ***********/ + val ConsumerMinSessionTimeoutMs = 6000 + val ConsumerMaxSessionTimeoutMs = 30000 + /** ********* Offset management configuration ***********/ val OffsetMetadataMaxSize = OffsetManagerConfig.DefaultMaxMetadataSize val OffsetsLoadBufferSize = OffsetManagerConfig.DefaultLoadBufferSize @@ -218,6 +222,9 @@ object KafkaConfig { val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries" val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms" val ControlledShutdownEnableProp = "controlled.shutdown.enable" + /** ********* Consumer coordinator configuration ***********/ + val ConsumerMinSessionTimeoutMsProp = "consumer.min.session.timeout.ms" + val ConsumerMaxSessionTimeoutMsProp = "consumer.max.session.timeout.ms" /** ********* Offset management configuration ***********/ val OffsetMetadataMaxSizeProp = "offset.metadata.max.bytes" val OffsetsLoadBufferSizeProp = "offsets.load.buffer.size" @@ -343,6 +350,9 @@ object KafkaConfig { val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens" val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server" + /** ********* Consumer coordinator configuration ***********/ + val ConsumerMinSessionTimeoutMsDoc = "The minimum allowed session timeout for registered consumers" + val ConsumerMaxSessionTimeoutMsDoc = "The maximum allowed session timeout for registered consumers" /** ********* Offset management configuration ***********/ val OffsetMetadataMaxSizeDoc = "The maximum size for a metadata entry associated with an offset commit" val OffsetsLoadBufferSizeDoc = "Batch size for reading from the offsets segments when loading offsets into the cache." @@ -461,11 +471,16 @@ object KafkaConfig { .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, HIGH, UncleanLeaderElectionEnableDoc) .define(InterBrokerSecurityProtocolProp, STRING, Defaults.InterBrokerSecurityProtocol, MEDIUM, InterBrokerSecurityProtocolDoc) .define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, MEDIUM, InterBrokerProtocolVersionDoc) + /** ********* Controlled shutdown configuration ***********/ .define(ControlledShutdownMaxRetriesProp, INT, Defaults.ControlledShutdownMaxRetries, MEDIUM, ControlledShutdownMaxRetriesDoc) .define(ControlledShutdownRetryBackoffMsProp, INT, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc) + /** ********* Consumer coordinator configuration ***********/ + .define(ConsumerMinSessionTimeoutMsProp, INT, Defaults.ConsumerMinSessionTimeoutMs, MEDIUM, ConsumerMinSessionTimeoutMsDoc) + .define(ConsumerMaxSessionTimeoutMsProp, INT, Defaults.ConsumerMaxSessionTimeoutMs, MEDIUM, ConsumerMaxSessionTimeoutMsDoc) + /** ********* Offset management configuration ***********/ .define(OffsetMetadataMaxSizeProp, INT, Defaults.OffsetMetadataMaxSize, HIGH, OffsetMetadataMaxSizeDoc) .define(OffsetsLoadBufferSizeProp, INT, Defaults.OffsetsLoadBufferSize, atLeast(1), HIGH, OffsetsLoadBufferSizeDoc) @@ -581,11 +596,16 @@ object KafkaConfig { uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[Boolean], interBrokerSecurityProtocol = SecurityProtocol.valueOf(parsed.get(InterBrokerSecurityProtocolProp).asInstanceOf[String]), interBrokerProtocolVersion = ApiVersion(parsed.get(InterBrokerProtocolVersionProp).asInstanceOf[String]), + /** ********* Controlled shutdown configuration ***********/ controlledShutdownMaxRetries = parsed.get(ControlledShutdownMaxRetriesProp).asInstanceOf[Int], controlledShutdownRetryBackoffMs = parsed.get(ControlledShutdownRetryBackoffMsProp).asInstanceOf[Int], controlledShutdownEnable = parsed.get(ControlledShutdownEnableProp).asInstanceOf[Boolean], + /** ********* Consumer coordinator configuration ***********/ + consumerMinSessionTimeoutMs = parsed.get(ConsumerMinSessionTimeoutMsProp).asInstanceOf[Int], + consumerMaxSessionTimeoutMs = parsed.get(ConsumerMaxSessionTimeoutMsProp).asInstanceOf[Int], + /** ********* Offset management configuration ***********/ offsetMetadataMaxSize = parsed.get(OffsetMetadataMaxSizeProp).asInstanceOf[Int], offsetsLoadBufferSize = parsed.get(OffsetsLoadBufferSizeProp).asInstanceOf[Int], @@ -729,6 +749,10 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val controlledShutdownRetryBackoffMs: Int = Defaults.ControlledShutdownRetryBackoffMs, val controlledShutdownEnable: Boolean = Defaults.ControlledShutdownEnable, + /** ********* Consumer coordinator configuration ***********/ + val consumerMinSessionTimeoutMs: Int = Defaults.ConsumerMinSessionTimeoutMs, + val consumerMaxSessionTimeoutMs: Int = Defaults.ConsumerMaxSessionTimeoutMs, + /** ********* Offset management configuration ***********/ val offsetMetadataMaxSize: Int = Defaults.OffsetMetadataMaxSize, val offsetsLoadBufferSize: Int = Defaults.OffsetsLoadBufferSize, @@ -951,6 +975,10 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ props.put(ControlledShutdownRetryBackoffMsProp, controlledShutdownRetryBackoffMs.toString) props.put(ControlledShutdownEnableProp, controlledShutdownEnable.toString) + /** ********* Consumer coordinator configuration ***********/ + props.put(ConsumerMinSessionTimeoutMsProp, consumerMinSessionTimeoutMs.toString) + props.put(ConsumerMaxSessionTimeoutMsProp, consumerMaxSessionTimeoutMs.toString) + /** ********* Offset management configuration ***********/ props.put(OffsetMetadataMaxSizeProp, offsetMetadataMaxSize.toString) props.put(OffsetsLoadBufferSizeProp, offsetsLoadBufferSize.toString) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index fbc6706..f56096b 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -40,12 +40,14 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { 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.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset + this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") + this.serverConfig.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, "10") // set small enough session timeout 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.SESSION_TIMEOUT_MS_CONFIG, "20") this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") override def generateConfigs() = { @@ -60,7 +62,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) } - def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(5) + def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(20) /* * 1. Produce a bunch of messages diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index a1eed96..17b17b9 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -24,8 +24,8 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.TopicPartition import org.apache.kafka.clients.consumer.NoOffsetForPartitionException -import kafka.utils.{ShutdownableThread, TestUtils, Logging} -import kafka.server.OffsetManager +import kafka.utils.{TestUtils, Logging} +import kafka.server.{KafkaConfig, OffsetManager} import java.util.ArrayList import org.junit.Assert._ @@ -47,9 +47,10 @@ class ConsumerTest extends IntegrationTestHarness with Logging { 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.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset + this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") + this.serverConfig.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, "100") // set small enough session timeout 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) @@ -146,8 +147,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { assertNull(this.consumers(0).partitionsFor("non-exist-topic")) } - // TODO: fix test after fixing consumer-side Coordinator logic - def failingTestPartitionReassignmentCallback() { + 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()) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 8014a5a..71f48c0 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -133,6 +133,9 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { Assert.assertEquals(expectedConfig.controlledShutdownRetryBackoffMs, actualConfig.controlledShutdownRetryBackoffMs) Assert.assertEquals(expectedConfig.controlledShutdownEnable, actualConfig.controlledShutdownEnable) + Assert.assertEquals(expectedConfig.consumerMinSessionTimeoutMs, actualConfig.consumerMinSessionTimeoutMs) + Assert.assertEquals(expectedConfig.consumerMaxSessionTimeoutMs, actualConfig.consumerMaxSessionTimeoutMs) + Assert.assertEquals(expectedConfig.offsetMetadataMaxSize, actualConfig.offsetMetadataMaxSize) Assert.assertEquals(expectedConfig.offsetsLoadBufferSize, actualConfig.offsetsLoadBufferSize) Assert.assertEquals(expectedConfig.offsetsTopicReplicationFactor, actualConfig.offsetsTopicReplicationFactor) @@ -330,6 +333,8 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.ConsumerMinSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ConsumerMaxSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.OffsetMetadataMaxSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.OffsetsLoadBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.OffsetsTopicReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") -- 1.7.12.4 From 78ba492e3e70fd9db61bc82469371d04a8d6b762 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Wed, 3 Jun 2015 21:40:35 -0700 Subject: [PATCH 13/59] kafka-1928; Move kafka.network over to using the network classes in org.apache.kafka.common.network; patched by Gwen Shapira; reviewed by Joel Koshy, Jay Kreps, Jiangjie Qin, Guozhang Wang and Jun Rao --- .../kafka/clients/ClusterConnectionStates.java | 78 ++--- .../apache/kafka/clients/CommonClientConfigs.java | 2 + .../org/apache/kafka/clients/InFlightRequests.java | 18 +- .../java/org/apache/kafka/clients/KafkaClient.java | 8 +- .../org/apache/kafka/clients/NetworkClient.java | 59 ++-- .../kafka/clients/consumer/ConsumerConfig.java | 11 +- .../kafka/clients/consumer/KafkaConsumer.java | 3 +- .../clients/consumer/internals/Coordinator.java | 4 +- .../kafka/clients/consumer/internals/Fetcher.java | 10 +- .../kafka/clients/producer/KafkaProducer.java | 3 +- .../kafka/clients/producer/ProducerConfig.java | 6 +- .../kafka/clients/producer/internals/Sender.java | 6 +- .../main/java/org/apache/kafka/common/Node.java | 10 + .../kafka/common/network/ByteBufferReceive.java | 10 +- .../kafka/common/network/ByteBufferSend.java | 20 +- .../common/network/InvalidReceiveException.java | 30 ++ .../org/apache/kafka/common/network/MultiSend.java | 100 ++++++ .../kafka/common/network/NetworkReceive.java | 59 +++- .../apache/kafka/common/network/NetworkSend.java | 2 +- .../org/apache/kafka/common/network/Receive.java | 8 +- .../apache/kafka/common/network/Selectable.java | 16 +- .../org/apache/kafka/common/network/Selector.java | 230 ++++++++------ .../java/org/apache/kafka/common/network/Send.java | 18 +- .../apache/kafka/common/requests/RequestSend.java | 2 +- .../apache/kafka/common/requests/ResponseSend.java | 41 +++ .../java/org/apache/kafka/clients/MockClient.java | 6 +- .../apache/kafka/clients/NetworkClientTest.java | 8 +- .../apache/kafka/common/network/SelectorTest.java | 86 +++--- .../java/org/apache/kafka/test/MockSelector.java | 25 +- core/src/main/scala/kafka/Kafka.scala | 12 +- .../scala/kafka/admin/ConsumerGroupCommand.scala | 2 +- .../scala/kafka/api/ConsumerMetadataRequest.scala | 7 +- .../kafka/api/ControlledShutdownRequest.scala | 9 +- core/src/main/scala/kafka/api/FetchRequest.scala | 2 +- core/src/main/scala/kafka/api/FetchResponse.scala | 73 +++-- .../main/scala/kafka/api/LeaderAndIsrRequest.scala | 12 +- .../main/scala/kafka/api/OffsetCommitRequest.scala | 10 +- .../main/scala/kafka/api/OffsetFetchRequest.scala | 15 +- core/src/main/scala/kafka/api/OffsetRequest.scala | 7 +- .../src/main/scala/kafka/api/ProducerRequest.scala | 7 +- core/src/main/scala/kafka/api/RequestKeys.scala | 4 +- .../main/scala/kafka/api/StopReplicaRequest.scala | 4 +- .../scala/kafka/api/TopicMetadataRequest.scala | 8 +- .../scala/kafka/api/UpdateMetadataRequest.scala | 4 +- core/src/main/scala/kafka/client/ClientUtils.scala | 2 +- .../main/scala/kafka/consumer/SimpleConsumer.scala | 19 +- .../consumer/ZookeeperConsumerConnector.scala | 4 +- .../controller/ControllerChannelManager.scala | 11 +- .../scala/kafka/javaapi/TopicMetadataRequest.scala | 7 +- .../main/scala/kafka/network/BlockingChannel.scala | 21 +- .../kafka/network/BoundedByteBufferReceive.scala | 90 ------ .../kafka/network/BoundedByteBufferSend.scala | 71 ----- .../main/scala/kafka/network/ByteBufferSend.scala | 40 --- core/src/main/scala/kafka/network/Handler.scala | 6 +- .../main/scala/kafka/network/RequestChannel.scala | 35 ++- .../kafka/network/RequestOrResponseSend.scala | 57 ++++ .../main/scala/kafka/network/SocketServer.scala | 334 ++++++++------------- .../main/scala/kafka/network/Transmission.scala | 122 -------- .../main/scala/kafka/producer/SyncProducer.scala | 19 +- core/src/main/scala/kafka/server/KafkaApis.scala | 44 +-- core/src/main/scala/kafka/server/KafkaConfig.scala | 56 +++- core/src/main/scala/kafka/server/KafkaServer.scala | 33 +- .../main/scala/kafka/server/MessageSetSend.scala | 71 ----- .../scala/kafka/tools/ConsumerOffsetChecker.scala | 2 +- .../test/scala/other/kafka/TestOffsetManager.scala | 6 +- .../test/scala/unit/kafka/KafkaConfigTest.scala | 17 +- .../unit/kafka/network/SocketServerTest.scala | 41 +-- .../kafka/server/KafkaConfigConfigDefTest.scala | 8 + 68 files changed, 1075 insertions(+), 1096 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/network/MultiSend.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java delete mode 100755 core/src/main/scala/kafka/network/BoundedByteBufferReceive.scala delete mode 100644 core/src/main/scala/kafka/network/BoundedByteBufferSend.scala delete mode 100644 core/src/main/scala/kafka/network/ByteBufferSend.scala create mode 100644 core/src/main/scala/kafka/network/RequestOrResponseSend.scala delete mode 100644 core/src/main/scala/kafka/network/Transmission.scala delete mode 100644 core/src/main/scala/kafka/server/MessageSetSend.scala 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 da76cc2..9ebda5e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java @@ -21,22 +21,22 @@ import java.util.Map; */ final class ClusterConnectionStates { private final long reconnectBackoffMs; - private final Map nodeState; + private final Map nodeState; public ClusterConnectionStates(long reconnectBackoffMs) { this.reconnectBackoffMs = reconnectBackoffMs; - this.nodeState = new HashMap(); + this.nodeState = new HashMap(); } /** - * Return true iff we can currently initiate a new connection to the given node. This will be the case if we are not + * Return true iff we can currently initiate a new connection. This will be the case if we are not * connected and haven't been connected for at least the minimum reconnection backoff period. - * @param node The node id to check + * @param id The connection id to check * @param now The current time in MS * @return true if we can initiate a new connection */ - public boolean canConnect(int node, long now) { - NodeConnectionState state = nodeState.get(node); + public boolean canConnect(String id, long now) { + NodeConnectionState state = nodeState.get(id); if (state == null) return true; else @@ -45,11 +45,11 @@ final class ClusterConnectionStates { /** * Return true if we are disconnected from the given node and can't re-establish a connection yet - * @param node The node to check + * @param id The connection to check * @param now The current time in ms */ - public boolean isBlackedOut(int node, long now) { - NodeConnectionState state = nodeState.get(node); + public boolean isBlackedOut(String id, long now) { + NodeConnectionState state = nodeState.get(id); if (state == null) return false; else @@ -60,11 +60,11 @@ final class ClusterConnectionStates { * 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 id The connection to check * @param now The current time in ms */ - public long connectionDelay(int node, long now) { - NodeConnectionState state = nodeState.get(node); + public long connectionDelay(String id, long now) { + NodeConnectionState state = nodeState.get(id); if (state == null) return 0; long timeWaited = now - state.lastConnectAttemptMs; if (state.state == ConnectionState.DISCONNECTED) { @@ -77,67 +77,67 @@ final class ClusterConnectionStates { } /** - * Enter the connecting state for the given node. - * @param node The id of the node we are connecting to + * Enter the connecting state for the given connection. + * @param id The id of the connection * @param now The current time. */ - public void connecting(int node, long now) { - nodeState.put(node, new NodeConnectionState(ConnectionState.CONNECTING, now)); + public void connecting(String id, long now) { + nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now)); } /** - * Return true iff we have a connection to the give node - * @param node The id of the node to check + * Return true iff a specific connection is connected + * @param id The id of the connection to check */ - public boolean isConnected(int node) { - NodeConnectionState state = nodeState.get(node); + public boolean isConnected(String id) { + NodeConnectionState state = nodeState.get(id); return state != null && state.state == ConnectionState.CONNECTED; } /** - * Return true iff we are in the process of connecting to the given node - * @param node The id of the node + * Return true iff we are in the process of connecting + * @param id The id of the connection */ - public boolean isConnecting(int node) { - NodeConnectionState state = nodeState.get(node); + public boolean isConnecting(String id) { + NodeConnectionState state = nodeState.get(id); return state != null && state.state == ConnectionState.CONNECTING; } /** - * Enter the connected state for the given node - * @param node The node we have connected to + * Enter the connected state for the given connection + * @param id The connection identifier */ - public void connected(int node) { - NodeConnectionState nodeState = nodeState(node); + public void connected(String id) { + NodeConnectionState nodeState = nodeState(id); nodeState.state = ConnectionState.CONNECTED; } /** * Enter the disconnected state for the given node - * @param node The node we have disconnected from + * @param id The connection we have disconnected */ - public void disconnected(int node) { - NodeConnectionState nodeState = nodeState(node); + public void disconnected(String id) { + NodeConnectionState nodeState = nodeState(id); nodeState.state = ConnectionState.DISCONNECTED; } /** - * Get the state of our connection to the given node - * @param node The id of the node + * Get the state of a given connection + * @param id The id of the connection * @return The state of our connection */ - public ConnectionState connectionState(int node) { - return nodeState(node).state; + public ConnectionState connectionState(String id) { + return nodeState(id).state; } /** * Get the state of a given node - * @param node The node to fetch the state for + * @param id The connection to fetch the state for */ - private NodeConnectionState nodeState(int node) { - NodeConnectionState state = this.nodeState.get(node); + private NodeConnectionState nodeState(String id) { + NodeConnectionState state = this.nodeState.get(id); if (state == null) - throw new IllegalStateException("No entry found for node " + node); + throw new IllegalStateException("No entry found for connection " + id); return state; } diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index cf32e4e..2c421f4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -55,4 +55,6 @@ public class CommonClientConfigs { 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."; + public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = "connections.max.idle.ms"; + public static final String CONNECTIONS_MAX_IDLE_MS_DOC = "Close idle connections after the number of milliseconds specified by this config."; } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java index 936487b..15d00d4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java +++ b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java @@ -24,14 +24,14 @@ import java.util.Map; final class InFlightRequests { private final int maxInFlightRequestsPerConnection; - private final Map> requests = new HashMap>(); + private final Map> requests = new HashMap>(); public InFlightRequests(int maxInFlightRequestsPerConnection) { this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection; } /** - * Add the given request to the queue for the node it was directed to + * Add the given request to the queue for the connection it was directed to */ public void add(ClientRequest request) { Deque reqs = this.requests.get(request.request().destination()); @@ -45,7 +45,7 @@ final class InFlightRequests { /** * Get the request queue for the given node */ - private Deque requestQueue(int node) { + private Deque requestQueue(String node) { Deque reqs = requests.get(node); if (reqs == null || reqs.isEmpty()) throw new IllegalStateException("Response from server for which there are no in-flight requests."); @@ -55,7 +55,7 @@ final class InFlightRequests { /** * Get the oldest request (the one that that will be completed next) for the given node */ - public ClientRequest completeNext(int node) { + public ClientRequest completeNext(String node) { return requestQueue(node).pollLast(); } @@ -63,7 +63,7 @@ final class InFlightRequests { * Get the last request we sent to the given node (but don't remove it from the queue) * @param node The node id */ - public ClientRequest lastSent(int node) { + public ClientRequest lastSent(String node) { return requestQueue(node).peekFirst(); } @@ -72,7 +72,7 @@ final class InFlightRequests { * @param node The node the request was sent to * @return The request */ - public ClientRequest completeLastSent(int node) { + public ClientRequest completeLastSent(String node) { return requestQueue(node).pollFirst(); } @@ -82,7 +82,7 @@ final class InFlightRequests { * @param node Node in question * @return true iff we have no requests still being sent to the given node */ - public boolean canSendMore(int node) { + public boolean canSendMore(String node) { Deque queue = requests.get(node); return queue == null || queue.isEmpty() || (queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection); @@ -93,7 +93,7 @@ final class InFlightRequests { * @param node The node * @return The request count. */ - public int inFlightRequestCount(int node) { + public int inFlightRequestCount(String node) { Deque queue = requests.get(node); return queue == null ? 0 : queue.size(); } @@ -114,7 +114,7 @@ final class InFlightRequests { * @param node The node * @return All the in-flight requests for that node that have been removed */ - public Iterable clearAll(int node) { + public Iterable clearAll(String node) { Deque reqs = requests.get(node); if (reqs == null) { return Collections.emptyList(); 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 1311f85..7ab2503 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -81,13 +81,13 @@ public interface KafkaClient extends Closeable { public List poll(long timeout, long now); /** - * Complete all in-flight requests for a given node + * Complete all in-flight requests for a given connection * - * @param node The node to complete requests for + * @param id The connection 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); + public List completeAll(String id, long now); /** * Complete all in-flight requests @@ -117,7 +117,7 @@ public interface KafkaClient extends Closeable { * * @param nodeId The id of the node */ - public int inFlightRequestCount(int nodeId); + public int inFlightRequestCount(String nodeId); /** * Generate a request header for the next request 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 435fbb5..48fe796 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -22,8 +22,8 @@ import java.util.Set; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.network.NetworkReceive; -import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Selectable; +import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; @@ -111,7 +111,7 @@ public class NetworkClient implements KafkaClient { if (isReady(node, now)) return true; - if (connectionStates.canConnect(node.id(), now)) + if (connectionStates.canConnect(node.idString(), now)) // if we are interested in sending to a node and we don't have a connection to it, initiate one initiateConnect(node, now); @@ -129,7 +129,7 @@ public class NetworkClient implements KafkaClient { */ @Override public long connectionDelay(Node node, long now) { - return connectionStates.connectionDelay(node.id(), now); + return connectionStates.connectionDelay(node.idString(), now); } /** @@ -142,7 +142,7 @@ public class NetworkClient implements KafkaClient { */ @Override public boolean connectionFailed(Node node) { - return connectionStates.connectionState(node.id()).equals(ConnectionState.DISCONNECTED); + return connectionStates.connectionState(node.idString()).equals(ConnectionState.DISCONNECTED); } /** @@ -154,7 +154,7 @@ public class NetworkClient implements KafkaClient { */ @Override public boolean isReady(Node node, long now) { - int nodeId = node.id(); + String nodeId = node.idString(); 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 @@ -165,11 +165,11 @@ public class NetworkClient implements KafkaClient { } /** - * Are we connected and ready and able to send more requests to the given node? + * Are we connected and ready and able to send more requests to the given connection? * * @param node The node */ - private boolean isSendable(int node) { + private boolean isSendable(String node) { return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node); } @@ -179,7 +179,7 @@ public class NetworkClient implements KafkaClient { * @param node The node to check * @return The connection state */ - public ConnectionState connectionState(int node) { + public ConnectionState connectionState(String node) { return connectionStates.connectionState(node); } @@ -190,7 +190,7 @@ public class NetworkClient implements KafkaClient { */ @Override public void send(ClientRequest request) { - int nodeId = request.request().destination(); + String nodeId = request.request().destination(); if (!isSendable(nodeId)) throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); @@ -252,7 +252,7 @@ public class NetworkClient implements KafkaClient { * @return All the collected responses */ @Override - public List completeAll(int node, long now) { + public List completeAll(String node, long now) { try { this.selector.muteAll(); this.selector.unmute(node); @@ -288,8 +288,8 @@ public class NetworkClient implements KafkaClient { * Get the number of in-flight requests for a given node */ @Override - public int inFlightRequestCount(int nodeId) { - return this.inFlightRequests.inFlightRequestCount(nodeId); + public int inFlightRequestCount(String node) { + return this.inFlightRequests.inFlightRequestCount(node); } /** @@ -334,11 +334,11 @@ public class NetworkClient implements KafkaClient { for (int i = 0; i < nodes.size(); i++) { int idx = Utils.abs((this.nodeIndexOffset + i) % nodes.size()); Node node = nodes.get(idx); - int currInflight = this.inFlightRequests.inFlightRequestCount(node.id()); - if (currInflight == 0 && this.connectionStates.isConnected(node.id())) { + int currInflight = this.inFlightRequests.inFlightRequestCount(node.idString()); + if (currInflight == 0 && this.connectionStates.isConnected(node.idString())) { // if we find an established connection with no in-flight requests we can stop right away return node; - } else if (!this.connectionStates.isBlackedOut(node.id(), now) && currInflight < inflight) { + } else if (!this.connectionStates.isBlackedOut(node.idString(), now) && currInflight < inflight) { // otherwise if this is the best we have found so far, record that inflight = currInflight; found = node; @@ -355,7 +355,7 @@ public class NetworkClient implements KafkaClient { */ private void handleCompletedSends(List responses, long now) { // if no response is expected then when the send is completed, return it - for (NetworkSend send : this.selector.completedSends()) { + for (Send send : this.selector.completedSends()) { ClientRequest request = this.inFlightRequests.lastSent(send.destination()); if (!request.expectResponse()) { this.inFlightRequests.completeLastSent(send.destination()); @@ -372,7 +372,7 @@ public class NetworkClient implements KafkaClient { */ private void handleCompletedReceives(List responses, long now) { for (NetworkReceive receive : this.selector.completedReceives()) { - int source = receive.source(); + String source = receive.source(); ClientRequest req = inFlightRequests.completeNext(source); ResponseHeader header = ResponseHeader.parse(receive.payload()); short apiKey = req.request().header().apiKey(); @@ -412,7 +412,7 @@ public class NetworkClient implements KafkaClient { * @param now The current time */ private void handleDisconnections(List responses, long now) { - for (int node : this.selector.disconnected()) { + for (String node : this.selector.disconnected()) { connectionStates.disconnected(node); log.debug("Node {} disconnected.", node); for (ClientRequest request : this.inFlightRequests.clearAll(node)) { @@ -433,9 +433,9 @@ public class NetworkClient implements KafkaClient { * Record any newly completed connections */ private void handleConnections() { - for (Integer id : this.selector.connected()) { - log.debug("Completed connection to node {}", id); - this.connectionStates.connected(id); + for (String node : this.selector.connected()) { + log.debug("Completed connection to node {}", node); + this.connectionStates.connected(node); } } @@ -451,7 +451,7 @@ public class NetworkClient implements KafkaClient { /** * Create a metadata request for the given topics */ - private ClientRequest metadataRequest(long now, int node, Set topics) { + private ClientRequest metadataRequest(long now, String node, Set topics) { MetadataRequest metadata = new MetadataRequest(new ArrayList(topics)); RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct()); return new ClientRequest(now, true, send, null); @@ -470,15 +470,17 @@ public class NetworkClient implements KafkaClient { this.lastNoNodeAvailableMs = now; return; } + String nodeConnectionId = node.idString(); - if (connectionStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) { + + if (connectionStates.isConnected(nodeConnectionId) && inFlightRequests.canSendMore(nodeConnectionId)) { Set topics = metadata.topics(); this.metadataFetchInProgress = true; - ClientRequest metadataRequest = metadataRequest(now, node.id(), topics); + ClientRequest metadataRequest = metadataRequest(now, nodeConnectionId, topics); log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); this.selector.send(metadataRequest.request()); this.inFlightRequests.add(metadataRequest); - } else if (connectionStates.canConnect(node.id(), now)) { + } else if (connectionStates.canConnect(nodeConnectionId, now)) { // we don't have a connection to this node right now, make one log.debug("Initialize connection to node {} for sending metadata request", node.id()); initiateConnect(node, now); @@ -497,16 +499,17 @@ public class NetworkClient implements KafkaClient { * Initiate a connection to the given node */ private void initiateConnect(Node node, long now) { + String nodeConnectionId = node.idString(); try { log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port()); - this.connectionStates.connecting(node.id(), now); - selector.connect(node.id(), + this.connectionStates.connecting(nodeConnectionId, now); + selector.connect(nodeConnectionId, 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()); + connectionStates.disconnected(nodeConnectionId); /* maybe the problem is our metadata, update it */ metadata.requestUpdate(); log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e); 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 bdff518..1e90524 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 @@ -156,6 +156,9 @@ public class ConsumerConfig extends AbstractConfig { 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."; + /** connections.max.idle.ms */ + public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; + static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, @@ -277,7 +280,13 @@ public class ConsumerConfig extends AbstractConfig { .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, - VALUE_DESERIALIZER_CLASS_DOC); + VALUE_DESERIALIZER_CLASS_DOC) + /* default is set to be a bit lower than the server default (10 min), to avoid both client and server closing connection at same time */ + .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, + Type.LONG, + 9 * 60 * 1000, + Importance.MEDIUM, + CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC); } public static Map addDeserializerToConfig(Map configs, 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 d301be4..d1d1ec1 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 @@ -472,7 +472,8 @@ public class KafkaConsumer implements Consumer { String metricGrpPrefix = "consumer"; Map metricsTags = new LinkedHashMap(); metricsTags.put("client-id", clientId); - this.client = new NetworkClient(new Selector(metrics, time, metricGrpPrefix, metricsTags), + this.client = new NetworkClient( + new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags), this.metadata, clientId, 100, // a fixed large enough value will suffice diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index fac7995..c1496a0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -445,7 +445,7 @@ public final class Coordinator { log.debug("Issuing consumer metadata request to broker {}", node.id()); ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.groupId); - RequestSend send = new RequestSend(node.id(), + RequestSend send = new RequestSend(node.idString(), this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), request.toStruct()); long now = time.milliseconds(); @@ -464,7 +464,7 @@ public final class Coordinator { log.debug("Issuing request ({}: {}) to coordinator {}", api, request, this.consumerCoordinator.id()); RequestHeader header = this.client.nextRequestHeader(api); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request); + RequestSend send = new RequestSend(this.consumerCoordinator.idString(), header, request); return new ClientRequest(now, true, send, handler); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index c5e577f..56281ee 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -124,7 +124,7 @@ public class Fetcher { */ public void initFetches(Cluster cluster, long now) { for (ClientRequest request : createFetchRequests(cluster)) { - Node node = cluster.nodeById(request.request().destination()); + Node node = cluster.nodeById(Integer.parseInt(request.request().destination())); if (client.ready(node, now)) { log.trace("Initiating fetch to node {}: {}", node.id(), request); client.send(request); @@ -209,12 +209,12 @@ public class Fetcher { } else if (this.client.ready(info.leader(), now)) { Node node = info.leader(); ListOffsetRequest request = new ListOffsetRequest(-1, partitions); - RequestSend send = new RequestSend(node.id(), + RequestSend send = new RequestSend(node.idString(), 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); + List responses = this.client.completeAll(node.idString(), now); if (responses.isEmpty()) throw new IllegalStateException("This should not happen."); ClientResponse response = responses.get(responses.size() - 1); @@ -258,7 +258,7 @@ public class Fetcher { 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) { + if (node != null && this.client.inFlightRequestCount(node.idString()) == 0) { Map fetch = fetchable.get(node.id()); if (fetch == null) { fetch = new HashMap(); @@ -274,7 +274,7 @@ public class Fetcher { for (Map.Entry> entry : fetchable.entrySet()) { int nodeId = entry.getKey(); final FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); - RequestSend send = new RequestSend(nodeId, this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct()); + RequestSend send = new RequestSend(Integer.toString(nodeId), this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct()); RequestCompletionHandler handler = new RequestCompletionHandler() { public void onComplete(ClientResponse response) { handleFetchResponse(response, fetch); 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 ded19d8..5a37580 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 @@ -227,7 +227,8 @@ public class KafkaProducer implements Producer { 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, "producer", metricTags), + NetworkClient client = new NetworkClient( + new Selector(config.getLong(ProducerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "producer", metricTags), this.metadata, clientId, config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), 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 023bd2e..aa26420 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 @@ -169,11 +169,13 @@ public class ProducerConfig extends AbstractConfig { 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."; + /** connections.max.idle.ms */ + public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; + /** partitioner.class */ public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class"; private static final String PARTITIONER_CLASS_DOC = "Partitioner class that implements the Partitioner interface."; - static { 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) @@ -223,6 +225,8 @@ public class ProducerConfig extends AbstractConfig { MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_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) + /* default is set to be a bit lower than the server default (10 min), to avoid both client and server closing connection at same time */ + .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, Type.LONG, 9 * 60 * 1000, Importance.MEDIUM, CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) .define(PARTITIONER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.producer.internals.DefaultPartitioner", Importance.MEDIUM, PARTITIONER_CLASS_DOC); } 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 1e943d6..07e65d4 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 @@ -320,7 +320,7 @@ public class Sender implements Runnable { recordsByPartition.put(tp, batch); } ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition); - RequestSend send = new RequestSend(destination, + RequestSend send = new RequestSend(Integer.toString(destination), this.client.nextRequestHeader(ApiKeys.PRODUCE), request.toStruct()); RequestCompletionHandler callback = new RequestCompletionHandler() { @@ -505,10 +505,10 @@ public class Sender implements Runnable { topicErrorSensor.record(count, now); } - public void recordLatency(int node, long latency) { + public void recordLatency(String node, long latency) { long now = time.milliseconds(); this.requestTimeSensor.record(latency, now); - if (node >= 0) { + if (!node.isEmpty()) { String nodeTimeName = "node-" + node + ".latency"; Sensor nodeRequestTime = this.metrics.getSensor(nodeTimeName); if (nodeRequestTime != 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 f4e4186..644cd71 100644 --- a/clients/src/main/java/org/apache/kafka/common/Node.java +++ b/clients/src/main/java/org/apache/kafka/common/Node.java @@ -18,12 +18,14 @@ package org.apache.kafka.common; public class Node { private final int id; + private final String idString; private final String host; private final int port; public Node(int id, String host, int port) { super(); this.id = id; + this.idString = Integer.toString(id); this.host = host; this.port = port; } @@ -40,6 +42,14 @@ public class Node { } /** + * String representation of the node id. + * Typically the integer id is used to serialize over the wire, the string representation is used as an identifier with NetworkClient code + */ + public String idString() { + return idString; + } + + /** * The host name for this node */ public String host() { diff --git a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java index 129ae82..159c301 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java @@ -25,11 +25,11 @@ import java.nio.channels.ScatteringByteChannel; */ public class ByteBufferReceive implements Receive { - private final int source; + private final String source; private final ByteBuffer[] buffers; private int remaining; - public ByteBufferReceive(int source, ByteBuffer... buffers) { + public ByteBufferReceive(String source, ByteBuffer... buffers) { super(); this.source = source; this.buffers = buffers; @@ -38,7 +38,7 @@ public class ByteBufferReceive implements Receive { } @Override - public int source() { + public String source() { return source; } @@ -54,8 +54,4 @@ public class ByteBufferReceive implements Receive { return read; } - public ByteBuffer[] reify() { - return buffers; - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java index c8213e1..df0e6d5 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java @@ -22,12 +22,12 @@ import java.nio.channels.GatheringByteChannel; */ public class ByteBufferSend implements Send { - private final int destination; + private final String destination; protected final ByteBuffer[] buffers; private int remaining; private int size; - public ByteBufferSend(int destination, ByteBuffer... buffers) { + public ByteBufferSend(String destination, ByteBuffer... buffers) { super(); this.destination = destination; this.buffers = buffers; @@ -37,7 +37,7 @@ public class ByteBufferSend implements Send { } @Override - public int destination() { + public String destination() { return destination; } @@ -47,16 +47,7 @@ public class ByteBufferSend implements Send { } @Override - public ByteBuffer[] reify() { - return this.buffers; - } - - @Override - public int remaining() { - return this.remaining; - } - - public int size() { + public long size() { return this.size; } @@ -64,9 +55,8 @@ public class ByteBufferSend implements Send { public long writeTo(GatheringByteChannel channel) throws IOException { long written = channel.write(buffers); if (written < 0) - throw new EOFException("This shouldn't happen."); + throw new EOFException("Wrote negative bytes to channel. This shouldn't happen."); remaining -= written; return written; } - } diff --git a/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java b/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java new file mode 100644 index 0000000..a5bdd62 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.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.common.network; + +import org.apache.kafka.common.KafkaException; + +public class InvalidReceiveException extends KafkaException { + + public InvalidReceiveException(String message) { + super(message); + } + + public InvalidReceiveException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/MultiSend.java b/clients/src/main/java/org/apache/kafka/common/network/MultiSend.java new file mode 100644 index 0000000..0e14a39 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/MultiSend.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.network; + +import org.apache.kafka.common.KafkaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.channels.GatheringByteChannel; +import java.util.Iterator; +import java.util.List; + +/** + * A set of composite sends, sent one after another + */ + +public class MultiSend implements Send { + + private static final Logger log = LoggerFactory.getLogger(MultiSend.class); + private String dest; + private long totalWritten = 0; + private List sends; + private Iterator sendsIterator; + private Send current; + private boolean doneSends = false; + private long size = 0; + + public MultiSend(String dest, List sends) { + this.dest = dest; + this.sends = sends; + this.sendsIterator = sends.iterator(); + nextSendOrDone(); + for (Send send: sends) + this.size += send.size(); + } + + @Override + public long size() { + return size; + } + + @Override + public String destination() { + return dest; + } + + @Override + public boolean completed() { + if (doneSends) { + if (totalWritten != size) + log.error("mismatch in sending bytes over socket; expected: " + size + " actual: " + totalWritten); + return true; + } else { + return false; + } + } + + @Override + public long writeTo(GatheringByteChannel channel) throws IOException { + if (completed()) + throw new KafkaException("This operation cannot be completed on a complete request."); + + int totalWrittenPerCall = 0; + boolean sendComplete = false; + do { + long written = current.writeTo(channel); + totalWritten += written; + totalWrittenPerCall += written; + sendComplete = current.completed(); + if (sendComplete) + nextSendOrDone(); + } while (!completed() && sendComplete); + if (log.isTraceEnabled()) + log.trace("Bytes written as part of multisend call : " + totalWrittenPerCall + "Total bytes written so far : " + totalWritten + "Expected bytes to write : " + size); + return totalWrittenPerCall; + } + + // update current if there's a next Send, mark sends as done if there isn't + private void nextSendOrDone() { + if (sendsIterator.hasNext()) + current = sendsIterator.next(); + else + doneSends = true; + } +} \ No newline at end of file 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 fc0d168..3ca0098 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 @@ -15,6 +15,7 @@ package org.apache.kafka.common.network; import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; import java.nio.channels.ScatteringByteChannel; /** @@ -22,24 +23,42 @@ import java.nio.channels.ScatteringByteChannel; */ public class NetworkReceive implements Receive { - private final int source; + public final static String UNKNOWN_SOURCE = ""; + public final static int UNLIMITED = -1; + + private final String source; private final ByteBuffer size; + private final int maxSize; private ByteBuffer buffer; - public NetworkReceive(int source, ByteBuffer buffer) { + + public NetworkReceive(String source, ByteBuffer buffer) { this.source = source; this.buffer = buffer; this.size = null; + this.maxSize = UNLIMITED; + } + + public NetworkReceive(String source) { + this.source = source; + this.size = ByteBuffer.allocate(4); + this.buffer = null; + this.maxSize = UNLIMITED; } - public NetworkReceive(int source) { + public NetworkReceive(int maxSize, String source) { this.source = source; this.size = ByteBuffer.allocate(4); this.buffer = null; + this.maxSize = maxSize; + } + + public NetworkReceive() { + this(UNKNOWN_SOURCE); } @Override - public int source() { + public String source() { return source; } @@ -48,13 +67,15 @@ public class NetworkReceive implements Receive { return !size.hasRemaining() && !buffer.hasRemaining(); } - @Override - public ByteBuffer[] reify() { - return new ByteBuffer[] {this.buffer}; + public long readFrom(ScatteringByteChannel channel) throws IOException { + return readFromReadableChannel(channel); } - @Override - public long readFrom(ScatteringByteChannel channel) throws IOException { + // Need a method to read from ReadableByteChannel because BlockingChannel requires read with timeout + // See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work + // This can go away after we get rid of BlockingChannel + @Deprecated + public long readFromReadableChannel(ReadableByteChannel channel) throws IOException { int read = 0; if (size.hasRemaining()) { int bytesRead = channel.read(size); @@ -63,10 +84,12 @@ public class NetworkReceive implements Receive { read += bytesRead; if (!size.hasRemaining()) { size.rewind(); - int requestSize = size.getInt(); - if (requestSize < 0) - throw new IllegalStateException("Invalid request (size = " + requestSize + ")"); - this.buffer = ByteBuffer.allocate(requestSize); + int receiveSize = size.getInt(); + if (receiveSize < 0) + throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + ")"); + if (maxSize != UNLIMITED && receiveSize > maxSize) + throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + " larger than " + maxSize + ")"); + this.buffer = ByteBuffer.allocate(receiveSize); } } if (buffer != null) { @@ -83,4 +106,14 @@ public class NetworkReceive implements Receive { return this.buffer; } + // Used only by BlockingChannel, so we may be able to get rid of this when/if we get rid of BlockingChannel + @Deprecated + public long readCompletely(ReadableByteChannel channel) throws IOException { + int totalRead = 0; + while (!complete()) { + totalRead += readFromReadableChannel(channel); + } + return totalRead; + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java b/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java index 68327cd..49964b0 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java +++ b/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java @@ -23,7 +23,7 @@ import java.nio.ByteBuffer; */ public class NetworkSend extends ByteBufferSend { - public NetworkSend(int destination, ByteBuffer... buffers) { + public NetworkSend(String destination, ByteBuffer... buffers) { super(destination, sizeDelimit(buffers)); } diff --git a/clients/src/main/java/org/apache/kafka/common/network/Receive.java b/clients/src/main/java/org/apache/kafka/common/network/Receive.java index 4e33078..4b14431 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Receive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Receive.java @@ -17,7 +17,6 @@ package org.apache.kafka.common.network; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.channels.ScatteringByteChannel; /** @@ -28,7 +27,7 @@ public interface Receive { /** * The numeric id of the source from which we are receiving data. */ - public int source(); + public String source(); /** * Are we done receiving data? @@ -36,11 +35,6 @@ public interface Receive { public boolean complete(); /** - * Turn this receive into ByteBuffer instances, if possible (otherwise returns null). - */ - public ByteBuffer[] reify(); - - /** * Read bytes into this receive from the given channel * @param channel The channel to read from * @return The number of bytes read 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 b5f8d83..618a0fa 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 @@ -29,12 +29,12 @@ public interface Selectable { * @param receiveBufferSize The receive buffer for the socket * @throws IOException If we cannot begin connecting */ - public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException; + public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException; /** * Begin disconnecting the connection identified by the given id */ - public void disconnect(int id); + public void disconnect(String id); /** * Wakeup this selector if it is blocked on I/O @@ -50,7 +50,7 @@ public interface Selectable { * Queue the given request for sending in the subsequent {@poll(long)} calls * @param send The request to send */ - public void send(NetworkSend send); + public void send(Send send); /** * Do I/O. Reads, writes, connection establishment, etc. @@ -62,7 +62,7 @@ public interface Selectable { /** * The list of sends that completed on the last {@link #poll(long, List) poll()} call. */ - public List completedSends(); + public List completedSends(); /** * The list of receives that completed on the last {@link #poll(long, List) poll()} call. @@ -73,25 +73,25 @@ public interface Selectable { * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()} * call. */ - public List disconnected(); + public List disconnected(); /** * The list of connections that completed their connection on the last {@link #poll(long, List) poll()} * call. */ - public List connected(); + public List connected(); /** * Disable reads from the given connection * @param id The id for the connection */ - public void mute(int id); + public void mute(String id); /** * Re-enable reads from the given connection * @param id The id for the connection */ - public void unmute(int id); + public void unmute(String id); /** * Disable reads from all connections 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 57de058..effb1e6 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 @@ -17,17 +17,8 @@ import java.io.IOException; import java.net.ConnectException; import java.net.InetSocketAddress; import java.net.Socket; -import java.nio.channels.CancelledKeyException; -import java.nio.channels.SelectionKey; -import java.nio.channels.SocketChannel; -import java.nio.channels.UnresolvedAddressException; -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; +import java.nio.channels.*; +import java.util.*; import java.util.concurrent.TimeUnit; import org.apache.kafka.common.KafkaException; @@ -40,20 +31,21 @@ 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.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * A selector interface for doing non-blocking multi-connection network I/O. + * A nioSelector interface for doing non-blocking multi-connection network I/O. *

      * This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and * responses. *

      - * A connection can be added to the selector associated with an integer id by doing + * A connection can be added to the nioSelector associated with an integer id by doing * *

      - * selector.connect(42, new InetSocketAddress("google.com", server.port), 64000, 64000);
      + * nioSelector.connect(42, new InetSocketAddress("google.com", server.port), 64000, 64000);
        * 
      * * The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating @@ -64,10 +56,10 @@ import org.slf4j.LoggerFactory; * *
        * List<NetworkRequest> requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
      - * selector.poll(TIMEOUT_MS, requestsToSend);
      + * nioSelector.poll(TIMEOUT_MS, requestsToSend);
        * 
      * - * The selector maintains several lists that are reset by each call to poll() which are available via + * The nioSelector maintains several lists that are reset by each call to poll() which are available via * various getters. These are reset by each call to poll(). * * This class is not thread safe! @@ -76,41 +68,59 @@ public class Selector implements Selectable { private static final Logger log = LoggerFactory.getLogger(Selector.class); - private final java.nio.channels.Selector selector; - private final Map keys; - private final List completedSends; + private final java.nio.channels.Selector nioSelector; + private final Map keys; + private final List completedSends; private final List completedReceives; - private final List disconnected; - private final List connected; - private final List failedSends; + private final List disconnected; + private final List connected; + private final List failedSends; private final Time time; private final SelectorMetrics sensors; private final String metricGrpPrefix; private final Map metricTags; + private final Map lruConnections; + private final long connectionsMaxIdleNanos; + private final int maxReceiveSize; + private final boolean metricsPerConnection; + private long currentTimeNanos; + private long nextIdleCloseCheckTime; + /** - * Create a new selector + * Create a new nioSelector */ - public Selector(Metrics metrics, Time time, String metricGrpPrefix, Map metricTags) { + public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, Time time, String metricGrpPrefix, Map metricTags, boolean metricsPerConnection) { try { - this.selector = java.nio.channels.Selector.open(); + this.nioSelector = java.nio.channels.Selector.open(); } catch (IOException e) { throw new KafkaException(e); } + this.maxReceiveSize = maxReceiveSize; + this.connectionsMaxIdleNanos = connectionMaxIdleMs * 1000 * 1000; this.time = time; this.metricGrpPrefix = metricGrpPrefix; this.metricTags = metricTags; - this.keys = new HashMap(); - this.completedSends = new ArrayList(); + this.keys = new HashMap(); + this.completedSends = new ArrayList(); this.completedReceives = new ArrayList(); - this.connected = new ArrayList(); - this.disconnected = new ArrayList(); - this.failedSends = new ArrayList(); + this.connected = new ArrayList(); + this.disconnected = new ArrayList(); + this.failedSends = new ArrayList(); this.sensors = new SelectorMetrics(metrics); + // initial capacity and load factor are default, we set them explicitly because we want to set accessOrder = true + this.lruConnections = new LinkedHashMap(16, .75F, true); + currentTimeNanos = new SystemTime().nanoseconds(); + nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos; + this.metricsPerConnection = metricsPerConnection; + } + + public Selector(long connectionMaxIdleMS, Metrics metrics, Time time, String metricGrpPrefix, Map metricTags) { + this(NetworkReceive.UNLIMITED, connectionMaxIdleMS, metrics, time, metricGrpPrefix, metricTags, true); } /** - * Begin connecting to the given address and add the connection to this selector associated with the given id + * Begin connecting to the given address and add the connection to this nioSelector associated with the given id * number. *

      * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)} @@ -123,7 +133,7 @@ public class Selector implements Selectable { * @throws IOException if DNS resolution fails on the hostname or if the broker is down */ @Override - public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { + public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { if (this.keys.containsKey(id)) throw new IllegalStateException("There is already a connection for id " + id); @@ -143,7 +153,18 @@ public class Selector implements Selectable { channel.close(); throw e; } - SelectionKey key = channel.register(this.selector, SelectionKey.OP_CONNECT); + SelectionKey key = channel.register(this.nioSelector, SelectionKey.OP_CONNECT); + key.attach(new Transmissions(id)); + this.keys.put(id, key); + } + + /** + * Register the nioSelector with an existing channel + * Use this on server-side, when a connection is accepted by a different thread but processed by the Selector + * Note that we are not checking if the connection id is valid - since the connection already exists + */ + public void register(String id, SocketChannel channel) throws ClosedChannelException { + SelectionKey key = channel.register(nioSelector, SelectionKey.OP_READ); key.attach(new Transmissions(id)); this.keys.put(id, key); } @@ -153,18 +174,18 @@ public class Selector implements Selectable { * processed until the next {@link #poll(long, List) poll()} call. */ @Override - public void disconnect(int id) { + public void disconnect(String id) { SelectionKey key = this.keys.get(id); if (key != null) key.cancel(); } /** - * Interrupt the selector if it is blocked waiting to do I/O. + * Interrupt the nioSelector if it is blocked waiting to do I/O. */ @Override public void wakeup() { - this.selector.wakeup(); + this.nioSelector.wakeup(); } /** @@ -172,12 +193,14 @@ public class Selector implements Selectable { */ @Override public void close() { - for (SelectionKey key : this.selector.keys()) - close(key); + List connections = new LinkedList(keys.keySet()); + for (String id: connections) + close(id); + try { - this.selector.close(); + this.nioSelector.close(); } catch (IOException e) { - log.error("Exception closing selector:", e); + log.error("Exception closing nioSelector:", e); } } @@ -185,7 +208,7 @@ public class Selector implements Selectable { * Queue the given request for sending in the subsequent {@poll(long)} calls * @param send The request to send */ - public void send(NetworkSend send) { + public void send(Send send) { SelectionKey key = keyForId(send.destination()); Transmissions transmissions = transmissions(key); if (transmissions.hasSend()) @@ -194,7 +217,7 @@ public class Selector implements Selectable { try { key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); } catch (CancelledKeyException e) { - close(key); + close(transmissions.id); this.failedSends.add(send.destination()); } } @@ -220,10 +243,11 @@ public class Selector implements Selectable { long startSelect = time.nanoseconds(); int readyKeys = select(timeout); long endSelect = time.nanoseconds(); + currentTimeNanos = endSelect; this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds()); if (readyKeys > 0) { - Set keys = this.selector.selectedKeys(); + Set keys = this.nioSelector.selectedKeys(); Iterator iter = keys.iterator(); while (iter.hasNext()) { SelectionKey key = iter.next(); @@ -232,8 +256,9 @@ public class Selector implements Selectable { Transmissions transmissions = transmissions(key); SocketChannel channel = channel(key); - // register all per-broker metrics at once - sensors.maybeRegisterNodeMetrics(transmissions.id); + // register all per-connection metrics at once + sensors.maybeRegisterConnectionMetrics(transmissions.id); + lruConnections.put(transmissions.id, currentTimeNanos); try { /* complete any connections that have finished their handshake */ @@ -247,8 +272,14 @@ public class Selector implements Selectable { /* read from any connections that have readable data */ if (key.isReadable()) { if (!transmissions.hasReceive()) - transmissions.receive = new NetworkReceive(transmissions.id); - transmissions.receive.readFrom(channel); + transmissions.receive = new NetworkReceive(maxReceiveSize, transmissions.id); + try { + transmissions.receive.readFrom(channel); + } catch (InvalidReceiveException e) { + log.error("Invalid data received from " + transmissions.id + " closing connection", e); + close(transmissions.id); + throw e; + } if (transmissions.receive.complete()) { transmissions.receive.payload().rewind(); this.completedReceives.add(transmissions.receive); @@ -260,7 +291,7 @@ public class Selector implements Selectable { /* write to any sockets that have space in their buffer and for which we have data */ if (key.isWritable()) { transmissions.send.writeTo(channel); - if (transmissions.send.remaining() <= 0) { + if (transmissions.send.completed()) { this.completedSends.add(transmissions.send); this.sensors.recordBytesSent(transmissions.id, transmissions.send.size()); transmissions.clearSend(); @@ -270,7 +301,7 @@ public class Selector implements Selectable { /* cancel any defunct sockets */ if (!key.isValid()) { - close(key); + close(transmissions.id); this.disconnected.add(transmissions.id); } } catch (IOException e) { @@ -279,15 +310,16 @@ public class Selector implements Selectable { log.info("Connection {} disconnected", desc); else log.warn("Error in I/O with connection to {}", desc, e); - close(key); + close(transmissions.id); this.disconnected.add(transmissions.id); } } } long endIo = time.nanoseconds(); this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); + maybeCloseOldestConnection(); } - + private String socketDescription(SocketChannel channel) { Socket socket = channel.socket(); if (socket == null) @@ -299,7 +331,7 @@ public class Selector implements Selectable { } @Override - public List completedSends() { + public List completedSends() { return this.completedSends; } @@ -309,17 +341,17 @@ public class Selector implements Selectable { } @Override - public List disconnected() { + public List disconnected() { return this.disconnected; } @Override - public List connected() { + public List connected() { return this.connected; } @Override - public void mute(int id) { + public void mute(String id) { mute(this.keyForId(id)); } @@ -328,7 +360,7 @@ public class Selector implements Selectable { } @Override - public void unmute(int id) { + public void unmute(String id) { unmute(this.keyForId(id)); } @@ -348,6 +380,25 @@ public class Selector implements Selectable { unmute(key); } + private void maybeCloseOldestConnection() { + if (currentTimeNanos > nextIdleCloseCheckTime) { + if (lruConnections.isEmpty()) { + nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos; + } else { + Map.Entry oldestConnectionEntry = lruConnections.entrySet().iterator().next(); + Long connectionLastActiveTime = oldestConnectionEntry.getValue(); + nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos; + if (currentTimeNanos > nextIdleCloseCheckTime) { + String connectionId = oldestConnectionEntry.getKey(); + if (log.isTraceEnabled()) + log.trace("About to close the idle connection from " + connectionId + + " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis"); + close(connectionId); + } + } + } + } + /** * Clear the results from the prior poll */ @@ -369,17 +420,19 @@ public class Selector implements Selectable { */ private int select(long ms) throws IOException { if (ms == 0L) - return this.selector.selectNow(); + return this.nioSelector.selectNow(); else if (ms < 0L) - return this.selector.select(); + return this.nioSelector.select(); else - return this.selector.select(ms); + return this.nioSelector.select(ms); } /** * Begin closing this connection */ - private void close(SelectionKey key) { + public void close(String id) { + SelectionKey key = keyForId(id); + lruConnections.remove(id); SocketChannel channel = channel(key); Transmissions trans = transmissions(key); if (trans != null) { @@ -401,10 +454,10 @@ public class Selector implements Selectable { /** * Get the selection key associated with this numeric id */ - private SelectionKey keyForId(int id) { + private SelectionKey keyForId(String id) { SelectionKey key = this.keys.get(id); if (key == null) - throw new IllegalStateException("Attempt to write to socket for which there is no open connection."); + throw new IllegalStateException("Attempt to write to socket for which there is no open connection. Connection id " + id + " existing connections " + keys.keySet().toString()); return key; } @@ -426,11 +479,11 @@ public class Selector implements Selectable { * The id and in-progress send and receive associated with a connection */ private static class Transmissions { - public int id; - public NetworkSend send; + public String id; + public Send send; public NetworkReceive receive; - public Transmissions(int id) { + public Transmissions(String id) { this.id = id; } @@ -464,20 +517,27 @@ public class Selector implements Selectable { public SelectorMetrics(Metrics metrics) { this.metrics = metrics; String metricGrpName = metricGrpPrefix + "-metrics"; + StringBuilder tagsSuffix = new StringBuilder(); + + for (Map.Entry tag: metricTags.entrySet()) { + tagsSuffix.append(tag.getKey()); + tagsSuffix.append("-"); + tagsSuffix.append(tag.getValue()); + } - this.connectionClosed = this.metrics.sensor("connections-closed"); + this.connectionClosed = this.metrics.sensor("connections-closed:" + tagsSuffix.toString()); 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 = this.metrics.sensor("connections-created:" + tagsSuffix.toString()); 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"); + this.bytesTransferred = this.metrics.sensor("bytes-sent-received:" + tagsSuffix.toString()); 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 = this.metrics.sensor("bytes-sent:" + tagsSuffix.toString(), bytesTransferred); 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); @@ -487,13 +547,13 @@ public class Selector implements Selectable { 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 = this.metrics.sensor("bytes-received:" + tagsSuffix.toString(), bytesTransferred); 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 = this.metrics.sensor("select-time:" + tagsSuffix.toString()); 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); @@ -501,7 +561,7 @@ public class Selector implements Selectable { 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 = this.metrics.sensor("io-time:" + tagsSuffix.toString()); 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); @@ -515,17 +575,17 @@ public class Selector implements Selectable { }); } - public void maybeRegisterNodeMetrics(int node) { - if (node >= 0) { - // if one sensor of the metrics has been registered for the node, + public void maybeRegisterConnectionMetrics(String connectionId) { + if (!connectionId.isEmpty() && metricsPerConnection) { + // if one sensor of the metrics has been registered for the connection, // then all other sensors should have been registered; and vice versa - String nodeRequestName = "node-" + node + ".bytes-sent"; + String nodeRequestName = "node-" + connectionId + ".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); + tags.put("node-id", "node-" + connectionId); nodeRequest = this.metrics.sensor(nodeRequestName); MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags); @@ -537,14 +597,14 @@ public class Selector implements Selectable { 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"; + String nodeResponseName = "node-" + connectionId + ".bytes-received"; Sensor nodeResponse = this.metrics.sensor(nodeResponseName); 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"; + String nodeTimeName = "node-" + connectionId + ".latency"; Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName); metricName = new MetricName("request-latency-avg", metricGrpName, tags); nodeRequestTime.add(metricName, new Avg()); @@ -554,22 +614,22 @@ public class Selector implements Selectable { } } - public void recordBytesSent(int node, int bytes) { + public void recordBytesSent(String connectionId, long bytes) { long now = time.milliseconds(); this.bytesSent.record(bytes, now); - if (node >= 0) { - String nodeRequestName = "node-" + node + ".bytes-sent"; + if (!connectionId.isEmpty()) { + String nodeRequestName = "node-" + connectionId + ".bytes-sent"; Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); if (nodeRequest != null) nodeRequest.record(bytes, now); } } - public void recordBytesReceived(int node, int bytes) { + public void recordBytesReceived(String connection, int bytes) { long now = time.milliseconds(); this.bytesReceived.record(bytes, now); - if (node >= 0) { - String nodeRequestName = "node-" + node + ".bytes-received"; + if (!connection.isEmpty()) { + String nodeRequestName = "node-" + connection + ".bytes-received"; Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); if (nodeRequest != null) nodeRequest.record(bytes, now); diff --git a/clients/src/main/java/org/apache/kafka/common/network/Send.java b/clients/src/main/java/org/apache/kafka/common/network/Send.java index 5d321a0..8f6daad 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Send.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Send.java @@ -13,7 +13,6 @@ package org.apache.kafka.common.network; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.channels.GatheringByteChannel; /** @@ -24,12 +23,7 @@ public interface Send { /** * The numeric id for the destination of this send */ - public int destination(); - - /** - * The number of bytes remaining to send - */ - public int remaining(); + public String destination(); /** * Is this send complete? @@ -37,11 +31,6 @@ public interface Send { public boolean completed(); /** - * An optional method to turn this send into an array of ByteBuffers if possible (otherwise returns null) - */ - public ByteBuffer[] reify(); - - /** * Write some as-yet unwritten bytes from this send to the provided channel. It may take multiple calls for the send * to be completely written * @param channel The channel to write to @@ -50,4 +39,9 @@ public interface Send { */ public long writeTo(GatheringByteChannel channel) throws IOException; + /** + * Size of the send + */ + public long size(); + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java index 27cbf39..3fec60b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java @@ -25,7 +25,7 @@ public class RequestSend extends NetworkSend { private final RequestHeader header; private final Struct body; - public RequestSend(int destination, RequestHeader header, Struct body) { + public RequestSend(String destination, RequestHeader header, Struct body) { super(destination, serialize(header, body)); this.header = header; this.body = body; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java new file mode 100644 index 0000000..12b06d1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java @@ -0,0 +1,41 @@ +/** + * 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 org.apache.kafka.common.network.NetworkSend; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class ResponseSend extends NetworkSend { + + public ResponseSend(String destination, ResponseHeader header, Struct body) { + super(destination, serialize(header, body)); + } + + public ResponseSend(String destination, ResponseHeader header, AbstractRequestResponse response) { + this(destination, header, response.toStruct()); + } + + private static ByteBuffer serialize(ResponseHeader header, Struct body) { + ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf()); + header.writeTo(buffer); + body.writeTo(buffer); + buffer.rewind(); + return buffer; + } +} 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 5e3fab1..d9c97e9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -78,7 +78,7 @@ public class MockClient implements KafkaClient { return false; } - public void disconnect(Integer node) { + public void disconnect(String node) { Iterator iter = requests.iterator(); while (iter.hasNext()) { ClientRequest request = iter.next(); @@ -115,7 +115,7 @@ public class MockClient implements KafkaClient { } @Override - public List completeAll(int node, long now) { + public List completeAll(String node, long now) { return completeAll(now); } @@ -158,7 +158,7 @@ public class MockClient implements KafkaClient { } @Override - public int inFlightRequestCount(int nodeId) { + public int inFlightRequestCount(String nodeId) { return requests.size(); } 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 8b27889..43238ce 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -65,7 +65,7 @@ public class NetworkClientTest { client.poll(1, time.milliseconds()); selector.clear(); assertTrue("Now the client is ready", client.ready(node, time.milliseconds())); - selector.disconnect(node.id()); + selector.disconnect(node.idString()); client.poll(1, time.milliseconds()); selector.clear(); assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds())); @@ -74,7 +74,7 @@ public class NetworkClientTest { @Test(expected = IllegalStateException.class) public void testSendToUnreadyNode() { - RequestSend send = new RequestSend(5, + RequestSend send = new RequestSend("5", client.nextRequestHeader(ApiKeys.METADATA), new MetadataRequest(Arrays.asList("test")).toStruct()); ClientRequest request = new ClientRequest(time.milliseconds(), false, send, null); @@ -86,7 +86,7 @@ public class NetworkClientTest { 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()); + RequestSend send = new RequestSend(node.idString(), reqHeader, produceRequest.toStruct()); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = new ClientRequest(time.milliseconds(), true, send, handler); awaitReady(client, node); @@ -101,7 +101,7 @@ public class NetworkClientTest { respHeader.writeTo(buffer); resp.writeTo(buffer); buffer.flip(); - selector.completeReceive(new NetworkReceive(node.id(), buffer)); + selector.completeReceive(new NetworkReceive(node.idString(), buffer)); List responses = client.poll(1, time.milliseconds()); assertEquals(1, responses.size()); assertTrue("The handler should have executed.", handler.executed); 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 d5b306b..d23b4b6 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 @@ -22,10 +22,7 @@ import java.net.InetSocketAddress; import java.net.ServerSocket; import java.net.Socket; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; +import java.util.*; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; @@ -49,7 +46,7 @@ public class SelectorTest { public void setup() throws Exception { this.server = new EchoServer(); this.server.start(); - this.selector = new Selector(new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap()); + this.selector = new Selector(5000, new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap()); } @After @@ -63,7 +60,7 @@ public class SelectorTest { */ @Test public void testServerDisconnect() throws Exception { - int node = 0; + String node = "0"; // connect and do a simple request blockingConnect(node); @@ -84,7 +81,7 @@ public class SelectorTest { */ @Test public void testClientDisconnect() throws Exception { - int node = 0; + String node = "0"; blockingConnect(node); selector.disconnect(node); selector.send(createSend(node, "hello1")); @@ -101,7 +98,7 @@ public class SelectorTest { */ @Test(expected = IllegalStateException.class) public void testCantSendWithInProgress() throws Exception { - int node = 0; + String node = "0"; blockingConnect(node); selector.send(createSend(node, "test1")); selector.send(createSend(node, "test2")); @@ -113,7 +110,7 @@ public class SelectorTest { */ @Test(expected = IllegalStateException.class) public void testCantSendWithoutConnecting() throws Exception { - selector.send(createSend(0, "test")); + selector.send(createSend("0", "test")); selector.poll(1000L); } @@ -122,7 +119,7 @@ public class SelectorTest { */ @Test(expected = IOException.class) public void testNoRouteToHost() throws Exception { - selector.connect(0, new InetSocketAddress("asdf.asdf.dsc", server.port), BUFFER_SIZE, BUFFER_SIZE); + selector.connect("0", new InetSocketAddress("asdf.asdf.dsc", server.port), BUFFER_SIZE, BUFFER_SIZE); } /** @@ -130,7 +127,7 @@ public class SelectorTest { */ @Test public void testConnectionRefused() throws Exception { - int node = 0; + String node = "0"; ServerSocket nonListeningSocket = new ServerSocket(0); int nonListeningPort = nonListeningSocket.getLocalPort(); selector.connect(node, new InetSocketAddress("localhost", nonListeningPort), BUFFER_SIZE, BUFFER_SIZE); @@ -151,14 +148,15 @@ public class SelectorTest { // create connections InetSocketAddress addr = new InetSocketAddress("localhost", server.port); for (int i = 0; i < conns; i++) - selector.connect(i, addr, BUFFER_SIZE, BUFFER_SIZE); - + selector.connect(Integer.toString(i), addr, BUFFER_SIZE, BUFFER_SIZE); // send echo requests and receive responses - int[] requests = new int[conns]; - int[] responses = new int[conns]; + Map requests = new HashMap(); + Map responses = new HashMap(); int responseCount = 0; - for (int i = 0; i < conns; i++) - selector.send(createSend(i, i + "-" + 0)); + for (int i = 0; i < conns; i++) { + String node = Integer.toString(i); + selector.send(createSend(node, node + "-0")); + } // loop until we complete all requests while (responseCount < conns * reqs) { @@ -171,19 +169,27 @@ public class SelectorTest { for (NetworkReceive receive : selector.completedReceives()) { String[] pieces = asString(receive).split("-"); assertEquals("Should be in the form 'conn-counter'", 2, pieces.length); - assertEquals("Check the source", receive.source(), Integer.parseInt(pieces[0])); + assertEquals("Check the source", receive.source(), pieces[0]); assertEquals("Check that the receive has kindly been rewound", 0, receive.payload().position()); - assertEquals("Check the request counter", responses[receive.source()], Integer.parseInt(pieces[1])); - responses[receive.source()]++; // increment the expected counter + if (responses.containsKey(receive.source())) { + assertEquals("Check the request counter", (int) responses.get(receive.source()), Integer.parseInt(pieces[1])); + responses.put(receive.source(), responses.get(receive.source()) + 1); + } else { + assertEquals("Check the request counter", 0, Integer.parseInt(pieces[1])); + responses.put(receive.source(), 1); + } responseCount++; } // prepare new sends for the next round - for (NetworkSend send : selector.completedSends()) { - int dest = send.destination(); - requests[dest]++; - if (requests[dest] < reqs) - selector.send(createSend(dest, dest + "-" + requests[dest])); + for (Send send : selector.completedSends()) { + String dest = send.destination(); + if (requests.containsKey(dest)) + requests.put(dest, requests.get(dest) + 1); + else + requests.put(dest, 1); + if (requests.get(dest) < reqs) + selector.send(createSend(dest, dest + "-" + requests.get(dest))); } } } @@ -193,7 +199,7 @@ public class SelectorTest { */ @Test public void testSendLargeRequest() throws Exception { - int node = 0; + String node = "0"; blockingConnect(node); String big = TestUtils.randomString(10 * BUFFER_SIZE); assertEquals(big, blockingRequest(node, big)); @@ -204,41 +210,41 @@ public class SelectorTest { */ @Test public void testEmptyRequest() throws Exception { - int node = 0; + String node = "0"; blockingConnect(node); assertEquals("", blockingRequest(node, "")); } @Test(expected = IllegalStateException.class) public void testExistingConnectionId() throws IOException { - blockingConnect(0); - blockingConnect(0); + blockingConnect("0"); + blockingConnect("0"); } @Test public void testMute() throws Exception { - blockingConnect(0); - blockingConnect(1); + blockingConnect("0"); + blockingConnect("1"); - selector.send(createSend(0, "hello")); - selector.send(createSend(1, "hi")); + selector.send(createSend("0", "hello")); + selector.send(createSend("1", "hi")); - selector.mute(1); + 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()); + assertEquals("The response should not be from the muted node", "0", selector.completedReceives().get(0).source()); - selector.unmute(1); + 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()); + 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 { + private String blockingRequest(String node, String s) throws IOException { selector.send(createSend(node, s)); selector.poll(1000L); while (true) { @@ -250,13 +256,13 @@ public class SelectorTest { } /* connect and wait for the connection to complete */ - private void blockingConnect(int node) throws IOException { + private void blockingConnect(String node) throws IOException { selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE); while (!selector.connected().contains(node)) selector.poll(10000L); } - private NetworkSend createSend(int node, String s) { + private NetworkSend createSend(String node, String s) { return new NetworkSend(node, ByteBuffer.wrap(s.getBytes())); } 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 ea89b06..51eb9d1 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockSelector.java +++ b/clients/src/test/java/org/apache/kafka/test/MockSelector.java @@ -20,6 +20,7 @@ import java.util.List; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Selectable; +import org.apache.kafka.common.network.Send; import org.apache.kafka.common.utils.Time; /** @@ -28,23 +29,23 @@ import org.apache.kafka.common.utils.Time; public class MockSelector implements Selectable { private final Time time; - private final List initiatedSends = new ArrayList(); - private final List completedSends = new ArrayList(); + private final List initiatedSends = new ArrayList(); + private final List completedSends = new ArrayList(); private final List completedReceives = new ArrayList(); - private final List disconnected = new ArrayList(); - private final List connected = new ArrayList(); + private final List disconnected = new ArrayList(); + private final List connected = new ArrayList(); public MockSelector(Time time) { this.time = time; } @Override - public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { + public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { this.connected.add(id); } @Override - public void disconnect(int id) { + public void disconnect(String id) { this.disconnected.add(id); } @@ -64,7 +65,7 @@ public class MockSelector implements Selectable { } @Override - public void send(NetworkSend send) { + public void send(Send send) { this.initiatedSends.add(send); } @@ -76,7 +77,7 @@ public class MockSelector implements Selectable { } @Override - public List completedSends() { + public List completedSends() { return completedSends; } @@ -94,21 +95,21 @@ public class MockSelector implements Selectable { } @Override - public List disconnected() { + public List disconnected() { return disconnected; } @Override - public List connected() { + public List connected() { return connected; } @Override - public void mute(int id) { + public void mute(String id) { } @Override - public void unmute(int id) { + public void unmute(String id) { } @Override diff --git a/core/src/main/scala/kafka/Kafka.scala b/core/src/main/scala/kafka/Kafka.scala index 9efabaa..6af7b80 100755 --- a/core/src/main/scala/kafka/Kafka.scala +++ b/core/src/main/scala/kafka/Kafka.scala @@ -17,6 +17,8 @@ package kafka +import java.util.Properties + import scala.collection.JavaConversions._ import joptsimple.OptionParser import metrics.KafkaMetricsReporter @@ -26,7 +28,7 @@ import org.apache.kafka.common.utils.Utils object Kafka extends Logging { - def getKafkaConfigFromArgs(args: Array[String]): KafkaConfig = { + def getPropsFromArgs(args: Array[String]): Properties = { val optionParser = new OptionParser val overrideOpt = optionParser.accepts("override", "Optional property that should override values set in server.properties file") .withRequiredArg() @@ -47,14 +49,14 @@ object Kafka extends Logging { props.putAll(CommandLineUtils.parseKeyValueArgs(options.valuesOf(overrideOpt))) } - - KafkaConfig.fromProps(props) + props } def main(args: Array[String]): Unit = { try { - val serverConfig = getKafkaConfigFromArgs(args) - KafkaMetricsReporter.startReporters(new VerifiableProperties(serverConfig.toProps)) + val serverProps = getPropsFromArgs(args) + val serverConfig = KafkaConfig.fromProps(serverProps) + KafkaMetricsReporter.startReporters(new VerifiableProperties(serverProps)) val kafkaServerStartable = new KafkaServerStartable(serverConfig) // attach shutdown handler to catch control-c diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 6d1c6ab..f23120e 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -174,7 +174,7 @@ object ConsumerGroupCommand { 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) + val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload()) offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) { diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index a3b1b78..258d5fe 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -18,9 +18,10 @@ package kafka.api import java.nio.ByteBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.network.RequestChannel.Response + import kafka.common.ErrorMapping +import kafka.network.{RequestOrResponseSend, RequestChannel} +import kafka.network.RequestChannel.Response object ConsumerMetadataRequest { val CurrentVersion = 0.shortValue @@ -64,7 +65,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, correlationId) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } def describe(details: Boolean) = { diff --git a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala index fe81635..8092007 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala @@ -18,10 +18,9 @@ package kafka.api import java.nio.ByteBuffer -import kafka.api.ApiUtils._ -import collection.mutable.ListBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.common.{TopicAndPartition, ErrorMapping} + +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.utils.Logging @@ -63,7 +62,7 @@ case class ControlledShutdownRequest(versionId: Short, override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { val errorResponse = ControlledShutdownResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]]), Set.empty[TopicAndPartition]) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean = false): String = { diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index b038c15..5b38f85 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -149,7 +149,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, (topicAndPartition, FetchResponsePartitionData(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1, MessageSet.Empty)) } val errorResponse = FetchResponse(correlationId, fetchResponsePartitionData) - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(errorResponse))) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 75aaf57..0b6b33a 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -22,8 +22,10 @@ import java.nio.channels.GatheringByteChannel import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.message.{MessageSet, ByteBufferMessageSet} -import kafka.network.{MultiSend, Send} import kafka.api.ApiUtils._ +import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.network.Send +import org.apache.kafka.common.network.MultiSend import scala.collection._ @@ -62,10 +64,12 @@ class PartitionDataSend(val partitionId: Int, buffer.putInt(partitionData.messages.sizeInBytes) buffer.rewind() - override def complete = !buffer.hasRemaining && messagesSentSize >= messageSize + override def completed = !buffer.hasRemaining && messagesSentSize >= messageSize - override def writeTo(channel: GatheringByteChannel): Int = { - var written = 0 + override def destination: String = "" + + override def writeTo(channel: GatheringByteChannel): Long = { + var written = 0L if(buffer.hasRemaining) written += channel.write(buffer) if(!buffer.hasRemaining && messagesSentSize < messageSize) { @@ -75,6 +79,8 @@ class PartitionDataSend(val partitionId: Int, } written } + + override def size = buffer.capacity() + messageSize } object TopicData { @@ -101,29 +107,32 @@ case class TopicData(topic: String, partitionData: Map[Int, FetchResponsePartiti val headerSize = TopicData.headerSize(topic) } -class TopicDataSend(val topicData: TopicData) extends Send { - private val size = topicData.sizeInBytes +class TopicDataSend(val dest: String, val topicData: TopicData) extends Send { + + private var sent = 0L - private var sent = 0 + override def completed: Boolean = sent >= size - override def complete = sent >= size + override def destination: String = dest + + override def size = topicData.headerSize + sends.size() private val buffer = ByteBuffer.allocate(topicData.headerSize) writeShortString(buffer, topicData.topic) buffer.putInt(topicData.partitionData.size) buffer.rewind() - val sends = new MultiSend(topicData.partitionData.toList - .map(d => new PartitionDataSend(d._1, d._2))) { - val expectedBytesToWrite = topicData.sizeInBytes - topicData.headerSize - } + private val sends = new MultiSend(dest, + JavaConversions.seqAsJavaList(topicData.partitionData.toList.map(d => new PartitionDataSend(d._1, d._2)))) - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - var written = 0 + override def writeTo(channel: GatheringByteChannel): Long = { + if (completed) + throw new KafkaException("This operation cannot be completed on a complete request.") + + var written = 0L if(buffer.hasRemaining) written += channel.write(buffer) - if(!buffer.hasRemaining && !sends.complete) { + if(!buffer.hasRemaining && !sends.completed) { written += sends.writeTo(channel) } sent += written @@ -200,34 +209,36 @@ case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchR } -class FetchResponseSend(val fetchResponse: FetchResponse) extends Send { - private val size = fetchResponse.sizeInBytes +class FetchResponseSend(val dest: String, val fetchResponse: FetchResponse) extends Send { + private val payloadSize = fetchResponse.sizeInBytes + + private var sent = 0L - private var sent = 0 + override def size = 4 /* for size byte */ + payloadSize - private val sendSize = 4 /* for size */ + size + override def completed = sent >= size - override def complete = sent >= sendSize + override def destination = dest private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize) - buffer.putInt(size) + buffer.putInt(payloadSize) buffer.putInt(fetchResponse.correlationId) buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count buffer.rewind() - val sends = new MultiSend(fetchResponse.dataGroupedByTopic.toList.map { - case(topic, data) => new TopicDataSend(TopicData(topic, + private val sends = new MultiSend(dest, JavaConversions.seqAsJavaList(fetchResponse.dataGroupedByTopic.toList.map { + case(topic, data) => new TopicDataSend(dest, TopicData(topic, data.map{case(topicAndPartition, message) => (topicAndPartition.partition, message)})) - }) { - val expectedBytesToWrite = fetchResponse.sizeInBytes - FetchResponse.headerSize - } + })) + + override def writeTo(channel: GatheringByteChannel): Long = { + if (completed) + throw new KafkaException("This operation cannot be completed on a complete request.") - def writeTo(channel: GatheringByteChannel):Int = { - expectIncomplete() - var written = 0 + var written = 0L if(buffer.hasRemaining) written += channel.write(buffer) - if(!buffer.hasRemaining && !sends.complete) { + if(!buffer.hasRemaining && !sends.completed) { written += sends.writeTo(channel) } sent += written diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index 431190a..c2584e0 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -19,14 +19,16 @@ package kafka.api import java.nio._ -import kafka.utils._ + import kafka.api.ApiUtils._ import kafka.cluster.BrokerEndPoint -import kafka.controller.LeaderIsrAndControllerEpoch -import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping +import kafka.controller.LeaderIsrAndControllerEpoch +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response -import collection.Set +import kafka.utils._ + +import scala.collection.Set object LeaderAndIsr { @@ -184,7 +186,7 @@ case class LeaderAndIsrRequest (versionId: Short, case (topicAndPartition, partitionAndState) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } val errorResponse = LeaderAndIsrResponse(correlationId, responseMap) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 317daed..5b362ef 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -18,11 +18,13 @@ package kafka.api import java.nio.ByteBuffer + import kafka.api.ApiUtils._ -import kafka.utils.{SystemTime, Logging} -import kafka.network.{RequestChannel, BoundedByteBufferSend} -import kafka.common.{OffsetMetadata, OffsetAndMetadata, ErrorMapping, TopicAndPartition} +import kafka.common.{ErrorMapping, OffsetAndMetadata, TopicAndPartition} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response +import kafka.utils.Logging + import scala.collection._ object OffsetCommitRequest extends Logging { @@ -162,7 +164,7 @@ case class OffsetCommitRequest(groupId: String, val commitStatus = requestInfo.mapValues(_ => errorCode) val commitResponse = OffsetCommitResponse(commitStatus, correlationId) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(commitResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, commitResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala index fa8bd6a..a83e147 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -17,16 +17,13 @@ package kafka.api +import java.nio.ByteBuffer + import kafka.api.ApiUtils._ -import kafka.utils.Logging -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.common._ -import kafka.common.TopicAndPartition +import kafka.common.{TopicAndPartition, _} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response - -import scala.Some - -import java.nio.ByteBuffer +import kafka.utils.Logging object OffsetFetchRequest extends Logging { val CurrentVersion: Short = 1 @@ -99,7 +96,7 @@ case class OffsetFetchRequest(groupId: String, )) }.toMap val errorResponse = OffsetFetchResponse(requestInfo=responseMap, correlationId=correlationId) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala index 3d483bc..f418868 100644 --- a/core/src/main/scala/kafka/api/OffsetRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetRequest.scala @@ -18,9 +18,10 @@ package kafka.api import java.nio.ByteBuffer -import kafka.common.{ErrorMapping, TopicAndPartition} + import kafka.api.ApiUtils._ -import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response @@ -117,7 +118,7 @@ case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequ (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), null)) } val errorResponse = OffsetResponse(correlationId, partitionOffsetResponseMap) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } 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 570b2da..c866180 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -18,11 +18,12 @@ package kafka.api import java.nio._ -import kafka.message._ + import kafka.api.ApiUtils._ import kafka.common._ +import kafka.message._ +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response -import kafka.network.{RequestChannel, BoundedByteBufferSend} object ProducerRequest { val CurrentVersion = 0.shortValue @@ -136,7 +137,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, (topicAndPartition, ProducerResponseStatus(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1l)) } val errorResponse = ProducerResponse(correlationId, producerResponseStatus) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } } diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala index ef7a86e..155cb65 100644 --- a/core/src/main/scala/kafka/api/RequestKeys.scala +++ b/core/src/main/scala/kafka/api/RequestKeys.scala @@ -20,6 +20,8 @@ package kafka.api import kafka.common.KafkaException import java.nio.ByteBuffer +import kafka.network.InvalidRequestException + object RequestKeys { val ProduceKey: Short = 0 val FetchKey: Short = 1 @@ -59,7 +61,7 @@ object RequestKeys { def deserializerForKey(key: Short): (ByteBuffer) => RequestOrResponse = { keyToNameAndDeserializerMap.get(key) match { case Some(nameAndSerializer) => nameAndSerializer._2 - case None => throw new KafkaException("Wrong request type %d".format(key)) + case None => throw new InvalidRequestException("Wrong request type %d".format(key)) } } } diff --git a/core/src/main/scala/kafka/api/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala index 5e14987..4441fc6 100644 --- a/core/src/main/scala/kafka/api/StopReplicaRequest.scala +++ b/core/src/main/scala/kafka/api/StopReplicaRequest.scala @@ -19,7 +19,7 @@ package kafka.api import java.nio._ import kafka.api.ApiUtils._ -import kafka.network.{BoundedByteBufferSend, RequestChannel, InvalidRequestException} +import kafka.network.{RequestOrResponseSend, RequestChannel, InvalidRequestException} import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.network.RequestChannel.Response import kafka.utils.Logging @@ -106,7 +106,7 @@ case class StopReplicaRequest(versionId: Short, case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) }.toMap val errorResponse = StopReplicaResponse(correlationId, responseMap) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 363bae0..401c583 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -18,13 +18,15 @@ package kafka.api import java.nio.ByteBuffer + import kafka.api.ApiUtils._ -import collection.mutable.ListBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.utils.Logging +import scala.collection.mutable.ListBuffer + object TopicMetadataRequest extends Logging { val CurrentVersion = 0.shortValue val DefaultClientId = "" @@ -80,7 +82,7 @@ case class TopicMetadataRequest(versionId: Short, topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } val errorResponse = TopicMetadataResponse(Seq(), topicMetadata, correlationId) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala index 69f0397..d59de82 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -21,8 +21,8 @@ import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.cluster.{Broker, BrokerEndPoint} import kafka.common.{ErrorMapping, KafkaException, TopicAndPartition} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response -import kafka.network.{BoundedByteBufferSend, RequestChannel} import org.apache.kafka.common.protocol.SecurityProtocol import scala.collection.Set @@ -128,7 +128,7 @@ case class UpdateMetadataRequest (versionId: Short, override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { val errorResponse = new UpdateMetadataResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]])) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 62394c0..68c7e7f 100755 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -153,7 +153,7 @@ object ClientUtils extends Logging{ debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group)) queryChannel.send(ConsumerMetadataRequest(group)) val response = queryChannel.receive() - val consumerMetadataResponse = ConsumerMetadataResponse.readFrom(response.buffer) + val consumerMetadataResponse = ConsumerMetadataResponse.readFrom(response.payload()) debug("Consumer metadata response: " + consumerMetadataResponse.toString) if (consumerMetadataResponse.errorCode == ErrorMapping.NoError) coordinatorOpt = consumerMetadataResponse.coordinatorOpt diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index 31a2639..c16f7ed 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -24,6 +24,7 @@ import kafka.api._ import kafka.network._ import kafka.utils._ import kafka.common.{ErrorMapping, TopicAndPartition} +import org.apache.kafka.common.network.{NetworkReceive, Receive} import org.apache.kafka.common.utils.Utils._ /** @@ -65,9 +66,9 @@ class SimpleConsumer(val host: String, } } - private def sendRequest(request: RequestOrResponse): Receive = { + private def sendRequest(request: RequestOrResponse): NetworkReceive = { lock synchronized { - var response: Receive = null + var response: NetworkReceive = null try { getOrMakeConnection() blockingChannel.send(request) @@ -94,12 +95,12 @@ class SimpleConsumer(val host: String, def send(request: TopicMetadataRequest): TopicMetadataResponse = { val response = sendRequest(request) - TopicMetadataResponse.readFrom(response.buffer) + TopicMetadataResponse.readFrom(response.payload()) } def send(request: ConsumerMetadataRequest): ConsumerMetadataResponse = { val response = sendRequest(request) - ConsumerMetadataResponse.readFrom(response.buffer) + ConsumerMetadataResponse.readFrom(response.payload()) } /** @@ -109,7 +110,7 @@ class SimpleConsumer(val host: String, * @return a set of fetched messages */ def fetch(request: FetchRequest): FetchResponse = { - var response: Receive = null + var response: NetworkReceive = null val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestTimer val aggregateTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestTimer aggregateTimer.time { @@ -117,7 +118,7 @@ class SimpleConsumer(val host: String, response = sendRequest(request) } } - val fetchResponse = FetchResponse.readFrom(response.buffer) + val fetchResponse = FetchResponse.readFrom(response.payload()) val fetchedSize = fetchResponse.sizeInBytes fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestSizeHist.update(fetchedSize) fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize) @@ -129,7 +130,7 @@ class SimpleConsumer(val host: String, * @param request a [[kafka.api.OffsetRequest]] object. * @return a [[kafka.api.OffsetResponse]] object. */ - def getOffsetsBefore(request: OffsetRequest) = OffsetResponse.readFrom(sendRequest(request).buffer) + def getOffsetsBefore(request: OffsetRequest) = OffsetResponse.readFrom(sendRequest(request).payload()) /** * Commit offsets for a topic @@ -140,7 +141,7 @@ class SimpleConsumer(val host: String, def commitOffsets(request: OffsetCommitRequest) = { // TODO: With KAFKA-1012, we have to first issue a ConsumerMetadataRequest and connect to the coordinator before // we can commit offsets. - OffsetCommitResponse.readFrom(sendRequest(request).buffer) + OffsetCommitResponse.readFrom(sendRequest(request).payload()) } /** @@ -149,7 +150,7 @@ class SimpleConsumer(val host: String, * @param request a [[kafka.api.OffsetFetchRequest]] object. * @return a [[kafka.api.OffsetFetchResponse]] object. */ - def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).buffer) + def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).payload()) private def getOrMakeConnection() { if(!isClosed && !blockingChannel.isConnected) { diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index a7f2acc..e42d104 100755 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -334,7 +334,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, try { kafkaCommitMeter.mark(offsetsToCommit.size) offsetsChannel.send(offsetCommitRequest) - val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().buffer) + val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().payload()) trace("Offset commit response: %s.".format(offsetCommitResponse)) val (commitFailed, retryableIfFailed, shouldRefreshCoordinator, errorCount) = { @@ -421,7 +421,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, ensureOffsetManagerConnected() try { offsetsChannel.send(offsetFetchRequest) - val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().buffer) + val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().payload()) trace("Offset fetch response: %s.".format(offsetFetchResponse)) val (leaderChanged, loadInProgress) = diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 6cf13f0..9f521fa 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -16,8 +16,9 @@ */ package kafka.controller -import kafka.network.{Receive, BlockingChannel} +import kafka.network.BlockingChannel import kafka.utils.{CoreUtils, Logging, ShutdownableThread} +import org.apache.kafka.common.network.NetworkReceive import collection.mutable.HashMap import kafka.cluster.Broker import java.util.concurrent.{LinkedBlockingQueue, BlockingQueue} @@ -120,7 +121,7 @@ class RequestSendThread(val controllerId: Int, val queueItem = queue.take() val request = queueItem._1 val callback = queueItem._2 - var receive: Receive = null + var receive: NetworkReceive = null try { lock synchronized { var isSendSuccessful = false @@ -147,11 +148,11 @@ class RequestSendThread(val controllerId: Int, var response: RequestOrResponse = null request.requestId.get match { case RequestKeys.LeaderAndIsrKey => - response = LeaderAndIsrResponse.readFrom(receive.buffer) + response = LeaderAndIsrResponse.readFrom(receive.payload()) case RequestKeys.StopReplicaKey => - response = StopReplicaResponse.readFrom(receive.buffer) + response = StopReplicaResponse.readFrom(receive.payload()) case RequestKeys.UpdateMetadataKey => - response = UpdateMetadataResponse.readFrom(receive.buffer) + response = UpdateMetadataResponse.readFrom(receive.payload()) } stateChangeLogger.trace("Controller %d epoch %d received response %s for a request sent to broker %s" .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString)) diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index b0b7be1..568d0ac 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -16,12 +16,11 @@ */ package kafka.javaapi -import kafka.api._ import java.nio.ByteBuffer + +import kafka.api._ + import scala.collection.mutable -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.common.ErrorMapping -import kafka.network.RequestChannel.Response class TopicMetadataRequest(val versionId: Short, val correlationId: Int, diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala index 6e2a38e..1197259 100644 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ b/core/src/main/scala/kafka/network/BlockingChannel.scala @@ -19,8 +19,10 @@ package kafka.network import java.net.InetSocketAddress import java.nio.channels._ -import kafka.utils.{nonthreadsafe, Logging} + import kafka.api.RequestOrResponse +import kafka.utils.{Logging, nonthreadsafe} +import org.apache.kafka.common.network.NetworkReceive object BlockingChannel{ @@ -43,6 +45,7 @@ class BlockingChannel( val host: String, private var writeChannel: GatheringByteChannel = null private val lock = new Object() private val connectTimeoutMs = readTimeoutMs + private var connectionId: String = "" def connect() = lock synchronized { if(!connected) { @@ -59,8 +62,15 @@ class BlockingChannel( val host: String, channel.socket.connect(new InetSocketAddress(host, port), connectTimeoutMs) writeChannel = channel + // Need to create a new ReadableByteChannel from input stream because SocketChannel doesn't implement read with timeout + // See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work readChannel = Channels.newChannel(channel.socket().getInputStream) connected = true + val localHost = channel.socket.getLocalAddress.getHostAddress + val localPort = channel.socket.getLocalPort + val remoteHost = channel.socket.getInetAddress.getHostAddress + val remotePort = channel.socket.getPort + connectionId = localHost + ":" + localPort + "-" + remoteHost + ":" + remotePort // 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), connectTimeoutMs = %d." debug(msg.format(channel.socket.getSoTimeout, @@ -95,20 +105,21 @@ class BlockingChannel( val host: String, def isConnected = connected - def send(request: RequestOrResponse):Int = { + def send(request: RequestOrResponse): Long = { if(!connected) throw new ClosedChannelException() - val send = new BoundedByteBufferSend(request) + val send = new RequestOrResponseSend(connectionId, request) send.writeCompletely(writeChannel) } - def receive(): Receive = { + def receive(): NetworkReceive = { if(!connected) throw new ClosedChannelException() - val response = new BoundedByteBufferReceive() + val response = new NetworkReceive() response.readCompletely(readChannel) + response.payload().rewind() response } diff --git a/core/src/main/scala/kafka/network/BoundedByteBufferReceive.scala b/core/src/main/scala/kafka/network/BoundedByteBufferReceive.scala deleted file mode 100755 index c0d7726..0000000 --- a/core/src/main/scala/kafka/network/BoundedByteBufferReceive.scala +++ /dev/null @@ -1,90 +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.network - -import java.nio._ -import java.nio.channels._ -import kafka.utils._ - -/** - * Represents a communication between the client and server - * - */ -@nonthreadsafe -private[kafka] class BoundedByteBufferReceive(val maxSize: Int) extends Receive with Logging { - - private val sizeBuffer = ByteBuffer.allocate(4) - private var contentBuffer: ByteBuffer = null - - def this() = this(Int.MaxValue) - - var complete: Boolean = false - - /** - * Get the content buffer for this transmission - */ - def buffer: ByteBuffer = { - expectComplete() - contentBuffer - } - - /** - * Read the bytes in this response from the given channel - */ - def readFrom(channel: ReadableByteChannel): Int = { - expectIncomplete() - var read = 0 - // have we read the request size yet? - if(sizeBuffer.remaining > 0) - read += CoreUtils.read(channel, sizeBuffer) - // have we allocated the request buffer yet? - if(contentBuffer == null && !sizeBuffer.hasRemaining) { - sizeBuffer.rewind() - val size = sizeBuffer.getInt() - if(size <= 0) - throw new InvalidRequestException("%d is not a valid request size.".format(size)) - if(size > maxSize) - throw new InvalidRequestException("Request of length %d is not valid, it is larger than the maximum size of %d bytes.".format(size, maxSize)) - contentBuffer = byteBufferAllocate(size) - } - // if we have a buffer read some stuff into it - if(contentBuffer != null) { - read = CoreUtils.read(channel, contentBuffer) - // did we get everything? - if(!contentBuffer.hasRemaining) { - contentBuffer.rewind() - complete = true - } - } - read - } - - private def byteBufferAllocate(size: Int): ByteBuffer = { - var buffer: ByteBuffer = null - try { - buffer = ByteBuffer.allocate(size) - } catch { - case e: OutOfMemoryError => - error("OOME with size " + size, e) - throw e - case e2: Throwable => - throw e2 - } - buffer - } -} diff --git a/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala b/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala deleted file mode 100644 index b95b73b..0000000 --- a/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala +++ /dev/null @@ -1,71 +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.network - -import java.nio._ -import java.nio.channels._ -import kafka.utils._ -import kafka.api.RequestOrResponse -import org.apache.kafka.common.requests.{AbstractRequestResponse, ResponseHeader} - -@nonthreadsafe -private[kafka] class BoundedByteBufferSend(val buffer: ByteBuffer) extends Send { - - private val sizeBuffer = ByteBuffer.allocate(4) - - // Avoid possibility of overflow for 2GB-4 byte buffer - if(buffer.remaining > Int.MaxValue - sizeBuffer.limit) - throw new IllegalStateException("Attempt to create a bounded buffer of " + buffer.remaining + " bytes, but the maximum " + - "allowable size for a bounded buffer is " + (Int.MaxValue - sizeBuffer.limit) + ".") - sizeBuffer.putInt(buffer.limit) - sizeBuffer.rewind() - - var complete: Boolean = false - - def this(size: Int) = this(ByteBuffer.allocate(size)) - - def this(request: RequestOrResponse) = { - this(request.sizeInBytes + (if(request.requestId != None) 2 else 0)) - request.requestId match { - case Some(requestId) => - buffer.putShort(requestId) - case None => - } - - request.writeTo(buffer) - buffer.rewind() - } - - def this(header: ResponseHeader, body: AbstractRequestResponse) = { - this(header.sizeOf + body.sizeOf) - header.writeTo(buffer) - body.writeTo(buffer) - buffer.rewind - } - - - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - val written = channel.write(Array(sizeBuffer, buffer)) - // if we are done, mark it off - if(!buffer.hasRemaining) - complete = true - written.asInstanceOf[Int] - } - -} diff --git a/core/src/main/scala/kafka/network/ByteBufferSend.scala b/core/src/main/scala/kafka/network/ByteBufferSend.scala deleted file mode 100644 index af30042..0000000 --- a/core/src/main/scala/kafka/network/ByteBufferSend.scala +++ /dev/null @@ -1,40 +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.network - -import java.nio._ -import java.nio.channels._ -import kafka.utils._ - -@nonthreadsafe -private[kafka] class ByteBufferSend(val buffer: ByteBuffer) extends Send { - - var complete: Boolean = false - - def this(size: Int) = this(ByteBuffer.allocate(size)) - - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - var written = 0 - written += channel.write(buffer) - if(!buffer.hasRemaining) - complete = true - written - } - -} diff --git a/core/src/main/scala/kafka/network/Handler.scala b/core/src/main/scala/kafka/network/Handler.scala index a030033..1a7d56e 100644 --- a/core/src/main/scala/kafka/network/Handler.scala +++ b/core/src/main/scala/kafka/network/Handler.scala @@ -17,17 +17,19 @@ package kafka.network +import org.apache.kafka.common.network.{NetworkReceive, Send} + private[kafka] object Handler { /** * A request handler is a function that turns an incoming * transmission into an outgoing transmission */ - type Handler = Receive => Option[Send] + type Handler = NetworkReceive => Option[Send] /** * A handler mapping finds the right Handler function for a given request */ - type HandlerMapping = (Short, Receive) => Handler + type HandlerMapping = (Short, NetworkReceive) => Handler } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 24edb61..357d8b9 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -17,22 +17,23 @@ package kafka.network +import java.nio.ByteBuffer import java.util.concurrent._ -import kafka.metrics.KafkaMetricsGroup + import com.yammer.metrics.core.Gauge -import java.nio.ByteBuffer import kafka.api._ import kafka.common.TopicAndPartition -import kafka.utils.{Logging, SystemTime} import kafka.message.ByteBufferMessageSet -import java.net._ +import kafka.metrics.KafkaMetricsGroup +import kafka.utils.{Logging, SystemTime} +import org.apache.kafka.common.network.Send import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} import org.apache.kafka.common.requests.{AbstractRequest, RequestHeader} import org.apache.log4j.Logger object RequestChannel extends Logging { - val AllDone = new Request(processor = 1, requestKey = 2, buffer = getShutdownReceive(), startTimeMs = 0, securityProtocol = SecurityProtocol.PLAINTEXT) + val AllDone = new Request(processor = 1, connectionId = "2", buffer = getShutdownReceive(), startTimeMs = 0, securityProtocol = SecurityProtocol.PLAINTEXT) def getShutdownReceive() = { val emptyProducerRequest = new ProducerRequest(0, 0, "", 0, 0, collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]()) @@ -43,7 +44,7 @@ object RequestChannel extends Logging { byteBuffer } - case class Request(processor: Int, requestKey: Any, private var buffer: ByteBuffer, startTimeMs: Long, remoteAddress: SocketAddress = new InetSocketAddress(0), securityProtocol: SecurityProtocol) { + case class Request(processor: Int, connectionId: String, private var buffer: ByteBuffer, startTimeMs: Long, securityProtocol: SecurityProtocol) { @volatile var requestDequeueTimeMs = -1L @volatile var apiLocalCompleteTimeMs = -1L @volatile var responseCompleteTimeMs = -1L @@ -71,7 +72,15 @@ object RequestChannel extends Logging { buffer = null private val requestLogger = Logger.getLogger("kafka.request.logger") - trace("Processor %d received request : %s".format(processor, if (requestObj != null) requestObj.describe(false) else header.toString + " : " + body.toString)) + + private def requestDesc: String = { + if (requestObj != null) + requestObj.describe(false) + else + header.toString + " -- " + body.toString + } + + trace("Processor %d received request : %s".format(processor, requestDesc)) def updateRequestMetrics() { val endTimeMs = SystemTime.milliseconds @@ -102,13 +111,13 @@ object RequestChannel extends Logging { m.responseSendTimeHist.update(responseSendTime) m.totalTimeHist.update(totalTime) } + if(requestLogger.isTraceEnabled) - requestLogger.trace("Completed request:%s from client %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" - .format(if (requestObj != null) requestObj.describe(true) else header.toString + " : " + body.toString, remoteAddress, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) - else if(requestLogger.isDebugEnabled) { - requestLogger.debug("Completed request:%s from client %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" - .format(if (requestObj != null) requestObj.describe(false) else header.toString + " : " + body.toString, remoteAddress, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) - } + requestLogger.trace("Completed request:%s from connection %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" + .format(requestDesc, connectionId, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) + else if(requestLogger.isDebugEnabled) + requestLogger.debug("Completed request:%s from connection %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" + .format(requestDesc, connectionId, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) } } diff --git a/core/src/main/scala/kafka/network/RequestOrResponseSend.scala b/core/src/main/scala/kafka/network/RequestOrResponseSend.scala new file mode 100644 index 0000000..364f24b --- /dev/null +++ b/core/src/main/scala/kafka/network/RequestOrResponseSend.scala @@ -0,0 +1,57 @@ +/* + * 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.network + +import java.nio.ByteBuffer +import java.nio.channels.GatheringByteChannel + +import kafka.api.RequestOrResponse +import kafka.utils.Logging +import org.apache.kafka.common.network.NetworkSend + +object RequestOrResponseSend { + def serialize(request: RequestOrResponse): ByteBuffer = { + val buffer = ByteBuffer.allocate(request.sizeInBytes + (if(request.requestId != None) 2 else 0)) + request.requestId match { + case Some(requestId) => + buffer.putShort(requestId) + case None => + } + request.writeTo(buffer) + buffer.rewind() + buffer + } +} + +class RequestOrResponseSend(val dest: String, val buffer: ByteBuffer) extends NetworkSend(dest, buffer) with Logging { + + def this(dest: String, request: RequestOrResponse) { + this(dest, RequestOrResponseSend.serialize(request)) + } + + def writeCompletely(channel: GatheringByteChannel): Long = { + var totalWritten = 0L + while(!completed()) { + val written = writeTo(channel) + trace(written + " bytes written.") + totalWritten += written + } + totalWritten + } + +} diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index edf6214..91319fa 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -17,23 +17,26 @@ package kafka.network +import java.io.IOException +import java.net._ +import java.nio.channels._ import java.util import java.util.concurrent._ import java.util.concurrent.atomic._ -import java.net._ -import java.io._ -import java.nio.channels._ +import com.yammer.metrics.core.Gauge import kafka.cluster.EndPoint -import org.apache.kafka.common.protocol.SecurityProtocol - -import scala.collection._ - import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup import kafka.utils._ -import com.yammer.metrics.core.{Gauge, Meter} -import org.apache.kafka.common.utils.Utils +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.network.InvalidReceiveException +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.protocol.types.SchemaException +import org.apache.kafka.common.utils.{SystemTime, Time, Utils} + +import scala.collection._ /** * An NIO socket server. The threading model is @@ -50,16 +53,20 @@ class SocketServer(val brokerId: Int, val maxRequestSize: Int = Int.MaxValue, val maxConnectionsPerIp: Int = Int.MaxValue, val connectionsMaxIdleMs: Long, - val maxConnectionsPerIpOverrides: Map[String, Int] ) extends Logging with KafkaMetricsGroup { + val maxConnectionsPerIpOverrides: Map[String, Int], + val time: Time, + val metrics: Metrics) extends Logging with KafkaMetricsGroup { this.logIdent = "[Socket Server on Broker " + brokerId + "], " - private val time = SystemTime + private val processors = new Array[Processor](numProcessorThreads) private[network] var acceptors = mutable.Map[EndPoint,Acceptor]() val requestChannel = new RequestChannel(numProcessorThreads, maxQueuedRequests) - /* a meter to track the average free capacity of the network processors */ - private val aggregateIdleMeter = newMeter("NetworkProcessorAvgIdlePercent", "percent", TimeUnit.NANOSECONDS) - + private val allMetricNames = (0 until numProcessorThreads).map { i => + val tags = new util.HashMap[String, String]() + tags.put("networkProcessor", i.toString) + new MetricName("io-wait-ratio", "socket-server-metrics", tags) + } /* I'm pushing the mapping of port-to-protocol to the processor level, so the processor can put the correct protocol in the request channel. @@ -75,26 +82,30 @@ class SocketServer(val brokerId: Int, def startup() { val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides) + newGauge("NetworkProcessorAvgIdlePercent", + new Gauge[Double] { + def value = allMetricNames.map( metricName => + metrics.metrics().get(metricName).value()).sum / numProcessorThreads + } + ) + + this.synchronized { for (i <- 0 until numProcessorThreads) { processors(i) = new Processor(i, time, maxRequestSize, - aggregateIdleMeter, - newMeter("IdlePercent", "percent", TimeUnit.NANOSECONDS, Map("networkProcessor" -> i.toString)), numProcessorThreads, requestChannel, quotas, connectionsMaxIdleMs, - portToProtocol) + portToProtocol, + metrics + ) Utils.newThread("kafka-network-thread-%d-%d".format(brokerId, i), processors(i), false).start() } } - newGauge("ResponsesBeingSent", new Gauge[Int] { - def value = processors.foldLeft(0) { (total, p) => total + p.countInterestOps(SelectionKey.OP_WRITE) } - }) - // register the processor threads for notification of responses requestChannel.addResponseListener((id:Int) => processors(id).wakeup()) @@ -140,18 +151,19 @@ class SocketServer(val brokerId: Int, */ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQuotas) extends Runnable with Logging { - protected val selector = Selector.open() private val startupLatch = new CountDownLatch(1) private val shutdownLatch = new CountDownLatch(1) private val alive = new AtomicBoolean(true) + def wakeup() + /** * Initiates a graceful shutdown by signaling to stop and waiting for the shutdown to complete */ def shutdown(): Unit = { alive.set(false) - selector.wakeup() - shutdownLatch.await + wakeup() + shutdownLatch.await() } /** @@ -163,13 +175,13 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ * Record that the thread startup is complete */ protected def startupComplete() = { - startupLatch.countDown + startupLatch.countDown() } /** * Record that the thread shutdown is complete */ - protected def shutdownComplete() = shutdownLatch.countDown + protected def shutdownComplete() = shutdownLatch.countDown() /** * Is the server still running? @@ -177,11 +189,6 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ protected def isRunning = alive.get /** - * Wakeup the thread for selection. - */ - def wakeup() = selector.wakeup() - - /** * Close the given key and associated socket */ def close(key: SelectionKey) { @@ -200,30 +207,6 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ swallowError(channel.close()) } } - - /** - * Close all open connections - */ - def closeAll() { - // removes cancelled keys from selector.keys set - this.selector.selectNow() - val iter = this.selector.keys().iterator() - while (iter.hasNext) { - val key = iter.next() - close(key) - } - } - - def countInterestOps(ops: Int): Int = { - var count = 0 - val it = this.selector.keys().iterator() - while (it.hasNext) { - if ((it.next().interestOps() & ops) != 0) { - count += 1 - } - } - count - } } /** @@ -237,6 +220,7 @@ private[kafka] class Acceptor(val host: String, connectionQuotas: ConnectionQuotas, protocol: SecurityProtocol, portToProtocol: ConcurrentHashMap[Int, SecurityProtocol]) extends AbstractServerThread(connectionQuotas) { + val nioSelector = java.nio.channels.Selector.open() val serverChannel = openServerSocket(host, port) portToProtocol.put(serverChannel.socket().getLocalPort, protocol) @@ -244,13 +228,13 @@ private[kafka] class Acceptor(val host: String, * Accept loop that checks for new connection attempts */ def run() { - serverChannel.register(selector, SelectionKey.OP_ACCEPT) + serverChannel.register(nioSelector, SelectionKey.OP_ACCEPT); startupComplete() var currentProcessor = 0 while(isRunning) { - val ready = selector.select(500) + val ready = nioSelector.select(500) if(ready > 0) { - val keys = selector.selectedKeys() + val keys = nioSelector.selectedKeys() val iter = keys.iterator() while(iter.hasNext && isRunning) { var key: SelectionKey = null @@ -258,7 +242,7 @@ private[kafka] class Acceptor(val host: String, key = iter.next iter.remove() if(key.isAcceptable) - accept(key, processors(currentProcessor)) + accept(key, processors(currentProcessor)) else throw new IllegalStateException("Unrecognized key state for acceptor thread.") @@ -272,7 +256,7 @@ private[kafka] class Acceptor(val host: String, } debug("Closing server socket and selector.") swallowError(serverChannel.close()) - swallowError(selector.close()) + swallowError(nioSelector.close()) shutdownComplete() } @@ -290,7 +274,7 @@ private[kafka] class Acceptor(val host: String, serverChannel.socket().setReceiveBufferSize(recvBufferSize) try { serverChannel.socket.bind(socketAddress) - info("Awaiting socket connections on %s:%d.".format(socketAddress.getHostName, port)) + info("Awaiting socket connections on %s:%d.".format(socketAddress.getHostName, serverChannel.socket.getLocalPort)) } catch { case e: SocketException => throw new KafkaException("Socket server failed to bind to %s:%d: %s.".format(socketAddress.getHostName, port, e.getMessage), e) @@ -324,6 +308,12 @@ private[kafka] class Acceptor(val host: String, } } + /** + * Wakeup the thread for selection. + */ + @Override + def wakeup = nioSelector.wakeup() + } /** @@ -333,19 +323,36 @@ private[kafka] class Acceptor(val host: String, private[kafka] class Processor(val id: Int, val time: Time, val maxRequestSize: Int, - val aggregateIdleMeter: Meter, - val idleMeter: Meter, val totalProcessorThreads: Int, val requestChannel: RequestChannel, connectionQuotas: ConnectionQuotas, val connectionsMaxIdleMs: Long, - val portToProtocol: ConcurrentHashMap[Int,SecurityProtocol]) extends AbstractServerThread(connectionQuotas) { + val portToProtocol: ConcurrentHashMap[Int,SecurityProtocol], + val metrics: Metrics) extends AbstractServerThread(connectionQuotas) with KafkaMetricsGroup { private val newConnections = new ConcurrentLinkedQueue[SocketChannel]() - private val connectionsMaxIdleNanos = connectionsMaxIdleMs * 1000 * 1000 - private var currentTimeNanos = SystemTime.nanoseconds - private val lruConnections = new util.LinkedHashMap[SelectionKey, Long] - private var nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos + private val inflightResponses = mutable.Map[String, RequestChannel.Response]() + + private val metricTags = new util.HashMap[String, String]() + metricTags.put("networkProcessor", id.toString) + + newGauge("IdlePercent", + new Gauge[Double] { + def value = { + metrics.metrics().get(new MetricName("io-wait-ratio", "socket-server-metrics", metricTags)).value() + } + }, + JavaConversions.mapAsScalaMap(metricTags) + ) + + private val selector = new org.apache.kafka.common.network.Selector( + maxRequestSize, + connectionsMaxIdleMs, + metrics, + time, + "socket-server", + metricTags, + false) override def run() { startupComplete() @@ -354,68 +361,51 @@ private[kafka] class Processor(val id: Int, configureNewConnections() // register any new responses for writing processNewResponses() - val startSelectTime = SystemTime.nanoseconds - val ready = selector.select(300) - currentTimeNanos = SystemTime.nanoseconds - val idleTime = currentTimeNanos - startSelectTime - idleMeter.mark(idleTime) - // We use a single meter for aggregate idle percentage for the thread pool. - // Since meter is calculated as total_recorded_value / time_window and - // time_window is independent of the number of threads, each recorded idle - // time should be discounted by # threads. - aggregateIdleMeter.mark(idleTime / totalProcessorThreads) - - trace("Processor id " + id + " selection time = " + idleTime + " ns") - if(ready > 0) { - val keys = selector.selectedKeys() - val iter = keys.iterator() - while(iter.hasNext && isRunning) { - var key: SelectionKey = null - try { - key = iter.next - iter.remove() - if(key.isReadable) - read(key) - else if(key.isWritable) - write(key) - else if(!key.isValid) - close(key) - else - throw new IllegalStateException("Unrecognized key state for processor thread.") - } catch { - case e: EOFException => { - debug("Closing socket connection to %s.".format(channelFor(key).socket.getInetAddress)) - close(key) - } case e: InvalidRequestException => { - info("Closing socket connection to %s due to invalid request: %s".format(channelFor(key).socket.getInetAddress, e.getMessage)) - close(key) - } case e: Throwable => { - error("Closing socket for " + channelFor(key).socket.getInetAddress + " because of error", e) - close(key) - } - } + + try { + selector.poll(300) + } catch { + case e @ (_: IllegalStateException | _: IOException) => { + error("Closing processor %s due to illegal state or IO exception".format(id)) + swallow(closeAll()) + shutdownComplete() + throw e } + case e: InvalidReceiveException => + // Log warning and continue since Selector already closed the connection + warn("Connection was closed due to invalid receive. Processor will continue handling other connections") } - maybeCloseOldestConnection + collection.JavaConversions.collectionAsScalaIterable(selector.completedReceives).foreach( receive => { + try { + val req = RequestChannel.Request(processor = id, connectionId = receive.source, buffer = receive.payload, startTimeMs = time.milliseconds, securityProtocol = SecurityProtocol.PLAINTEXT) + requestChannel.sendRequest(req) + } catch { + case e @ (_: InvalidRequestException | _: SchemaException) => { + // note that even though we got an exception, we can assume that receive.source is valid. Issues with constructing a valid receive object were handled earlier + error("Closing socket for " + receive.source + " because of error", e) + selector.close(receive.source) + } + } + selector.mute(receive.source) + }) + + collection.JavaConversions.iterableAsScalaIterable(selector.completedSends()).foreach( send => { + val resp = inflightResponses.remove(send.destination()).get + resp.request.updateRequestMetrics() + selector.unmute(send.destination()) + }) } - debug("Closing selector.") + + + + debug("Closing selector - processor " + id) closeAll() - swallowError(selector.close()) shutdownComplete() } - /** - * Close the given key and associated socket - */ - override def close(key: SelectionKey): Unit = { - lruConnections.remove(key) - super.close(key) - } - private def processNewResponses() { var curr = requestChannel.receiveResponse(id) while(curr != null) { - val key = curr.request.requestKey.asInstanceOf[SelectionKey] try { curr.responseAction match { case RequestChannel.NoOpAction => { @@ -423,26 +413,21 @@ private[kafka] class Processor(val id: Int, // that are sitting in the server's socket buffer curr.request.updateRequestMetrics trace("Socket server received empty response to send, registering for read: " + curr) - key.interestOps(SelectionKey.OP_READ) - key.attach(null) + selector.unmute(curr.request.connectionId) } case RequestChannel.SendAction => { - trace("Socket server received response to send, registering for write: " + curr) - key.interestOps(SelectionKey.OP_WRITE) - key.attach(curr) + trace("Socket server received response to send, registering for write and sending data: " + curr) + selector.send(curr.responseSend) + inflightResponses += (curr.request.connectionId -> curr) } case RequestChannel.CloseConnectionAction => { curr.request.updateRequestMetrics trace("Closing socket connection actively according to the response code.") - close(key) + selector.close(curr.request.connectionId) } - case responseCode => throw new KafkaException("No mapping found for response code " + responseCode) - } - } catch { - case e: CancelledKeyException => { - debug("Ignoring response for closed socket.") - close(key) } + + } finally { curr = requestChannel.receiveResponse(id) } @@ -464,84 +449,27 @@ private[kafka] class Processor(val id: Int, while(!newConnections.isEmpty) { val channel = newConnections.poll() debug("Processor " + id + " listening to new connection from " + channel.socket.getRemoteSocketAddress) - channel.register(selector, SelectionKey.OP_READ) + val localHost = channel.socket().getLocalAddress.getHostAddress + val localPort = channel.socket().getLocalPort + val remoteHost = channel.socket().getInetAddress.getHostAddress + val remotePort = channel.socket().getPort + val connectionId = localHost + ":" + localPort + "-" + remoteHost + ":" + remotePort + selector.register(connectionId, channel) } } - /* - * Process reads from ready sockets + /** + * Close all open connections */ - def read(key: SelectionKey) { - lruConnections.put(key, currentTimeNanos) - val socketChannel = channelFor(key) - var receive = key.attachment.asInstanceOf[Receive] - if(key.attachment == null) { - receive = new BoundedByteBufferReceive(maxRequestSize) - key.attach(receive) - } - val read = receive.readFrom(socketChannel) - val address = socketChannel.socket.getRemoteSocketAddress() - trace(read + " bytes read from " + address) - if(read < 0) { - close(key) - } else if(receive.complete) { - val port = socketChannel.socket().getLocalPort - val protocol = portToProtocol.get(port) - val req = RequestChannel.Request(processor = id, requestKey = key, buffer = receive.buffer, startTimeMs = time.milliseconds, remoteAddress = address, securityProtocol = protocol) - requestChannel.sendRequest(req) - key.attach(null) - // explicitly reset interest ops to not READ, no need to wake up the selector just yet - key.interestOps(key.interestOps & (~SelectionKey.OP_READ)) - } else { - // more reading to be done - trace("Did not finish reading, registering for read again on connection " + socketChannel.socket.getRemoteSocketAddress()) - key.interestOps(SelectionKey.OP_READ) - wakeup() - } + def closeAll() { + selector.close() } - /* - * Process writes to ready sockets + /** + * Wakeup the thread for selection. */ - def write(key: SelectionKey) { - val socketChannel = channelFor(key) - val response = key.attachment().asInstanceOf[RequestChannel.Response] - val responseSend = response.responseSend - if(responseSend == null) - throw new IllegalStateException("Registered for write interest but no response attached to key.") - val written = responseSend.writeTo(socketChannel) - trace(written + " bytes written to " + socketChannel.socket.getRemoteSocketAddress() + " using key " + key) - if(responseSend.complete) { - response.request.updateRequestMetrics() - key.attach(null) - trace("Finished writing, registering for read on connection " + socketChannel.socket.getRemoteSocketAddress()) - key.interestOps(SelectionKey.OP_READ) - } else { - trace("Did not finish writing, registering for write again on connection " + socketChannel.socket.getRemoteSocketAddress()) - key.interestOps(SelectionKey.OP_WRITE) - wakeup() - } - } - - private def channelFor(key: SelectionKey) = key.channel().asInstanceOf[SocketChannel] - - private def maybeCloseOldestConnection { - if(currentTimeNanos > nextIdleCloseCheckTime) { - if(lruConnections.isEmpty) { - nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos - } else { - val oldestConnectionEntry = lruConnections.entrySet.iterator().next() - val connectionLastActiveTime = oldestConnectionEntry.getValue - nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos - if(currentTimeNanos > nextIdleCloseCheckTime) { - val key: SelectionKey = oldestConnectionEntry.getKey - trace("About to close the idle connection from " + key.channel.asInstanceOf[SocketChannel].socket.getRemoteSocketAddress - + " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis") - close(key) - } - } - } - } + @Override + def wakeup = selector.wakeup() } diff --git a/core/src/main/scala/kafka/network/Transmission.scala b/core/src/main/scala/kafka/network/Transmission.scala deleted file mode 100644 index 2827103..0000000 --- a/core/src/main/scala/kafka/network/Transmission.scala +++ /dev/null @@ -1,122 +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.network - -import java.nio._ -import java.nio.channels._ -import kafka.utils.Logging -import kafka.common.KafkaException - -/** - * Represents a stateful transfer of data to or from the network - */ -private[network] trait Transmission extends Logging { - - def complete: Boolean - - protected def expectIncomplete(): Unit = { - if(complete) - throw new KafkaException("This operation cannot be completed on a complete request.") - } - - protected def expectComplete(): Unit = { - if(!complete) - throw new KafkaException("This operation cannot be completed on an incomplete request.") - } - -} - -/** - * A transmission that is being received from a channel - */ -trait Receive extends Transmission { - - def buffer: ByteBuffer - - def readFrom(channel: ReadableByteChannel): Int - - def readCompletely(channel: ReadableByteChannel): Int = { - var totalRead = 0 - while(!complete) { - val read = readFrom(channel) - trace(read + " bytes read.") - totalRead += read - } - totalRead - } - -} - -/** - * A transmission that is being sent out to the channel - */ -trait Send extends Transmission { - - def writeTo(channel: GatheringByteChannel): Int - - def writeCompletely(channel: GatheringByteChannel): Int = { - var totalWritten = 0 - while(!complete) { - val written = writeTo(channel) - trace(written + " bytes written.") - totalWritten += written - } - totalWritten - } - -} - -/** - * A set of composite sends, sent one after another - */ -abstract class MultiSend[S <: Send](val sends: List[S]) extends Send { - val expectedBytesToWrite: Int - private var current = sends - var totalWritten = 0 - - /** - * This method continues to write to the socket buffer till an incomplete - * write happens. On an incomplete write, it returns to the caller to give it - * a chance to schedule other work till the buffered write completes. - */ - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete - var totalWrittenPerCall = 0 - var sendComplete: Boolean = false - do { - val written = current.head.writeTo(channel) - totalWritten += written - totalWrittenPerCall += written - sendComplete = current.head.complete - if(sendComplete) - current = current.tail - } while (!complete && sendComplete) - trace("Bytes written as part of multisend call : " + totalWrittenPerCall + "Total bytes written so far : " + totalWritten + "Expected bytes to write : " + expectedBytesToWrite) - totalWrittenPerCall - } - - def complete: Boolean = { - if (current == Nil) { - if (totalWritten != expectedBytesToWrite) - error("mismatch in sending bytes over socket; expected: " + expectedBytesToWrite + " actual: " + totalWritten) - true - } else { - false - } - } -} diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index 0f09951..dcee501 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -17,11 +17,12 @@ package kafka.producer -import kafka.api._ -import kafka.network.{BlockingChannel, BoundedByteBufferSend, Receive} -import kafka.utils._ import java.util.Random +import kafka.api._ +import kafka.network.{RequestOrResponseSend, BlockingChannel} +import kafka.utils._ +import org.apache.kafka.common.network.NetworkReceive import org.apache.kafka.common.utils.Utils._ object SyncProducer { @@ -50,7 +51,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { * data. So, leaving the rest of the logging at TRACE, while errors should be logged at ERROR level */ if (logger.isDebugEnabled) { - val buffer = new BoundedByteBufferSend(request).buffer + val buffer = new RequestOrResponseSend("", request).buffer trace("verifying sendbuffer of size " + buffer.limit) val requestTypeId = buffer.getShort() if(requestTypeId == RequestKeys.ProduceKey) { @@ -63,12 +64,12 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { /** * Common functionality for the public send methods */ - private def doSend(request: RequestOrResponse, readResponse: Boolean = true): Receive = { + private def doSend(request: RequestOrResponse, readResponse: Boolean = true): NetworkReceive = { lock synchronized { verifyRequest(request) getOrMakeConnection() - var response: Receive = null + var response: NetworkReceive = null try { blockingChannel.send(request) if(readResponse) @@ -95,7 +96,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { producerRequestStats.getProducerRequestStats(config.host, config.port).requestSizeHist.update(requestSize) producerRequestStats.getProducerRequestAllBrokersStats.requestSizeHist.update(requestSize) - var response: Receive = null + var response: NetworkReceive = null val specificTimer = producerRequestStats.getProducerRequestStats(config.host, config.port).requestTimer val aggregateTimer = producerRequestStats.getProducerRequestAllBrokersStats.requestTimer aggregateTimer.time { @@ -104,14 +105,14 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { } } if(producerRequest.requiredAcks != 0) - ProducerResponse.readFrom(response.buffer) + ProducerResponse.readFrom(response.payload) else null } def send(request: TopicMetadataRequest): TopicMetadataResponse = { val response = doSend(request) - TopicMetadataResponse.readFrom(response.buffer) + TopicMetadataResponse.readFrom(response.payload) } def close() = { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 387e387..d63bc18 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -18,7 +18,6 @@ package kafka.server import org.apache.kafka.common.protocol.SecurityProtocol -import org.apache.kafka.common.requests.{JoinGroupResponse, JoinGroupRequest, HeartbeatRequest, HeartbeatResponse, ResponseHeader} import org.apache.kafka.common.TopicPartition import kafka.api._ import kafka.admin.AdminUtils @@ -28,10 +27,9 @@ import kafka.coordinator.ConsumerCoordinator import kafka.log._ import kafka.network._ import kafka.network.RequestChannel.Response +import org.apache.kafka.common.requests.{JoinGroupRequest, JoinGroupResponse, HeartbeatRequest, HeartbeatResponse, ResponseHeader, ResponseSend} import kafka.utils.{ZkUtils, ZKGroupTopicDirs, SystemTime, Logging} - import scala.collection._ - import org.I0Itec.zkclient.ZkClient /** @@ -54,7 +52,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handle(request: RequestChannel.Request) { try{ - trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress) + trace("Handling request: " + request.requestObj + " from connection: " + request.connectionId) request.requestId match { case RequestKeys.ProduceKey => handleProducerRequest(request) case RequestKeys.FetchKey => handleFetchRequest(request) @@ -84,7 +82,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (response == null) requestChannel.closeConnection(request.processor, request) else - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(respHeader, response))) + requestChannel.sendResponse(new Response(request, new ResponseSend(request.connectionId, respHeader, response))) } error("error when handling request %s".format(request.requestObj), e) } finally @@ -99,7 +97,7 @@ class KafkaApis(val requestChannel: RequestChannel, try { val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest, offsetManager) val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, response, error) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(leaderAndIsrResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, leaderAndIsrResponse))) } catch { case e: KafkaStorageException => fatal("Disk error during leadership change.", e) @@ -114,7 +112,7 @@ class KafkaApis(val requestChannel: RequestChannel, val stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest] val (response, error) = replicaManager.stopReplicas(stopReplicaRequest) val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.correlationId, response.toMap, error) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(stopReplicaResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, stopReplicaResponse))) replicaManager.replicaFetcherManager.shutdownIdleFetcherThreads() } @@ -123,7 +121,7 @@ class KafkaApis(val requestChannel: RequestChannel, replicaManager.maybeUpdateMetadataCache(updateMetadataRequest, metadataCache) val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(updateMetadataResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, updateMetadataResponse))) } def handleControlledShutdownRequest(request: RequestChannel.Request) { @@ -134,7 +132,7 @@ class KafkaApis(val requestChannel: RequestChannel, val partitionsRemaining = controller.shutdownBroker(controlledShutdownRequest.brokerId) val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId, ErrorMapping.NoError, partitionsRemaining) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, controlledShutdownResponse))) } @@ -158,7 +156,7 @@ class KafkaApis(val requestChannel: RequestChannel, } val response = OffsetCommitResponse(commitStatus, offsetCommitRequest.correlationId) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } if (offsetCommitRequest.versionId == 0) { @@ -260,7 +258,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } else { val response = ProducerResponse(produceRequest.correlationId, responseStatus) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } } @@ -305,7 +303,7 @@ class KafkaApis(val requestChannel: RequestChannel, } val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } // call the replica manager to fetch messages from the local replica @@ -363,7 +361,7 @@ class KafkaApis(val requestChannel: RequestChannel, } }) val response = OffsetResponse(offsetRequest.correlationId, responseMap) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } def fetchOffsets(logManager: LogManager, topicAndPartition: TopicAndPartition, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { @@ -466,7 +464,7 @@ class KafkaApis(val requestChannel: RequestChannel, val brokers = metadataCache.getAliveBrokers trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) val response = new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(request.securityProtocol)), topicMetadata, metadataRequest.correlationId) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } /* @@ -514,8 +512,10 @@ class KafkaApis(val requestChannel: RequestChannel, } trace("Sending offset fetch response %s for correlation id %d to client %s." - .format(response, offsetFetchRequest.correlationId, offsetFetchRequest.clientId)) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + .format(response, offsetFetchRequest.correlationId, offsetFetchRequest.clientId)) + + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) + } /* @@ -540,7 +540,7 @@ class KafkaApis(val requestChannel: RequestChannel, trace("Sending consumer metadata %s for correlation id %d to client %s." .format(response, consumerMetadataRequest.correlationId, consumerMetadataRequest.clientId)) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } def handleJoinGroupRequest(request: RequestChannel.Request) { @@ -554,8 +554,8 @@ class KafkaApis(val requestChannel: RequestChannel, val partitionList = partitions.map(tp => new TopicPartition(tp.topic, tp.partition)).toBuffer val responseBody = new JoinGroupResponse(errorCode, generationId, consumerId, partitionList) trace("Sending join group response %s for correlation id %d to client %s." - .format(responseBody, request.header.correlationId, request.header.clientId)) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(respHeader, responseBody))) + .format(responseBody, request.header.correlationId, request.header.clientId)) + requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, respHeader, responseBody))) } // let the coordinator to handle join-group @@ -574,10 +574,10 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending a heartbeat response def sendResponseCallback(errorCode: Short) { - val responseBody = new HeartbeatResponse(errorCode) + val response = new HeartbeatResponse(errorCode) trace("Sending heartbeat response %s for correlation id %d to client %s." - .format(responseBody, request.header.correlationId, request.header.clientId)) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(respHeader, responseBody))) + .format(response, request.header.correlationId, request.header.clientId)) + requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, respHeader, response))) } // let the coordinator to handle heartbeat diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 6f25afd..2d75186 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -17,6 +17,7 @@ package kafka.server +import java.util import java.util.Properties import kafka.api.ApiVersion @@ -24,9 +25,11 @@ import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils +import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.config.ConfigDef +import org.apache.kafka.common.metrics.MetricsReporter import org.apache.kafka.common.protocol.SecurityProtocol -import scala.collection.{immutable, JavaConversions, Map} +import scala.collection.{mutable, immutable, JavaConversions, Map} object Defaults { /** ********* Zookeeper Configuration ***********/ @@ -130,6 +133,10 @@ object Defaults { val DeleteTopicEnable = false val CompressionType = "producer" + + val MetricNumSamples = 2 + val MetricSampleWindowMs = 1000 + val MetricReporterClasses = "" } object KafkaConfig { @@ -240,6 +247,10 @@ object KafkaConfig { val DeleteTopicEnableProp = "delete.topic.enable" val CompressionTypeProp = "compression.type" + val MetricSampleWindowMsProp = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG + val MetricNumSamplesProp: String = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG + val MetricReporterClassesProp: String = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG + /* Documentation */ /** ********* Zookeeper Configuration ***********/ @@ -374,6 +385,10 @@ object KafkaConfig { "('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 MetricSampleWindowMsDoc = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC + val MetricNumSamplesDoc = CommonClientConfigs.METRICS_NUM_SAMPLES_DOC + val MetricReporterClassesDoc = CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC + private val configDef = { import ConfigDef.Range._ @@ -494,6 +509,9 @@ object KafkaConfig { .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) .define(CompressionTypeProp, STRING, Defaults.CompressionType, HIGH, CompressionTypeDoc) + .define(MetricNumSamplesProp, INT, Defaults.MetricNumSamples, atLeast(1), LOW, MetricNumSamplesDoc) + .define(MetricSampleWindowMsProp, LONG, Defaults.MetricSampleWindowMs, atLeast(1), LOW, MetricSampleWindowMsDoc) + .define(MetricReporterClassesProp, LIST, Defaults.MetricReporterClasses, LOW, MetricReporterClassesDoc) } def configNames() = { @@ -618,7 +636,10 @@ object KafkaConfig { offsetCommitTimeoutMs = parsed.get(OffsetCommitTimeoutMsProp).asInstanceOf[Int], offsetCommitRequiredAcks = parsed.get(OffsetCommitRequiredAcksProp).asInstanceOf[Short], deleteTopicEnable = parsed.get(DeleteTopicEnableProp).asInstanceOf[Boolean], - compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String] + compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String], + metricNumSamples = parsed.get(MetricNumSamplesProp).asInstanceOf[Int], + metricSampleWindowMs = parsed.get(MetricSampleWindowMsProp).asInstanceOf[Long], + _metricReporterClasses = parsed.get(MetricReporterClassesProp).asInstanceOf[java.util.List[String]] ) } @@ -653,7 +674,7 @@ object KafkaConfig { } } -class KafkaConfig(/** ********* Zookeeper Configuration ***********/ +class KafkaConfig (/** ********* Zookeeper Configuration ***********/ val zkConnect: String, val zkSessionTimeoutMs: Int = Defaults.ZkSessionTimeoutMs, private val _zkConnectionTimeoutMs: Option[Int] = None, @@ -766,7 +787,11 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val offsetCommitRequiredAcks: Short = Defaults.OffsetCommitRequiredAcks, val deleteTopicEnable: Boolean = Defaults.DeleteTopicEnable, - val compressionType: String = Defaults.CompressionType + val compressionType: String = Defaults.CompressionType, + + val metricSampleWindowMs: Long = Defaults.MetricSampleWindowMs, + val metricNumSamples: Int = Defaults.MetricNumSamples, + private val _metricReporterClasses: java.util.List[String] = util.Arrays.asList(Defaults.MetricReporterClasses) ) { val zkConnectionTimeoutMs: Int = _zkConnectionTimeoutMs.getOrElse(zkSessionTimeoutMs) @@ -786,6 +811,8 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val maxConnectionsPerIpOverrides: Map[String, Int] = getMap(KafkaConfig.MaxConnectionsPerIpOverridesProp, _maxConnectionsPerIpOverrides).map { case (k, v) => (k, v.toInt)} + val metricReporterClasses: java.util.List[MetricsReporter] = getMetricClasses(_metricReporterClasses) + private def getLogRetentionTimeMillis: Long = { val millisInMinute = 60L * 1000L val millisInHour = 60L * millisInMinute @@ -850,6 +877,24 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ } } + private def getMetricClasses(metricClasses: java.util.List[String]): java.util.List[MetricsReporter] = { + + val reporterList = new util.ArrayList[MetricsReporter](); + val iterator = metricClasses.iterator() + + while (iterator.hasNext) { + val reporterName = iterator.next() + if (!reporterName.isEmpty) { + val reporter: MetricsReporter = CoreUtils.createObject[MetricsReporter](reporterName) + reporter.configure(toProps.asInstanceOf[java.util.Map[String, _]]) + reporterList.add(reporter) + } + } + + reporterList + + } + validateValues() @@ -992,6 +1037,9 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ props.put(OffsetCommitRequiredAcksProp, offsetCommitRequiredAcks.toString) props.put(DeleteTopicEnableProp, deleteTopicEnable.toString) props.put(CompressionTypeProp, compressionType.toString) + props.put(MetricNumSamplesProp, metricNumSamples.toString) + props.put(MetricSampleWindowMsProp, metricSampleWindowMs.toString) + props.put(MetricReporterClassesProp, JavaConversions.collectionAsScalaIterable(_metricReporterClasses).mkString(",")) props } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index e66710d..b320ce9 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -21,18 +21,21 @@ import kafka.admin._ import kafka.log.LogConfig import kafka.log.CleanerConfig import kafka.log.LogManager -import kafka.utils._ import java.util.concurrent._ import atomic.{AtomicInteger, AtomicBoolean} import java.io.File -import collection.mutable +import kafka.utils._ +import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.network.NetworkReceive + +import scala.collection.{JavaConversions, mutable} import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} import kafka.cluster.{EndPoint, Broker} import kafka.api.{ControlledShutdownResponse, ControlledShutdownRequest} import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBrokerIdException} -import kafka.network.{Receive, BlockingChannel, SocketServer} +import kafka.network.{BlockingChannel, SocketServer} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge import kafka.coordinator.ConsumerCoordinator @@ -48,6 +51,19 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg private var shutdownLatch = new CountDownLatch(1) + private val metricConfig: MetricConfig = new MetricConfig() + .samples(config.metricNumSamples) + .timeWindow(config.metricSampleWindowMs, TimeUnit.MILLISECONDS) + private val jmxPrefix: String = "kafka.server" + private val reporters: java.util.List[MetricsReporter] = config.metricReporterClasses + reporters.add(new JmxReporter(jmxPrefix)) + + + + // This exists so SocketServer (which uses Client libraries) can use the client Time objects without having to convert all of Kafka to use them + // Once we get rid of kafka.utils.time, we can get rid of this too + private val socketServerTime: org.apache.kafka.common.utils.Time = new org.apache.kafka.common.utils.SystemTime() + val brokerState: BrokerState = new BrokerState var apis: KafkaApis = null @@ -117,6 +133,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.brokerId = getBrokerId this.logIdent = "[Kafka Server " + config.brokerId + "], " + val metrics = new Metrics(metricConfig, reporters, socketServerTime) + + socketServer = new SocketServer(config.brokerId, config.listeners, config.numNetworkThreads, @@ -126,7 +145,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.socketRequestMaxBytes, config.maxConnectionsPerIp, config.connectionsMaxIdleMs, - config.maxConnectionsPerIpOverrides) + config.maxConnectionsPerIpOverrides, + socketServerTime, + metrics) socketServer.startup() /* start replica manager */ @@ -262,14 +283,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg // 2. issue a controlled shutdown to the controller if (channel != null) { - var response: Receive = null + var response: NetworkReceive = null try { // send the controlled shutdown request val request = new ControlledShutdownRequest(correlationId.getAndIncrement, config.brokerId) channel.send(request) response = channel.receive() - val shutdownResponse = ControlledShutdownResponse.readFrom(response.buffer) + val shutdownResponse = ControlledShutdownResponse.readFrom(response.payload()) if (shutdownResponse.errorCode == ErrorMapping.NoError && shutdownResponse.partitionsRemaining != null && shutdownResponse.partitionsRemaining.size == 0) { shutdownSucceeded = true diff --git a/core/src/main/scala/kafka/server/MessageSetSend.scala b/core/src/main/scala/kafka/server/MessageSetSend.scala deleted file mode 100644 index 5667648..0000000 --- a/core/src/main/scala/kafka/server/MessageSetSend.scala +++ /dev/null @@ -1,71 +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 java.nio._ -import java.nio.channels._ -import kafka.network._ -import kafka.message._ -import kafka.utils._ -import kafka.common.ErrorMapping - -/** - * A zero-copy message response that writes the bytes needed directly from the file - * wholly in kernel space - */ -@nonthreadsafe -private[server] class MessageSetSend(val messages: MessageSet, val errorCode: Short) extends Send { - - private var sent: Int = 0 - private val size: Int = messages.sizeInBytes - private val header = ByteBuffer.allocate(6) - header.putInt(size + 2) - header.putShort(errorCode) - header.rewind() - - var complete: Boolean = false - - def this(messages: MessageSet) = this(messages, ErrorMapping.NoError) - - def this() = this(MessageSet.Empty) - - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - var written = 0 - if(header.hasRemaining) - written += channel.write(header) - if(!header.hasRemaining) { - val fileBytesSent = messages.writeTo(channel, sent, size - sent) - written += fileBytesSent - sent += fileBytesSent - } - - if(logger.isTraceEnabled) - if (channel.isInstanceOf[SocketChannel]) { - val socketChannel = channel.asInstanceOf[SocketChannel] - logger.trace(sent + " bytes written to " + socketChannel.socket.getRemoteSocketAddress() + " expecting to send " + size + " bytes") - } - - if(sent >= size) - complete = true - written - } - - def sendSize: Int = size + header.capacity - -} diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index ad64cee..3d52f62 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -162,7 +162,7 @@ object ConsumerOffsetChecker extends Logging { debug("Sending offset fetch request to coordinator %s:%d.".format(channel.host, channel.port)) channel.send(OffsetFetchRequest(group, topicPartitions)) - val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().buffer) + val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload()) debug("Received offset fetch response %s.".format(offsetFetchResponse)) offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala index 4e90534..8047da4 100644 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -72,7 +72,7 @@ object TestOffsetManager { offsetsChannel.send(commitRequest) numCommits.getAndIncrement commitTimer.time { - val response = OffsetCommitResponse.readFrom(offsetsChannel.receive().buffer) + val response = OffsetCommitResponse.readFrom(offsetsChannel.receive().payload()) if (response.commitStatus.exists(_._2 != ErrorMapping.NoError)) numErrors.getAndIncrement } offset += 1 @@ -119,7 +119,7 @@ object TestOffsetManager { val group = "group-" + id try { metadataChannel.send(ConsumerMetadataRequest(group)) - val coordinatorId = ConsumerMetadataResponse.readFrom(metadataChannel.receive().buffer).coordinatorOpt.map(_.id).getOrElse(-1) + val coordinatorId = ConsumerMetadataResponse.readFrom(metadataChannel.receive().payload()).coordinatorOpt.map(_.id).getOrElse(-1) val channel = if (channels.contains(coordinatorId)) channels(coordinatorId) @@ -135,7 +135,7 @@ object TestOffsetManager { channel.send(fetchRequest) fetchTimer.time { - val response = OffsetFetchResponse.readFrom(channel.receive().buffer) + val response = OffsetFetchResponse.readFrom(channel.receive().payload()) if (response.requestInfo.exists(_._2.error != ErrorMapping.NoError)) { numErrors.getAndIncrement } diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index bc4aef3..4cb92de 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -20,6 +20,7 @@ import java.io.{FileOutputStream, File} import java.security.Permission import kafka.Kafka +import kafka.server.KafkaConfig import org.junit.{After, Before, Test} import junit.framework.Assert._ @@ -57,20 +58,20 @@ class KafkaTest { val propertiesFile = prepareDefaultConfig() // We should load configuration file without any arguments - val config1 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile)) + val config1 = KafkaConfig.fromProps(Kafka.getPropsFromArgs(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")) + val config2 = KafkaConfig.fromProps(Kafka.getPropsFromArgs(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", "log.cleanup.policy=compact")) + val config3 = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "--override", "log.cleanup.policy=compact"))) assertEquals(1, config3.brokerId) assertEquals("compact", config3.logCleanupPolicy) // We should be also able to set several properties - val config4 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "log.cleanup.policy=compact", "--override", "broker.id=2")) + val config4 = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "--override", "log.cleanup.policy=compact", "--override", "broker.id=2"))) assertEquals(2, config4.brokerId) assertEquals("compact", config4.logCleanupPolicy) } @@ -78,25 +79,25 @@ class KafkaTest { @Test(expected = classOf[ExitCalled]) def testGetKafkaConfigFromArgsWrongSetValue(): Unit = { val propertiesFile = prepareDefaultConfig() - Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "a=b=c")) + KafkaConfig.fromProps(Kafka.getPropsFromArgs(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")) + KafkaConfig.fromProps(Kafka.getPropsFromArgs(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")) + KafkaConfig.fromProps(Kafka.getPropsFromArgs(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")) + KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "broker.id=1", "--override", "broker.id=2"))) } def prepareDefaultConfig(): String = { diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 95d5621..7dc2fad 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -20,7 +20,10 @@ package kafka.network; import java.net._ import java.io._ import kafka.cluster.EndPoint +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.network.NetworkSend import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.utils.SystemTime import org.junit._ import org.scalatest.junit.JUnitSuite import java.util.Random @@ -46,7 +49,9 @@ class SocketServerTest extends JUnitSuite { maxRequestSize = 50, maxConnectionsPerIp = 5, connectionsMaxIdleMs = 60*1000, - maxConnectionsPerIpOverrides = Map.empty[String,Int]) + maxConnectionsPerIpOverrides = Map.empty[String,Int], + new SystemTime(), + new Metrics()) server.startup() def sendRequest(socket: Socket, id: Short, request: Array[Byte]) { @@ -71,7 +76,7 @@ class SocketServerTest extends JUnitSuite { val byteBuffer = ByteBuffer.allocate(request.requestObj.sizeInBytes) request.requestObj.writeTo(byteBuffer) byteBuffer.rewind() - val send = new BoundedByteBufferSend(byteBuffer) + val send = new NetworkSend(request.connectionId, byteBuffer) channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) } @@ -112,33 +117,17 @@ class SocketServerTest extends JUnitSuite { assertEquals(serializedBytes.toSeq, receiveResponse(traceSocket).toSeq) } - @Test(expected = classOf[IOException]) + @Test def tooBigRequestIsRejected() { val tooManyBytes = new Array[Byte](server.maxRequestSize + 1) new Random().nextBytes(tooManyBytes) val socket = connect() sendRequest(socket, 0, tooManyBytes) - receiveResponse(socket) - } - - @Test - def testNullResponse() { - val socket = connect() - val bytes = new Array[Byte](40) - sendRequest(socket, 0, bytes) - - val request = server.requestChannel.receiveRequest - // Since the response is not sent yet, the selection key should not be readable. - TestUtils.waitUntilTrue( - () => { (request.requestKey.asInstanceOf[SelectionKey].interestOps & SelectionKey.OP_READ) != SelectionKey.OP_READ }, - "Socket key shouldn't be available for read") - - server.requestChannel.sendResponse(new RequestChannel.Response(0, request, null)) - - // After the response is sent to the client (which is async and may take a bit of time), the socket key should be available for reads. - TestUtils.waitUntilTrue( - () => { (request.requestKey.asInstanceOf[SelectionKey].interestOps & SelectionKey.OP_READ) == SelectionKey.OP_READ }, - "Socket key should be available for reads") + try { + receiveResponse(socket) + } catch { + case e: IOException => // thats fine + } } @Test @@ -198,7 +187,9 @@ class SocketServerTest extends JUnitSuite { maxRequestSize = 50, maxConnectionsPerIp = 5, connectionsMaxIdleMs = 60*1000, - maxConnectionsPerIpOverrides = overrides) + maxConnectionsPerIpOverrides = overrides, + new SystemTime(), + new Metrics()) overrideServer.startup() // make the maximum allowable number of connections and then leak them val conns = ((0 until overrideNum).map(i => connect(overrideServer))) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 71f48c0..ace6321 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -245,6 +245,10 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { //BrokerCompressionCodec.isValid(compressionType) case KafkaConfig.CompressionTypeProp => expected.setProperty(name, randFrom(BrokerCompressionCodec.brokerCompressionOptions)) + case KafkaConfig.MetricNumSamplesProp => expected.setProperty(name, "2") + case KafkaConfig.MetricSampleWindowMsProp => expected.setProperty(name, "1000") + case KafkaConfig.MetricReporterClassesProp => expected.setProperty(name, "") + case nonNegativeIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString) } }) @@ -348,6 +352,10 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.MetricNumSamplesProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") + case KafkaConfig.MetricSampleWindowMsProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") + case KafkaConfig.MetricReporterClassesProp => // ignore string + case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") } }) -- 1.7.12.4 From ca6d01bc690c2182a611bb65ca46995edc9f356e Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 5 Jun 2015 11:20:25 -0700 Subject: [PATCH 14/59] KAFKA-2246; UnknownTopicOrPartitionException should be an instance of InvalidMetadataException; reviewed by Ewen Cheslack-Postava and Joel Koshy --- .../apache/kafka/common/errors/UnknownTopicOrPartitionException.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java index ec423bd..70fd55c 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java @@ -15,7 +15,7 @@ package org.apache.kafka.common.errors; /** * This topic/partition doesn't exist */ -public class UnknownTopicOrPartitionException extends RetriableException { +public class UnknownTopicOrPartitionException extends InvalidMetadataException { private static final long serialVersionUID = 1L; -- 1.7.12.4 From 9f80665ec6deff8525b61096034af8dc0cc9a03c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 10 Jun 2015 11:28:53 -0700 Subject: [PATCH 15/59] KAFKA-2253; fix deadlock between removeWatchersLock and watcher operations list lock; reviewed by Onur Karaman and Jiangjie Qin --- .../main/scala/kafka/server/DelayedOperation.scala | 45 ++++++++++++++-------- 1 file changed, 29 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedOperation.scala b/core/src/main/scala/kafka/server/DelayedOperation.scala index 123078d..0b53532 100644 --- a/core/src/main/scala/kafka/server/DelayedOperation.scala +++ b/core/src/main/scala/kafka/server/DelayedOperation.scala @@ -189,8 +189,7 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br // 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) - watchers.watch(operation) + watchForOperation(key, operation) if (!watchCreated) { watchCreated = true @@ -241,22 +240,34 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br def delayed() = timeoutTimer.size /* - * Return the watch list of the given key + * Return all the current watcher lists, + * note that the returned watchers may be removed from the list by other threads */ - private def watchersFor(key: Any) = inReadLock(removeWatchersLock) { watchersForKey.getAndMaybePut(key) } + private def allWatchers = inReadLock(removeWatchersLock) { watchersForKey.values } /* - * Return all the current watcher lists + * Return the watch list of the given key, note that we need to + * grab the removeWatchersLock to avoid the operation being added to a removed watcher list */ - private def allWatchers = inReadLock(removeWatchersLock) { watchersForKey.values } + private def watchForOperation(key: Any, operation: T) { + inReadLock(removeWatchersLock) { + val watcher = watchersForKey.getAndMaybePut(key) + watcher.watch(operation) + } + } /* * Remove the key from watcher lists if its list is empty */ - private def removeKeyIfEmpty(key: Any) = inWriteLock(removeWatchersLock) { - val watchers = watchersForKey.get(key) - if (watchers != null && watchers.watched == 0) { - watchersForKey.remove(key) + private def removeKeyIfEmpty(key: Any, watchers: Watchers) { + inWriteLock(removeWatchersLock) { + // if the current key is no longer correlated to the watchers to remove, skip + if (watchersForKey.get(key) != watchers) + return + + if (watchers != null && watchers.watched == 0) { + watchersForKey.remove(key) + } } } @@ -298,10 +309,11 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br iter.remove() } } - - if (operations.size == 0) - removeKeyIfEmpty(key) } + + if (operations.size == 0) + removeKeyIfEmpty(key, this) + completed } @@ -317,10 +329,11 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br purged += 1 } } - - if (operations.size == 0) - removeKeyIfEmpty(key) } + + if (operations.size == 0) + removeKeyIfEmpty(key, this) + purged } } -- 1.7.12.4 From ab76dbd1f7f0bfdc9841baee64cfa37de1d870fb Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Thu, 11 Jun 2015 15:27:51 -0700 Subject: [PATCH 16/59] kafka-2005; Generate html report for system tests; patched by Ashish Singh; reviewed by Jun Rao --- system_test/system_test_runner.py | 135 ++++++++++++++++++++++++++++++++++++-- 1 file changed, 131 insertions(+), 4 deletions(-) diff --git a/system_test/system_test_runner.py b/system_test/system_test_runner.py index 5078d44..d6251b2 100644 --- a/system_test/system_test_runner.py +++ b/system_test/system_test_runner.py @@ -52,6 +52,131 @@ logging.config.fileConfig('logging.conf') # message to facilitate debugging. d = {'name_of_class': '(system_test_runner)'} +class report: + systemTestEnv = None + reportString = "" + reportFileName = "system_test_report.html" + systemTestReport = None + header = """ + Kafka System Test Report + + + + + + """ + footer = """ """ + + def __init__(self, systemTestEnv): + self.totalSkipped = 0 + self.totalPassed = 0 + self.totalTests = 0 + self.totalFailed = 0 + self.systemTestEnv = systemTestEnv + self.systemTestReport = open(self.reportFileName, 'w') + + def __del__(self): + self.systemTestReport.close() + self.systemTestReport = None + + def writeHtmlPage(self, body): + html = """ + + + """ + html += self.header + html += body + html += self.footer + html += """ + + """ + self.systemTestReport.write(html) + + def wrapIn(self, tag, content): + html = "\n<" + tag + ">" + html += "\n " + content + html += "\n" + return html + + def genModal(self, className, caseName, systemTestResult): + key = "validation_status" + id = className + "_" + caseName + info = self.wrapIn("h4", "Validation Status") + for validatedItem in sorted(systemTestResult[key].iterkeys()): + testItemStatus = systemTestResult[key][validatedItem] + info += validatedItem + " : " + testItemStatus + return self.wrapIn("div class=\"modal fade\" id=\"" + id + "\" tabindex=\"-1\" role=\"dialog\" aria-labelledby=\"" + id + "Label\" aria-hidden=\"true\"", + self.wrapIn("div class=\"modal-dialog\"", + self.wrapIn("div class=\"modal-content\"", + self.wrapIn("div class=\"modal-header\"", + self.wrapIn("h4 class=\"modal-title\" id=\"" + id + "Label\"", + className + " - " + caseName)) + + self.wrapIn("div class=\"modal-body\"", + info) + + self.wrapIn("div class=\"modal-footer\"", + self.wrapIn("button type=\"button\" class=\"btn btn-default\" data-dismiss=\"modal\"", "Close"))))) + + def summarize(self): + testItemsTableHeader = self.wrapIn("thead", + self.wrapIn("tr", + self.wrapIn("th", "Test Class Name") + + self.wrapIn("th", "Test Case Name") + + self.wrapIn("th", "Validation Status"))) + testItemsTableBody = "" + modals = "" + + for systemTestResult in self.systemTestEnv.systemTestResultsList: + self.totalTests += 1 + if "_test_class_name" in systemTestResult: + testClassName = systemTestResult["_test_class_name"] + else: + testClassName = "" + + if "_test_case_name" in systemTestResult: + testCaseName = systemTestResult["_test_case_name"] + else: + testCaseName = "" + + if "validation_status" in systemTestResult: + testItemStatus = "SKIPPED" + for key in systemTestResult["validation_status"].iterkeys(): + testItemStatus = systemTestResult["validation_status"][key] + if "FAILED" == testItemStatus: + break; + if "FAILED" == testItemStatus: + self.totalFailed += 1 + validationStatus = self.wrapIn("div class=\"text-danger\" data-toggle=\"modal\" data-target=\"#" + testClassName + "_" + testCaseName + "\"", "FAILED") + modals += self.genModal(testClassName, testCaseName, systemTestResult) + elif "PASSED" == testItemStatus: + self.totalPassed += 1 + validationStatus = self.wrapIn("div class=\"text-success\"", "PASSED") + else: + self.totalSkipped += 1 + validationStatus = self.wrapIn("div class=\"text-warning\"", "SKIPPED") + else: + self.reportString += "|" + + testItemsTableBody += self.wrapIn("tr", + self.wrapIn("td", testClassName) + + self.wrapIn("td", testCaseName) + + self.wrapIn("td", validationStatus)) + + testItemsTableBody = self.wrapIn("tbody", testItemsTableBody) + testItemsTable = self.wrapIn("table class=\"table table-striped\"", testItemsTableHeader + testItemsTableBody) + + statsTblBody = self.wrapIn("tr class=\"active\"", self.wrapIn("td", "Total tests") + self.wrapIn("td", str(self.totalTests))) + statsTblBody += self.wrapIn("tr class=\"success\"", self.wrapIn("td", "Total tests passed") + self.wrapIn("td", str(self.totalPassed))) + statsTblBody += self.wrapIn("tr class=\"danger\"", self.wrapIn("td", "Total tests failed") + self.wrapIn("td", str(self.totalFailed))) + statsTblBody += self.wrapIn("tr class=\"warning\"", self.wrapIn("td", "Total tests skipped") + self.wrapIn("td", str(self.totalSkipped))) + testStatsTable = self.wrapIn("table class=\"table\"", statsTblBody) + + body = self.wrapIn("div class=\"container\"", + self.wrapIn("h2", "Kafka System Test Report") + + self.wrapIn("div class=\"row\"", self.wrapIn("div class=\"col-md-4\"", testStatsTable)) + + self.wrapIn("div class=\"row\"", self.wrapIn("div class=\"col-md-6\"", testItemsTable)) + + modals) + self.writeHtmlPage(self.wrapIn("body", body)) + def main(): nLogger = logging.getLogger('namedLogger') aLogger = logging.getLogger('anonymousLogger') @@ -98,7 +223,7 @@ def main(): nLogger.info("SKIPPING : checking remote machines", extra=d) print - # get all defined names within a module: + # get all defined names within a module: definedItemList = dir(SystemTestEnv) aLogger.debug("=================================================") aLogger.debug("SystemTestEnv keys:") @@ -129,7 +254,7 @@ def main(): and not moduleFileName.startswith("__"): # found a test module file - nLogger.info("found a test module file : " + moduleFileName, extra=d) + nLogger.info("found a test module file : " + moduleFileName, extra=d) testModuleClassName = system_test_utils.sys_call("grep ^class " + testModulePathName + "/" + \ moduleFileName + " | sed 's/^class //g' | sed 's/(.*):.*//g'") @@ -138,7 +263,7 @@ def main(): # collect the test suite class data testSuiteClassDict = {} testSuiteClassDict["suite"] = dirName - extLenToRemove = systemTestEnv.SYSTEM_TEST_MODULE_EXT.__len__() * -1 + extLenToRemove = systemTestEnv.SYSTEM_TEST_MODULE_EXT.__len__() * -1 testSuiteClassDict["module"] = moduleFileName[:extLenToRemove] testSuiteClassDict["class"] = testModuleClassName testSuiteClassDictList.append(testSuiteClassDict) @@ -149,7 +274,7 @@ def main(): # add testsuite directory to sys.path such that the module can be loaded sys.path.append(systemTestEnv.SYSTEM_TEST_BASE_DIR + "/" + suiteName) - + if not systemTestEnv.printTestDescriptionsOnly: aLogger.info("=================================================") aLogger.info("Running Test for : ") @@ -165,6 +290,8 @@ def main(): instance.runTest() print + report(systemTestEnv).summarize() + if not systemTestEnv.printTestDescriptionsOnly: totalFailureCount = 0 print -- 1.7.12.4 From 017c00caf44aaad3418cb99d3ef42c4d1b066ddd Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 12 Jun 2015 10:24:54 -0700 Subject: [PATCH 17/59] kafka-2266; Client Selector can drop idle connections without notifying NetworkClient; patched by Jason Gustafson; reviewed by Jun Rao --- .../java/org/apache/kafka/common/network/Selector.java | 2 ++ .../org/apache/kafka/common/network/SelectorTest.java | 17 ++++++++++++++++- 2 files changed, 18 insertions(+), 1 deletion(-) 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 effb1e6..1da215b 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 @@ -393,6 +393,8 @@ public class Selector implements Selectable { if (log.isTraceEnabled()) log.trace("About to close the idle connection from " + connectionId + " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis"); + + disconnected.add(connectionId); close(connectionId); } } 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 d23b4b6..158f982 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 @@ -26,6 +26,7 @@ import java.util.*; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; import org.junit.After; @@ -40,13 +41,15 @@ public class SelectorTest { private static final int BUFFER_SIZE = 4 * 1024; private EchoServer server; + private Time time; private Selectable selector; @Before public void setup() throws Exception { this.server = new EchoServer(); this.server.start(); - this.selector = new Selector(5000, new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap()); + this.time = new MockTime(); + this.selector = new Selector(5000, new Metrics(), time, "MetricGroup", new LinkedHashMap()); } @After @@ -244,6 +247,18 @@ public class SelectorTest { assertEquals("The response should be from the previously muted node", "1", selector.completedReceives().get(0).source()); } + + @Test + public void testCloseOldestConnection() throws Exception { + String id = "0"; + blockingConnect(id); + + time.sleep(6000); // The max idle time is 5000ms + selector.poll(0); + + assertTrue("The idle connection should have been closed", selector.disconnected().contains(id)); + } + private String blockingRequest(String node, String s) throws IOException { selector.send(createSend(node, s)); selector.poll(1000L); -- 1.7.12.4 From d31a2c2381bebc9c4b27e36fdf986183732e13eb Mon Sep 17 00:00:00 2001 From: Alexander Pakulov Date: Fri, 12 Jun 2015 14:16:03 -0700 Subject: [PATCH 18/59] kafka-2232; make MockProducer generic; patched by Alexander Pakulov; reviewed by Jun Rao --- .../kafka/clients/producer/MockProducer.java | 53 +++++++++++++--------- .../kafka/clients/producer/MockProducerTest.java | 31 +++++++++++-- .../java/org/apache/kafka/test/MockSerializer.java | 1 - 3 files changed, 58 insertions(+), 27 deletions(-) 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 e66491c..36e7ffa 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 @@ -30,6 +30,7 @@ import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; import org.apache.kafka.clients.producer.internals.DefaultPartitioner; import org.apache.kafka.clients.producer.internals.ProduceRequestResult; import org.apache.kafka.common.*; +import org.apache.kafka.common.serialization.Serializer; /** @@ -38,14 +39,16 @@ import org.apache.kafka.common.*; * 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 DefaultPartitioner(); - private final List> sent; + private final Partitioner partitioner; + private final List> sent; private final Deque completions; private boolean autoComplete; private Map offsets; + private final Serializer keySerializer; + private final Serializer valueSerializer; /** * Create a mock producer @@ -55,31 +58,37 @@ public class MockProducer implements Producer { * the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after * {@link #send(ProducerRecord) send()} to complete the call and unblock the @{link * java.util.concurrent.Future Future<RecordMetadata>} that is returned. + * @param partitioner The partition strategy + * @param keySerializer The serializer for key that implements {@link Serializer}. + * @param valueSerializer The serializer for value that implements {@link Serializer}. */ - public MockProducer(Cluster cluster, boolean autoComplete) { + public MockProducer(Cluster cluster, boolean autoComplete, Partitioner partitioner, Serializer keySerializer, Serializer valueSerializer) { this.cluster = cluster; this.autoComplete = autoComplete; + this.partitioner = partitioner; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; this.offsets = new HashMap(); - this.sent = new ArrayList>(); + this.sent = new ArrayList>(); this.completions = new ArrayDeque(); } /** - * Create a new mock producer with invented metadata the given autoComplete setting. + * Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers * - * Equivalent to {@link #MockProducer(Cluster, boolean) new MockProducer(null, autoComplete)} + * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)} */ - public MockProducer(boolean autoComplete) { - this(Cluster.empty(), autoComplete); + public MockProducer(boolean autoComplete, Serializer keySerializer, Serializer valueSerializer) { + this(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer); } /** - * Create a new auto completing mock producer + * Create a new mock producer with invented metadata the given autoComplete setting, partitioner and key\value serializers * - * Equivalent to {@link #MockProducer(boolean) new MockProducer(true)} + * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer)} */ - public MockProducer() { - this(true); + public MockProducer(boolean autoComplete, Partitioner partitioner, Serializer keySerializer, Serializer valueSerializer) { + this(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer); } /** @@ -88,7 +97,7 @@ public class MockProducer implements Producer { * @see #history() */ @Override - public synchronized Future send(ProducerRecord record) { + public synchronized Future send(ProducerRecord record) { return send(record, null); } @@ -98,7 +107,7 @@ public class MockProducer implements Producer { * @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 = partition(record, this.cluster); @@ -154,8 +163,8 @@ public class MockProducer implements Producer { /** * 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); } /** @@ -193,10 +202,11 @@ public class MockProducer implements Producer { /** * computes partition for given record. */ - private int partition(ProducerRecord record, Cluster cluster) { + private int partition(ProducerRecord record, Cluster cluster) { Integer partition = record.partition(); + String topic = record.topic(); if (partition != null) { - List partitions = cluster.partitionsForTopic(record.topic()); + List partitions = cluster.partitionsForTopic(topic); int numPartitions = partitions.size(); // they have given us a partition, use it if (partition < 0 || partition >= numPartitions) @@ -206,10 +216,11 @@ public class MockProducer implements Producer { + "]."); return partition; } - return this.partitioner.partition(record.topic(), null, record.key(), null, record.value(), cluster); + byte[] keyBytes = keySerializer.serialize(topic, record.key()); + byte[] valueBytes = valueSerializer.serialize(topic, record.value()); + return this.partitioner.partition(topic, record.key(), keyBytes, record.value(), valueBytes, cluster); } - private static class Completion { private final long offset; private final RecordMetadata metadata; 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 6372f1a..7a46c56 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 @@ -17,14 +17,22 @@ package org.apache.kafka.clients.producer; import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.ArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import org.apache.kafka.clients.producer.internals.DefaultPartitioner; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.test.MockSerializer; import org.junit.Test; public class MockProducerTest { @@ -34,23 +42,36 @@ public class MockProducerTest { @Test @SuppressWarnings("unchecked") public void testAutoCompleteMock() throws Exception { - MockProducer producer = new MockProducer(true); + MockProducer producer = new MockProducer(true, new MockSerializer(), new MockSerializer()); 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)); 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()); + assertEquals("We should have the record in our history", singletonList(record), producer.history()); + producer.clear(); + assertEquals("Clear should erase our history", 0, producer.history().size()); + } + + @Test + public void testPartitioner() throws Exception { + PartitionInfo partitionInfo0 = new PartitionInfo(topic, 0, null, null, null); + PartitionInfo partitionInfo1 = new PartitionInfo(topic, 1, null, null, null); + Cluster cluster = new Cluster(new ArrayList(0), asList(partitionInfo0, partitionInfo1)); + MockProducer producer = new MockProducer(cluster, true, new DefaultPartitioner(), new StringSerializer(), new StringSerializer()); + ProducerRecord record = new ProducerRecord(topic, "key", "value"); + Future metadata = producer.send(record); + assertEquals("Partition should be correct", 1, metadata.get().partition()); producer.clear(); assertEquals("Clear should erase our history", 0, producer.history().size()); } @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()); + MockProducer producer = new MockProducer(false, new MockSerializer(), new MockSerializer()); + 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/test/MockSerializer.java b/clients/src/test/java/org/apache/kafka/test/MockSerializer.java index e75d2e4..0348258 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockSerializer.java +++ b/clients/src/test/java/org/apache/kafka/test/MockSerializer.java @@ -31,7 +31,6 @@ public class MockSerializer implements Serializer { @Override public void configure(Map configs, boolean isKey) { - } @Override -- 1.7.12.4 From 719f2bddd147d4583a15ee8398bb0edccbbcc3f4 Mon Sep 17 00:00:00 2001 From: Alexey Ozeritskiy Date: Fri, 12 Jun 2015 18:45:48 -0700 Subject: [PATCH 19/59] kafka-2164; ReplicaFetcherThread: suspicious log message on reset offset; patched by Alexey Ozeritski; reviewed by Jun Rao --- core/src/main/scala/kafka/server/ReplicaFetcherThread.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index b31b432..181cbc1 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -111,9 +111,9 @@ class ReplicaFetcherThread(name:String, * Roll out a new log at the follower with the start offset equal to the current leader's start offset and continue fetching. */ val leaderStartOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, OffsetRequest.EarliestTime, brokerConfig.brokerId) - replicaMgr.logManager.truncateFullyAndStartAt(topicAndPartition, leaderStartOffset) warn("Replica %d for partition %s reset its fetch offset from %d to current leader %d's start offset %d" .format(brokerConfig.brokerId, topicAndPartition, replica.logEndOffset.messageOffset, sourceBroker.id, leaderStartOffset)) + replicaMgr.logManager.truncateFullyAndStartAt(topicAndPartition, leaderStartOffset) leaderStartOffset } } -- 1.7.12.4 From 20a31a29f7aa6ce6687a13aa0cf60b92c5ac4d1e Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 15 Jun 2015 17:43:56 -0700 Subject: [PATCH 20/59] kafka-2101; Metric metadata-age is reset on a failed update; patched by Tim Brooks; reviewed by Jun Rao --- .../src/main/java/org/apache/kafka/clients/Metadata.java | 11 +++++++---- .../apache/kafka/clients/producer/internals/Sender.java | 2 +- .../test/java/org/apache/kafka/clients/MetadataTest.java | 14 ++++++++++++++ 3 files changed, 22 insertions(+), 5 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 07f1cdb..0387f26 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -36,6 +36,7 @@ public final class Metadata { private final long metadataExpireMs; private int version; private long lastRefreshMs; + private long lastSuccessfulRefreshMs; private Cluster cluster; private boolean needUpdate; private final Set topics; @@ -57,6 +58,7 @@ public final class Metadata { this.refreshBackoffMs = refreshBackoffMs; this.metadataExpireMs = metadataExpireMs; this.lastRefreshMs = 0L; + this.lastSuccessfulRefreshMs = 0L; this.version = 0; this.cluster = Cluster.empty(); this.needUpdate = false; @@ -83,7 +85,7 @@ public final class Metadata { * is now */ public synchronized long timeToNextUpdate(long nowMs) { - long timeToExpire = needUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0); + long timeToExpire = needUpdate ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0); long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs; return Math.max(timeToExpire, timeToAllowUpdate); } @@ -146,6 +148,7 @@ public final class Metadata { public synchronized void update(Cluster cluster, long now) { this.needUpdate = false; this.lastRefreshMs = now; + this.lastSuccessfulRefreshMs = now; this.version += 1; this.cluster = cluster; notifyAll(); @@ -168,10 +171,10 @@ public final class Metadata { } /** - * The last time metadata was updated. + * The last time metadata was successfully updated. */ - public synchronized long lastUpdate() { - return this.lastRefreshMs; + public synchronized long lastSuccessfulUpdate() { + return this.lastSuccessfulRefreshMs; } /** 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 07e65d4..0baf16e 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 @@ -410,7 +410,7 @@ public class Sender implements Runnable { 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; + return (now - metadata.lastSuccessfulUpdate()) / 1000.0; } }); } diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index 928087d..249d6b8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -85,6 +85,20 @@ public class MetadataTest { } } + @Test + public void testFailedUpdate() { + long time = 100; + metadata.update(Cluster.empty(), time); + + assertEquals(100, metadata.timeToNextUpdate(1000)); + metadata.failedUpdate(1100); + + assertEquals(100, metadata.timeToNextUpdate(1100)); + assertEquals(100, metadata.lastSuccessfulUpdate()); + + } + + private Thread asyncFetch(final String topic) { Thread thread = new Thread() { public void run() { -- 1.7.12.4 From 54e54f08077c9d71a5121e640b55836e6f7f2c9b Mon Sep 17 00:00:00 2001 From: Andrii Biletskyi Date: Tue, 16 Jun 2015 14:46:48 -0700 Subject: [PATCH 21/59] kafka-2195; Add versionId to AbstractRequest.getErrorResponse and AbstractRequest.getRequest; patched by Andrii Biletskyi; reviewed by Jun Rao --- .../kafka/common/requests/AbstractRequest.java | 24 +++---- .../common/requests/ConsumerMetadataRequest.java | 14 ++++- .../apache/kafka/common/requests/FetchRequest.java | 18 ++++-- .../kafka/common/requests/HeartbeatRequest.java | 20 ++++-- .../kafka/common/requests/JoinGroupRequest.java | 26 ++++++-- .../kafka/common/requests/JoinGroupResponse.java | 4 -- .../kafka/common/requests/ListOffsetRequest.java | 14 ++++- .../kafka/common/requests/MetadataRequest.java | 21 ++++++- .../kafka/common/requests/MetadataResponse.java | 73 ++++++++++------------ .../kafka/common/requests/OffsetCommitRequest.java | 14 ++++- .../kafka/common/requests/OffsetFetchRequest.java | 20 ++++-- .../kafka/common/requests/ProduceRequest.java | 17 +++-- .../kafka/common/requests/RequestResponseTest.java | 23 ++++--- .../main/scala/kafka/network/RequestChannel.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 2 +- 15 files changed, 192 insertions(+), 100 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 5e5308e..5d3d528 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -28,33 +28,33 @@ public abstract class AbstractRequest extends AbstractRequestResponse { } /** - * Get an error response for a request + * Get an error response for a request for a given api version */ - public abstract AbstractRequestResponse getErrorResponse(Throwable e); + public abstract AbstractRequestResponse getErrorResponse(int versionId, Throwable e); /** * Factory method for getting a request object based on ApiKey ID and a buffer */ - public static AbstractRequest getRequest(int requestId, ByteBuffer buffer) { + public static AbstractRequest getRequest(int requestId, int versionId, ByteBuffer buffer) { switch (ApiKeys.forId(requestId)) { case PRODUCE: - return ProduceRequest.parse(buffer); + return ProduceRequest.parse(buffer, versionId); case FETCH: - return FetchRequest.parse(buffer); + return FetchRequest.parse(buffer, versionId); case LIST_OFFSETS: - return ListOffsetRequest.parse(buffer); + return ListOffsetRequest.parse(buffer, versionId); case METADATA: - return MetadataRequest.parse(buffer); + return MetadataRequest.parse(buffer, versionId); case OFFSET_COMMIT: - return OffsetCommitRequest.parse(buffer); + return OffsetCommitRequest.parse(buffer, versionId); case OFFSET_FETCH: - return OffsetFetchRequest.parse(buffer); + return OffsetFetchRequest.parse(buffer, versionId); case CONSUMER_METADATA: - return ConsumerMetadataRequest.parse(buffer); + return ConsumerMetadataRequest.parse(buffer, versionId); case JOIN_GROUP: - return JoinGroupRequest.parse(buffer); + return JoinGroupRequest.parse(buffer, versionId); case HEARTBEAT: - return HeartbeatRequest.parse(buffer); + return HeartbeatRequest.parse(buffer, versionId); default: return null; } 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 04b90bf..fef22d7 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 @@ -41,14 +41,24 @@ public class ConsumerMetadataRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { - return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode()); + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch(versionId) { + case 0: + return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CONSUMER_METADATA.id))); + } } public String groupId() { return groupId; } + public static ConsumerMetadataRequest parse(ByteBuffer buffer, int versionId) { + return new ConsumerMetadataRequest(ProtoUtils.parseRequest(ApiKeys.CONSUMER_METADATA.id, versionId, buffer)); + } + public static ConsumerMetadataRequest parse(ByteBuffer buffer) { return new ConsumerMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer)); } 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 8686d83..4f52c32 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 @@ -120,17 +120,23 @@ public class FetchRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map responseData = new HashMap(); for (Map.Entry entry: fetchData.entrySet()) { FetchResponse.PartitionData partitionResponse = new FetchResponse.PartitionData(Errors.forException(e).code(), - FetchResponse.INVALID_HIGHWATERMARK, - FetchResponse.EMPTY_RECORD_SET); + FetchResponse.INVALID_HIGHWATERMARK, + FetchResponse.EMPTY_RECORD_SET); responseData.put(entry.getKey(), partitionResponse); } - return new FetchResponse(responseData); + switch(versionId) { + case 0: + return new FetchResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.FETCH.id))); + } } public int replicaId() { @@ -149,6 +155,10 @@ public class FetchRequest extends AbstractRequest { return fetchData; } + public static FetchRequest parse(ByteBuffer buffer, int versionId) { + return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer)); + } + public static FetchRequest parse(ByteBuffer buffer) { return new FetchRequest((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 51d081f..d4d4a35 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 @@ -48,6 +48,17 @@ public class HeartbeatRequest extends AbstractRequest { consumerId = struct.getString(CONSUMER_ID_KEY_NAME); } + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch(versionId) { + case 0: + return new HeartbeatResponse(Errors.forException(e).code()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.HEARTBEAT.id))); + } + } + public String groupId() { return groupId; } @@ -60,12 +71,11 @@ public class HeartbeatRequest extends AbstractRequest { return consumerId; } - public static HeartbeatRequest parse(ByteBuffer buffer) { - return new HeartbeatRequest((Struct) CURRENT_SCHEMA.read(buffer)); + public static HeartbeatRequest parse(ByteBuffer buffer, int versionId) { + return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer)); } - @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { - return new HeartbeatResponse(Errors.forException(e).code()); + public static HeartbeatRequest parse(ByteBuffer 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/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index 6795682..1ffe076 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 @@ -12,6 +12,7 @@ */ package org.apache.kafka.common.requests; +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; @@ -20,6 +21,7 @@ import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.List; public class JoinGroupRequest extends AbstractRequest { @@ -65,6 +67,21 @@ public class JoinGroupRequest extends AbstractRequest { strategy = struct.getString(STRATEGY_KEY_NAME); } + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch (versionId) { + case 0: + return new JoinGroupResponse( + Errors.forException(e).code(), + JoinGroupResponse.UNKNOWN_GENERATION_ID, + JoinGroupResponse.UNKNOWN_CONSUMER_ID, + Collections.emptyList()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id))); + } + } + public String groupId() { return groupId; } @@ -85,12 +102,11 @@ public class JoinGroupRequest extends AbstractRequest { return strategy; } - public static JoinGroupRequest parse(ByteBuffer buffer) { - return new JoinGroupRequest((Struct) CURRENT_SCHEMA.read(buffer)); + public static JoinGroupRequest parse(ByteBuffer buffer, int versionId) { + return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, versionId, buffer)); } - @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { - return new JoinGroupResponse(Errors.forException(e).code()); + public static JoinGroupRequest parse(ByteBuffer 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 8d418cd..7bf544e 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 @@ -75,10 +75,6 @@ public class JoinGroupResponse extends AbstractRequestResponse { this.assignedPartitions = assignedPartitions; } - public JoinGroupResponse(short errorCode) { - this(errorCode, UNKNOWN_GENERATION_ID, UNKNOWN_CONSUMER_ID, Collections.emptyList()); - } - public JoinGroupResponse(Struct struct) { super(struct); assignedPartitions = new ArrayList(); 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 19267ee..a1bb2b2 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 @@ -107,7 +107,7 @@ public class ListOffsetRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map responseData = new HashMap(); for (Map.Entry entry: offsetData.entrySet()) { @@ -115,7 +115,13 @@ public class ListOffsetRequest extends AbstractRequest { responseData.put(entry.getKey(), partitionResponse); } - return new ListOffsetResponse(responseData); + switch(versionId) { + case 0: + return new ListOffsetResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id))); + } } public int replicaId() { @@ -126,6 +132,10 @@ public class ListOffsetRequest extends AbstractRequest { return offsetData; } + public static ListOffsetRequest parse(ByteBuffer buffer, int versionId) { + return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer)); + } + public static ListOffsetRequest parse(ByteBuffer buffer) { return new ListOffsetRequest((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 7e0ce15..f70e8da 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 @@ -18,6 +18,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; @@ -47,18 +50,30 @@ public class MetadataRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map topicErrors = new HashMap(); - for (String topic: topics) { + for (String topic : topics) { topicErrors.put(topic, Errors.forException(e)); } - return new MetadataResponse(topicErrors); + + Cluster cluster = new Cluster(new ArrayList(), new ArrayList()); + switch (versionId) { + case 0: + return new MetadataResponse(cluster, topicErrors); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id))); + } } public List topics() { return topics; } + public static MetadataRequest parse(ByteBuffer buffer, int versionId) { + return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer)); + } + public static MetadataRequest parse(ByteBuffer 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 44e2ce6..c8f2d08 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,7 +28,7 @@ import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; public class MetadataResponse extends AbstractRequestResponse { - + 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"; @@ -67,30 +67,15 @@ public class MetadataResponse extends AbstractRequestResponse { private final Cluster cluster; private final Map errors; - /* Constructor for error responses where most of the data, except error per topic, is irrelevant */ - public MetadataResponse(Map topicErrors) { - super(new Struct(CURRENT_SCHEMA)); - - struct.set(BROKERS_KEY_NAME, new ArrayList().toArray()); - List topicArray = new ArrayList(); - for (Map.Entry topicError : topicErrors.entrySet()) { - Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, topicError.getValue().code()); - topicData.set(TOPIC_KEY_NAME, topicError.getKey()); - topicData.set(PARTITION_METADATA_KEY_NAME, new ArrayList().toArray()); - topicArray.add(topicData); - } - struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray()); - - this.errors = topicErrors; - this.cluster = new Cluster(new ArrayList(), new ArrayList()); - } - - public MetadataResponse(Cluster cluster) { + /** + * Constructor for MetadataResponse where there are errors for some of the topics, + * error data take precedence over cluster information for particular topic + */ + public MetadataResponse(Cluster cluster, Map errors) { super(new Struct(CURRENT_SCHEMA)); List brokerArray = new ArrayList(); - for (Node node: cluster.nodes()) { + for (Node node : cluster.nodes()) { Struct broker = struct.instance(BROKERS_KEY_NAME); broker.set(NODE_ID_KEY_NAME, node.id()); broker.set(HOST_KEY_NAME, node.host()); @@ -100,27 +85,33 @@ public class MetadataResponse extends AbstractRequestResponse { struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); List topicArray = new ArrayList(); - for (String topic: cluster.topics()) { + for (String topic : cluster.topics()) { Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); - 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, Errors.NONE.code()); - partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition()); - partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id()); - ArrayList replicas = new ArrayList(); - for (Node node: fetchPartitionData.replicas()) - replicas.add(node.id()); - partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); - ArrayList isr = new ArrayList(); - for (Node node: fetchPartitionData.inSyncReplicas()) - isr.add(node.id()); - partitionData.set(ISR_KEY_NAME, isr.toArray()); - partitionArray.add(partitionData); + if (errors.containsKey(topic)) { + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, errors.get(topic).code()); + } else { + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, Errors.NONE.code()); + 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, Errors.NONE.code()); + partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition()); + partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id()); + ArrayList replicas = new ArrayList(); + for (Node node : fetchPartitionData.replicas()) + replicas.add(node.id()); + partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); + ArrayList isr = new ArrayList(); + for (Node node : fetchPartitionData.inSyncReplicas()) + isr.add(node.id()); + partitionData.set(ISR_KEY_NAME, isr.toArray()); + partitionArray.add(partitionData); + } + topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray()); } - topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); } struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray()); @@ -183,4 +174,4 @@ public class MetadataResponse extends AbstractRequestResponse { public static MetadataResponse parse(ByteBuffer buffer) { return new MetadataResponse((Struct) CURRENT_SCHEMA.read(buffer)); } -} +} \ No newline at end of file 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 8bf6cbb..d6e6386 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 @@ -217,12 +217,22 @@ public class OffsetCommitRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map responseData = new HashMap(); for (Map.Entry entry: offsetData.entrySet()) { responseData.put(entry.getKey(), Errors.forException(e).code()); } - return new OffsetCommitResponse(responseData); + + switch (versionId) { + // OffsetCommitResponseV0 == OffsetCommitResponseV1 == OffsetCommitResponseV2 + case 0: + case 1: + case 2: + return new OffsetCommitResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_COMMIT.id))); + } } public String groupId() { 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 deec1fa..b5e8a0f 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 @@ -88,16 +88,24 @@ public class OffsetFetchRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map responseData = new HashMap(); for (TopicPartition partition: partitions) { responseData.put(partition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, - OffsetFetchResponse.NO_METADATA, - Errors.forException(e).code())); + OffsetFetchResponse.NO_METADATA, + Errors.forException(e).code())); } - return new OffsetFetchResponse(responseData); + switch (versionId) { + // OffsetFetchResponseV0 == OffsetFetchResponseV1 + case 0: + case 1: + return new OffsetFetchResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id))); + } } public String groupId() { @@ -108,6 +116,10 @@ public class OffsetFetchRequest extends AbstractRequest { return partitions; } + public static OffsetFetchRequest parse(ByteBuffer buffer, int versionId) { + return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer)); + } + public static OffsetFetchRequest parse(ByteBuffer buffer) { return new OffsetFetchRequest((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 fabeae3..715504b 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 @@ -90,19 +90,24 @@ public class ProduceRequest extends AbstractRequest { } @Override - public AbstractRequestResponse getErrorResponse(Throwable e) { - + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { /* In case the producer doesn't actually want any response */ if (acks == 0) return null; Map responseMap = new HashMap(); - for (Map.Entry entry: partitionRecords.entrySet()) { + for (Map.Entry entry : partitionRecords.entrySet()) { responseMap.put(entry.getKey(), new ProduceResponse.PartitionResponse(Errors.forException(e).code(), ProduceResponse.INVALID_OFFSET)); } - return new ProduceResponse(responseMap); + switch (versionId) { + case 0: + return new ProduceResponse(responseMap); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id))); + } } public short acks() { @@ -117,6 +122,10 @@ public class ProduceRequest extends AbstractRequest { return partitionRecords; } + public static ProduceRequest parse(ByteBuffer buffer, int versionId) { + return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer)); + } + public static ProduceRequest parse(ByteBuffer buffer) { return new ProduceRequest((Struct) CURRENT_SCHEMA.read(buffer)); } 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 e3cc196..8b2aca8 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 @@ -38,31 +38,31 @@ public class RequestResponseTest { createRequestHeader(), createResponseHeader(), createConsumerMetadataRequest(), - createConsumerMetadataRequest().getErrorResponse(new UnknownServerException()), + createConsumerMetadataRequest().getErrorResponse(0, new UnknownServerException()), createConsumerMetadataResponse(), createFetchRequest(), - createFetchRequest().getErrorResponse(new UnknownServerException()), + createFetchRequest().getErrorResponse(0, new UnknownServerException()), createFetchResponse(), createHeartBeatRequest(), - createHeartBeatRequest().getErrorResponse(new UnknownServerException()), + createHeartBeatRequest().getErrorResponse(0, new UnknownServerException()), createHeartBeatResponse(), createJoinGroupRequest(), - createJoinGroupRequest().getErrorResponse(new UnknownServerException()), + createJoinGroupRequest().getErrorResponse(0, new UnknownServerException()), createJoinGroupResponse(), createListOffsetRequest(), - createListOffsetRequest().getErrorResponse(new UnknownServerException()), + createListOffsetRequest().getErrorResponse(0, new UnknownServerException()), createListOffsetResponse(), createMetadataRequest(), - createMetadataRequest().getErrorResponse(new UnknownServerException()), + createMetadataRequest().getErrorResponse(0, new UnknownServerException()), createMetadataResponse(), createOffsetCommitRequest(), - createOffsetCommitRequest().getErrorResponse(new UnknownServerException()), + createOffsetCommitRequest().getErrorResponse(0, new UnknownServerException()), createOffsetCommitResponse(), createOffsetFetchRequest(), - createOffsetFetchRequest().getErrorResponse(new UnknownServerException()), + createOffsetFetchRequest().getErrorResponse(0, new UnknownServerException()), createOffsetFetchResponse(), createProduceRequest(), - createProduceRequest().getErrorResponse(new UnknownServerException()), + createProduceRequest().getErrorResponse(0, new UnknownServerException()), createProduceResponse()); for (AbstractRequestResponse req: requestResponseList) { @@ -145,7 +145,10 @@ public class RequestResponseTest { Node[] isr = new Node[1]; isr[0] = node; Cluster cluster = new Cluster(Arrays.asList(node), Arrays.asList(new PartitionInfo("topic1", 1, node, replicas, isr))); - return new MetadataResponse(cluster); + + Map errors = new HashMap(); + errors.put("topic2", Errors.LEADER_NOT_AVAILABLE); + return new MetadataResponse(cluster, errors); } private AbstractRequest createOffsetCommitRequest() { diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 357d8b9..2074128 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -66,7 +66,7 @@ object RequestChannel extends Logging { null val body: AbstractRequest = if (requestObj == null) - AbstractRequest.getRequest(header.apiKey, buffer) + AbstractRequest.getRequest(header.apiKey, header.apiVersion, buffer) else null diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index d63bc18..c7debe4 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -74,7 +74,7 @@ class KafkaApis(val requestChannel: RequestChannel, if ( request.requestObj != null) request.requestObj.handleError(e, requestChannel, request) else { - val response = request.body.getErrorResponse(e) + val response = request.body.getErrorResponse(request.header.apiVersion, e) val respHeader = new ResponseHeader(request.header.correlationId) /* If request doesn't have a default error response, we just close the connection. -- 1.7.12.4 From 395716ebf0af2520a33ef352a62c10a71239bb4f Mon Sep 17 00:00:00 2001 From: Proneet Verma Date: Tue, 16 Jun 2015 15:03:40 -0700 Subject: [PATCH 22/59] kafka-2270; incorrect package name in unit tests; patched by Proneet Verma; reviewed by Jun Rao --- core/src/test/scala/unit/kafka/KafkaConfigTest.scala | 3 +-- core/src/test/scala/unit/kafka/common/ConfigTest.scala | 3 +-- core/src/test/scala/unit/kafka/common/TopicTest.scala | 3 +-- .../test/scala/unit/kafka/consumer/PartitionAssignorTest.scala | 10 ++++------ .../test/scala/unit/kafka/integration/MinIsrConfigTest.scala | 3 +-- core/src/test/scala/unit/kafka/log/LogConfigTest.scala | 3 +-- .../scala/unit/kafka/server/KafkaConfigConfigDefTest.scala | 3 +-- .../scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala | 5 ++--- .../src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala | 5 ++--- core/src/test/scala/unit/kafka/zk/ZKPathTest.scala | 4 +--- 10 files changed, 15 insertions(+), 27 deletions(-) diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 4cb92de..4764c89 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -14,12 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package unit.kafka +package kafka import java.io.{FileOutputStream, File} import java.security.Permission -import kafka.Kafka import kafka.server.KafkaConfig import org.junit.{After, Before, Test} import junit.framework.Assert._ diff --git a/core/src/test/scala/unit/kafka/common/ConfigTest.scala b/core/src/test/scala/unit/kafka/common/ConfigTest.scala index 7bff96c..0aca938 100644 --- a/core/src/test/scala/unit/kafka/common/ConfigTest.scala +++ b/core/src/test/scala/unit/kafka/common/ConfigTest.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package unit.kafka.common +package kafka.common import junit.framework.Assert._ import collection.mutable.ArrayBuffer import org.junit.Test -import kafka.common.InvalidConfigException import kafka.producer.ProducerConfig import kafka.consumer.ConsumerConfig diff --git a/core/src/test/scala/unit/kafka/common/TopicTest.scala b/core/src/test/scala/unit/kafka/common/TopicTest.scala index 0fb2588..79532c8 100644 --- a/core/src/test/scala/unit/kafka/common/TopicTest.scala +++ b/core/src/test/scala/unit/kafka/common/TopicTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.common +package kafka.common import junit.framework.Assert._ import collection.mutable.ArrayBuffer -import kafka.common.{Topic, InvalidTopicException} import org.junit.Test class TopicTest { diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala index 1910fcb..adf0801 100644 --- a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala @@ -15,20 +15,18 @@ * limitations under the License. */ -package unit.kafka.consumer +package kafka.consumer import org.scalatest.junit.JUnit3Suite import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient import org.apache.zookeeper.data.Stat -import kafka.consumer._ import kafka.utils.{TestUtils, Logging, ZkUtils, Json} import junit.framework.Assert._ import kafka.common.TopicAndPartition -import unit.kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo -import kafka.consumer.ConsumerThreadId -import unit.kafka.consumer.PartitionAssignorTest.Scenario -import unit.kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo +import kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo +import kafka.consumer.PartitionAssignorTest.Scenario +import kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo class PartitionAssignorTest extends JUnit3Suite with Logging { diff --git a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala index 70a5b3a..a2c9713 100644 --- a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.integration +package kafka.integration import java.util.Properties -import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.scalatest.junit.JUnit3Suite diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index f3546ad..3fd5a53 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.log +package kafka.log import java.util.Properties -import kafka.log.{Defaults, LogConfig} import org.apache.kafka.common.config.ConfigException import org.junit.{Assert, Test} import org.scalatest.junit.JUnit3Suite diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index ace6321..c487f36 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -14,13 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package unit.kafka.server +package kafka.server import java.util.Properties import kafka.api.ApiVersion import kafka.message._ -import kafka.server.{Defaults, KafkaConfig} import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.{Assert, Test} import org.scalatest.junit.JUnit3Suite diff --git a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala index fe8d2ae..fd8cf7b 100644 --- a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala @@ -15,13 +15,12 @@ * limitations under the License. */ -package unit.kafka.utils +package kafka.utils import java.util.concurrent.TimeUnit import junit.framework.Assert._ -import org.junit.{Test} -import kafka.utils.ByteBoundedBlockingQueue +import org.junit.Test class ByteBoundedBlockingQueueTest { val sizeFunction = (a: String) => a.length diff --git a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala index e832a01..6380b6e 100644 --- a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.utils +package kafka.utils import junit.framework.Assert._ -import org.junit.{Test, After, Before} -import kafka.utils.CommandLineUtils; +import org.junit.Test class CommandLineUtilsTest { diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala index 64c3ba2..d3e44c6 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala @@ -15,13 +15,11 @@ * limitations under the License. */ -package unit.kafka.zk +package kafka.zk import junit.framework.Assert import kafka.consumer.ConsumerConfig import kafka.utils.{ZkPath, TestUtils, ZkUtils} -import kafka.zk.ZooKeeperTestHarness -import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.config.ConfigException import org.scalatest.junit.JUnit3Suite -- 1.7.12.4 From 28ecea421794d0c9a1c4f95375ccd1a6dfd8f365 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 16 Jun 2015 15:25:16 -0700 Subject: [PATCH 23/59] kafka-2272; listeners endpoint parsing fails if the hostname has capital letter; patched by Sriharsha Chintalapani; reviewed by Jun Rao --- core/src/main/scala/kafka/cluster/EndPoint.scala | 2 +- core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index e9008e6..76997b5 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -42,7 +42,7 @@ object EndPoint { * @return */ def createEndPoint(connectionString: String): EndPoint = { - val uriParseExp = """^(.*)://\[?([0-9a-z\-.:]*)\]?:(-?[0-9]+)""".r + val uriParseExp = """^(.*)://\[?([0-9a-zA-Z\-.:]*)\]?:(-?[0-9]+)""".r connectionString match { case uriParseExp(protocol, "", port) => new EndPoint(null, port.toInt, SecurityProtocol.valueOf(protocol)) case uriParseExp(protocol, host, port) => new EndPoint(host, port.toInt, SecurityProtocol.valueOf(protocol)) diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala index bb2506c..abe511f 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala @@ -120,5 +120,11 @@ class BrokerEndPointTest extends JUnit3Suite with Logging { assert(endpoint.host == "::1") assert(endpoint.port == 9092) assert(endpoint.connectionString == "PLAINTEXT://[::1]:9092") + // test hostname + connectionString = "PLAINTEXT://MyHostname:9092" + endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == "MyHostname") + assert(endpoint.port == 9092) + assert(endpoint.connectionString == "PLAINTEXT://MyHostname:9092") } } -- 1.7.12.4 From 7009f1d6fffe3866723d1d33a28a4572053eb4e5 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Tue, 16 Jun 2015 15:30:52 -0700 Subject: [PATCH 24/59] kafka-2264; SESSION_TIMEOUT_MS_CONFIG in ConsumerConfig should be int; patched by Manikumar Reddy; reviewed by Jun Rao --- .../main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java | 2 +- .../main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 2 +- .../org/apache/kafka/clients/consumer/internals/Coordinator.java | 6 +++--- .../apache/kafka/clients/consumer/internals/CoordinatorTest.java | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 1e90524..daff34d 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 @@ -167,7 +167,7 @@ public class ConsumerConfig extends AbstractConfig { CommonClientConfigs.BOOSTRAP_SERVERS_DOC) .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC) .define(SESSION_TIMEOUT_MS_CONFIG, - Type.LONG, + Type.INT, 30000, Importance.HIGH, SESSION_TIMEOUT_MS_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 d1d1ec1..951c34c 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 @@ -484,7 +484,7 @@ public class KafkaConsumer implements Consumer { this.coordinator = new Coordinator(this.client, config.getString(ConsumerConfig.GROUP_ID_CONFIG), this.retryBackoffMs, - config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), + config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), this.metadata, this.subscriptions, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index c1496a0..41cb945 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -69,7 +69,7 @@ public final class Coordinator { private final String groupId; private final Metadata metadata; private final Heartbeat heartbeat; - private final long sessionTimeoutMs; + private final int sessionTimeoutMs; private final String assignmentStrategy; private final SubscriptionState subscriptions; private final CoordinatorMetrics sensors; @@ -84,7 +84,7 @@ public final class Coordinator { public Coordinator(KafkaClient client, String groupId, long retryBackoffMs, - long sessionTimeoutMs, + int sessionTimeoutMs, String assignmentStrategy, Metadata metadata, SubscriptionState subscriptions, @@ -123,7 +123,7 @@ public final class Coordinator { // repeat processing the response until succeed or fatal error do { JoinGroupRequest request = new JoinGroupRequest(groupId, - (int) this.sessionTimeoutMs, + this.sessionTimeoutMs, subscribedTopics, this.consumerId, this.assignmentStrategy); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java index b06c4a7..1454ab7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java @@ -50,7 +50,7 @@ public class CoordinatorTest { private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); private long retryBackoffMs = 0L; - private long sessionTimeoutMs = 10L; + private int sessionTimeoutMs = 10; private String rebalanceStrategy = "not-matter"; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); -- 1.7.12.4 From 478505632edc8f4b51e4ed561d1adf455256c3e4 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 16 Jun 2015 15:37:58 -0700 Subject: [PATCH 25/59] kafka-2252; Socket connection closing is logged, but not corresponding opening of socket; patched by Gwen Shapira; reviewed by Jun Rao --- clients/src/main/java/org/apache/kafka/common/network/Selector.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 1da215b..4aee214 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 @@ -267,6 +267,7 @@ public class Selector implements Selectable { key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); this.connected.add(transmissions.id); this.sensors.connectionCreated.record(); + log.debug("Connection {} created", transmissions.id); } /* read from any connections that have readable data */ @@ -307,7 +308,7 @@ public class Selector implements Selectable { } catch (IOException e) { String desc = socketDescription(channel); if (e instanceof EOFException || e instanceof ConnectException) - log.info("Connection {} disconnected", desc); + log.debug("Connection {} disconnected", desc); else log.warn("Error in I/O with connection to {}", desc, e); close(transmissions.id); -- 1.7.12.4 From 9650e12df2502f85ea665ae46a982d6fd0d2a954 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Tue, 16 Jun 2015 15:50:17 -0700 Subject: [PATCH 26/59] kafka-2262; LogSegmentSize validation should be consistent; patched by Manikumar Reddy; reviewed by Jun Rao --- core/src/main/scala/kafka/log/LogConfig.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index a907da0..f64fd79 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -22,6 +22,7 @@ import org.apache.kafka.common.utils.Utils import scala.collection._ import org.apache.kafka.common.config.ConfigDef import kafka.message.BrokerCompressionCodec +import kafka.message.Message object Defaults { val SegmentSize = 1024 * 1024 @@ -162,7 +163,7 @@ object LogConfig { import java.util.Arrays.asList new ConfigDef() - .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(0), MEDIUM, SegmentSizeDoc) + .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(Message.MinHeaderSize), 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) -- 1.7.12.4 From ba86f0a25dd8675d3c840c33605494a2f637576f Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 16 Jun 2015 16:41:20 -0700 Subject: [PATCH 27/59] trivial fix for stylecheck error on Jenkins --- .../java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java | 2 +- .../src/main/java/org/apache/kafka/common/requests/FetchRequest.java | 2 +- .../main/java/org/apache/kafka/common/requests/HeartbeatRequest.java | 2 +- .../main/java/org/apache/kafka/common/requests/ListOffsetRequest.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) 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 fef22d7..5b3e04a 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 @@ -42,7 +42,7 @@ public class ConsumerMetadataRequest extends AbstractRequest { @Override public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { - switch(versionId) { + switch (versionId) { case 0: return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode()); default: 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 4f52c32..df073a0 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 @@ -130,7 +130,7 @@ public class FetchRequest extends AbstractRequest { responseData.put(entry.getKey(), partitionResponse); } - switch(versionId) { + switch (versionId) { case 0: return new FetchResponse(responseData); default: 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 d4d4a35..89719f1 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 @@ -50,7 +50,7 @@ public class HeartbeatRequest extends AbstractRequest { @Override public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { - switch(versionId) { + switch (versionId) { case 0: return new HeartbeatResponse(Errors.forException(e).code()); default: 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 a1bb2b2..6da4a0e 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 @@ -115,7 +115,7 @@ public class ListOffsetRequest extends AbstractRequest { responseData.put(entry.getKey(), partitionResponse); } - switch(versionId) { + switch (versionId) { case 0: return new ListOffsetResponse(responseData); default: -- 1.7.12.4 From 5c9040745466945a04ea0315de583ccdab0614ac Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 18 Jun 2015 14:07:33 -0700 Subject: [PATCH 28/59] kafka-2249; KafkaConfig does not preserve original Properties; patched by Gwen Shapira; reviewed by Jun Rao --- .../apache/kafka/common/config/AbstractConfig.java | 12 +- core/src/main/scala/kafka/cluster/Partition.scala | 2 +- .../scala/kafka/controller/KafkaController.scala | 4 +- .../kafka/controller/PartitionLeaderSelector.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 156 ++---- core/src/main/scala/kafka/log/LogManager.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 4 +- core/src/main/scala/kafka/server/KafkaConfig.scala | 573 ++++++--------------- core/src/main/scala/kafka/server/KafkaServer.scala | 55 +- .../scala/kafka/server/ReplicaFetcherThread.scala | 4 +- .../scala/kafka/server/TopicConfigManager.scala | 5 +- core/src/main/scala/kafka/utils/CoreUtils.scala | 26 - .../src/test/scala/other/kafka/StressTestLog.scala | 10 +- .../scala/other/kafka/TestLinearWriteSpeed.scala | 7 +- .../unit/kafka/log/BrokerCompressionTest.scala | 7 +- .../test/scala/unit/kafka/log/CleanerTest.scala | 55 +- .../unit/kafka/log/LogCleanerIntegrationTest.scala | 8 +- .../test/scala/unit/kafka/log/LogConfigTest.scala | 19 +- .../test/scala/unit/kafka/log/LogManagerTest.scala | 17 +- core/src/test/scala/unit/kafka/log/LogTest.scala | 121 +++-- .../kafka/server/DynamicConfigChangeTest.scala | 17 +- .../kafka/server/KafkaConfigConfigDefTest.scala | 20 +- 22 files changed, 444 insertions(+), 682 deletions(-) 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 c4fa058..bae528d 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 @@ -57,15 +57,19 @@ public class AbstractConfig { return values.get(key); } - public int getInt(String key) { + public Short getShort(String key) { + return (Short) get(key); + } + + public Integer getInt(String key) { return (Integer) get(key); } - public long getLong(String key) { + public Long getLong(String key) { return (Long) get(key); } - public double getDouble(String key) { + public Double getDouble(String key) { return (Double) get(key); } @@ -92,7 +96,7 @@ public class AbstractConfig { return keys; } - public Map originals() { + public Map originals() { Map copy = new HashMap(); copy.putAll(originals); return copy; diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 730a232..6cb6477 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -86,7 +86,7 @@ class Partition(val topic: String, case Some(replica) => replica case None => if (isReplicaLocal(replicaId)) { - val config = LogConfig.fromProps(logManager.defaultConfig.toProps, AdminUtils.fetchTopicConfig(zkClient, topic)) + val config = LogConfig.fromProps(logManager.defaultConfig.originals, AdminUtils.fetchTopicConfig(zkClient, topic)) val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) val offsetMap = checkpoint.read diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 69bba24..3635057 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -325,7 +325,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt info("starting the partition rebalance scheduler") autoRebalanceScheduler.startup() autoRebalanceScheduler.schedule("partition-rebalance-thread", checkAndTriggerPartitionRebalance, - 5, config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS) + 5, config.leaderImbalanceCheckIntervalSeconds.toLong, TimeUnit.SECONDS) } deleteTopicManager.start() } @@ -1013,7 +1013,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt // if the replica to be removed from the ISR is the last surviving member of the ISR and unclean leader election // is disallowed for the corresponding topic, then we must preserve the ISR membership so that the replica can // eventually be restored as the leader. - if (newIsr.isEmpty && !LogConfig.fromProps(config.toProps, AdminUtils.fetchTopicConfig(zkClient, + if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(zkClient, topicAndPartition.topic)).uncleanLeaderElectionEnable) { info("Retaining last ISR %d of partition %s since unclean leader election is disabled".format(replicaId, topicAndPartition)) newIsr = leaderAndIsr.isr diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index 3b15ab4..bb6b5c8 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -61,7 +61,7 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi case true => // Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration // for unclean leader election. - if (!LogConfig.fromProps(config.toProps, AdminUtils.fetchTopicConfig(controllerContext.zkClient, + if (!LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(controllerContext.zkClient, topicAndPartition.topic)).uncleanLeaderElectionEnable) { throw new NoReplicaOnlineException(("No broker in ISR for partition " + "%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) + diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index f64fd79..e9af221 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -18,92 +18,52 @@ package kafka.log import java.util.Properties +import kafka.server.KafkaConfig import org.apache.kafka.common.utils.Utils import scala.collection._ -import org.apache.kafka.common.config.ConfigDef +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} import kafka.message.BrokerCompressionCodec import kafka.message.Message 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 - val RetentionMs = Long.MaxValue - val MaxMessageSize = Int.MaxValue - val MaxIndexSize = 1024 * 1024 - val IndexInterval = 4096 - val FileDeleteDelayMs = 60 * 1000L - val DeleteRetentionMs = 24 * 60 * 60 * 1000L - val MinCleanableDirtyRatio = 0.5 - val Compact = false - val UncleanLeaderElectionEnable = true - val MinInSyncReplicas = 1 - val CompressionType = "producer" + val SegmentSize = kafka.server.Defaults.LogSegmentBytes + val SegmentMs = kafka.server.Defaults.LogRollHours * 60 * 60 * 1000L + val SegmentJitterMs = kafka.server.Defaults.LogRollJitterHours * 60 * 60 * 1000L + val FlushInterval = kafka.server.Defaults.LogFlushIntervalMessages + val FlushMs = kafka.server.Defaults.LogFlushSchedulerIntervalMs + val RetentionSize = kafka.server.Defaults.LogRetentionBytes + val RetentionMs = kafka.server.Defaults.LogRetentionHours * 60 * 60 * 1000L + val MaxMessageSize = kafka.server.Defaults.MessageMaxBytes + val MaxIndexSize = kafka.server.Defaults.LogIndexSizeMaxBytes + val IndexInterval = kafka.server.Defaults.LogIndexIntervalBytes + val FileDeleteDelayMs = kafka.server.Defaults.LogDeleteDelayMs + val DeleteRetentionMs = kafka.server.Defaults.LogCleanerDeleteRetentionMs + val MinCleanableDirtyRatio = kafka.server.Defaults.LogCleanerMinCleanRatio + val Compact = kafka.server.Defaults.LogCleanupPolicy + val UncleanLeaderElectionEnable = kafka.server.Defaults.UncleanLeaderElectionEnable + val MinInSyncReplicas = kafka.server.Defaults.MinInSyncReplicas + val CompressionType = kafka.server.Defaults.CompressionType } -/** - * Configuration settings for a 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 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 - * @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(segmentSize: Int = Defaults.SegmentSize, - segmentMs: Long = Defaults.SegmentMs, - segmentJitterMs: Long = Defaults.SegmentJitterMs, - flushInterval: Long = Defaults.FlushInterval, - flushMs: Long = Defaults.FlushMs, - retentionSize: Long = Defaults.RetentionSize, - retentionMs: Long = Defaults.RetentionMs, - maxMessageSize: Int = Defaults.MaxMessageSize, - maxIndexSize: Int = Defaults.MaxIndexSize, - indexInterval: Int = Defaults.IndexInterval, - fileDeleteDelayMs: Long = Defaults.FileDeleteDelayMs, - deleteRetentionMs: Long = Defaults.DeleteRetentionMs, - minCleanableRatio: Double = Defaults.MinCleanableDirtyRatio, - compact: Boolean = Defaults.Compact, - uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable, - minInSyncReplicas: Int = Defaults.MinInSyncReplicas, - compressionType: String = Defaults.CompressionType) { - - def toProps: Properties = { - val props = new Properties() - 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) - props.put(RetentionBytesProp, retentionSize.toString) - props.put(RetentionMsProp, retentionMs.toString) - props.put(MaxMessageBytesProp, maxMessageSize.toString) - props.put(IndexIntervalBytesProp, indexInterval.toString) - props.put(DeleteRetentionMsProp, deleteRetentionMs.toString) - props.put(FileDeleteDelayMsProp, fileDeleteDelayMs.toString) - props.put(MinCleanableDirtyRatioProp, minCleanableRatio.toString) - props.put(CleanupPolicyProp, if(compact) "compact" else "delete") - props.put(UncleanLeaderElectionEnableProp, uncleanLeaderElectionEnable.toString) - props.put(MinInSyncReplicasProp, minInSyncReplicas.toString) - props.put(CompressionTypeProp, compressionType) - props - } +case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props) { + + val segmentSize = getInt(LogConfig.SegmentBytesProp) + val segmentMs = getLong(LogConfig.SegmentMsProp) + val segmentJitterMs = getLong(LogConfig.SegmentJitterMsProp) + val maxIndexSize = getInt(LogConfig.SegmentIndexBytesProp) + val flushInterval = getLong(LogConfig.FlushMessagesProp) + val flushMs = getLong(LogConfig.FlushMsProp) + val retentionSize = getLong(LogConfig.RetentionBytesProp) + val retentionMs = getLong(LogConfig.RetentionMsProp) + val maxMessageSize = getInt(LogConfig.MaxMessageBytesProp) + val indexInterval = getInt(LogConfig.IndexIntervalBytesProp) + val fileDeleteDelayMs = getLong(LogConfig.FileDeleteDelayMsProp) + val deleteRetentionMs = getLong(LogConfig.DeleteRetentionMsProp) + val minCleanableRatio = getDouble(LogConfig.MinCleanableDirtyRatioProp) + val compact = getString(LogConfig.CleanupPolicyProp).toLowerCase != LogConfig.Delete + val uncleanLeaderElectionEnable = getBoolean(LogConfig.UncleanLeaderElectionEnableProp) + val minInSyncReplicas = getInt(LogConfig.MinInSyncReplicasProp) + val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase def randomSegmentJitter: Long = if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs) @@ -111,6 +71,10 @@ case class LogConfig(segmentSize: Int = Defaults.SegmentSize, object LogConfig { + def main(args: Array[String]) { + System.out.println(configDef.toHtmlTable) + } + val Delete = "delete" val Compact = "compact" @@ -179,7 +143,7 @@ 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(Compact, Delete), MEDIUM, + .define(CleanupPolicyProp, STRING, Defaults.Compact, in(Compact, Delete), MEDIUM, CompactDoc) .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, MEDIUM, UncleanLeaderElectionEnableDoc) @@ -187,6 +151,8 @@ object LogConfig { .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc) } + def apply(): LogConfig = LogConfig(new Properties()) + def configNames() = { import JavaConversions._ configDef.names().toList.sorted @@ -194,37 +160,13 @@ object LogConfig { /** - * Parse the given properties instance into a LogConfig object - */ - def fromProps(props: Properties): LogConfig = { - import kafka.utils.CoreUtils.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(RetentionMsProp).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[Boolean], - minInSyncReplicas = parsed.get(MinInSyncReplicasProp).asInstanceOf[Int], - compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String].toLowerCase()) - } - - /** * Create a log config instance using the given properties and defaults */ - def fromProps(defaults: Properties, overrides: Properties): LogConfig = { - val props = new Properties(defaults) + def fromProps(defaults: java.util.Map[_ <: Object, _ <: Object], overrides: Properties): LogConfig = { + val props = new Properties() + props.putAll(defaults) props.putAll(overrides) - fromProps(props) + LogConfig(props) } /** diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index e781eba..538fc83 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -356,7 +356,7 @@ class LogManager(val logDirs: Array[File], .format(topicAndPartition.topic, topicAndPartition.partition, dataDir.getAbsolutePath, - {import JavaConversions._; config.toProps.mkString(", ")})) + {import JavaConversions._; config.originals.mkString(", ")})) log } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c7debe4..ad6f058 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -428,9 +428,9 @@ class KafkaApis(val requestChannel: RequestChannel, val aliveBrokers = metadataCache.getAliveBrokers val offsetsTopicReplicationFactor = if (aliveBrokers.length > 0) - Math.min(config.offsetsTopicReplicationFactor, aliveBrokers.length) + Math.min(config.offsetsTopicReplicationFactor.toInt, aliveBrokers.length) else - config.offsetsTopicReplicationFactor + config.offsetsTopicReplicationFactor.toInt AdminUtils.createTopic(zkClient, topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor, offsetManager.offsetsTopicConfig) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 2d75186..e0b2480 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -26,7 +26,7 @@ import kafka.consumer.ConsumerConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.common.config.ConfigDef +import org.apache.kafka.common.config.{ConfigException, AbstractConfig, ConfigDef} import org.apache.kafka.common.metrics.MetricsReporter import org.apache.kafka.common.protocol.SecurityProtocol import scala.collection.{mutable, immutable, JavaConversions, Map} @@ -141,6 +141,10 @@ object Defaults { object KafkaConfig { + def main(args: Array[String]) { + System.out.println(configDef.toHtmlTable) + } + /** ********* Zookeeper Configuration ***********/ val ZkConnectProp = "zookeeper.connect" val ZkSessionTimeoutMsProp = "zookeeper.session.timeout.ms" @@ -482,14 +486,14 @@ object KafkaConfig { .define(ProducerPurgatoryPurgeIntervalRequestsProp, INT, Defaults.ProducerPurgatoryPurgeIntervalRequests, MEDIUM, ProducerPurgatoryPurgeIntervalRequestsDoc) .define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AutoLeaderRebalanceEnable, HIGH, AutoLeaderRebalanceEnableDoc) .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LeaderImbalancePerBrokerPercentage, HIGH, LeaderImbalancePerBrokerPercentageDoc) - .define(LeaderImbalanceCheckIntervalSecondsProp, INT, Defaults.LeaderImbalanceCheckIntervalSeconds, HIGH, LeaderImbalanceCheckIntervalSecondsDoc) + .define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LeaderImbalanceCheckIntervalSeconds, HIGH, LeaderImbalanceCheckIntervalSecondsDoc) .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, HIGH, UncleanLeaderElectionEnableDoc) .define(InterBrokerSecurityProtocolProp, STRING, Defaults.InterBrokerSecurityProtocol, MEDIUM, InterBrokerSecurityProtocolDoc) .define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, MEDIUM, InterBrokerProtocolVersionDoc) /** ********* Controlled shutdown configuration ***********/ .define(ControlledShutdownMaxRetriesProp, INT, Defaults.ControlledShutdownMaxRetries, MEDIUM, ControlledShutdownMaxRetriesDoc) - .define(ControlledShutdownRetryBackoffMsProp, INT, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) + .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc) /** ********* Consumer coordinator configuration ***********/ @@ -520,139 +524,6 @@ object KafkaConfig { } /** - * Parse the given properties instance into a KafkaConfig object - */ - def fromProps(props: Properties): KafkaConfig = { - import kafka.utils.CoreUtils.evaluateDefaults - val parsed = configDef.parse(evaluateDefaults(props)) - new KafkaConfig( - /** ********* Zookeeper Configuration ***********/ - zkConnect = parsed.get(ZkConnectProp).asInstanceOf[String], - zkSessionTimeoutMs = parsed.get(ZkSessionTimeoutMsProp).asInstanceOf[Int], - _zkConnectionTimeoutMs = Option(parsed.get(ZkConnectionTimeoutMsProp)).map(_.asInstanceOf[Int]), - zkSyncTimeMs = parsed.get(ZkSyncTimeMsProp).asInstanceOf[Int], - - /** ********* General Configuration ***********/ - maxReservedBrokerId = parsed.get(MaxReservedBrokerIdProp).asInstanceOf[Int], - brokerId = parsed.get(BrokerIdProp).asInstanceOf[Int], - messageMaxBytes = parsed.get(MessageMaxBytesProp).asInstanceOf[Int], - numNetworkThreads = parsed.get(NumNetworkThreadsProp).asInstanceOf[Int], - numIoThreads = parsed.get(NumIoThreadsProp).asInstanceOf[Int], - backgroundThreads = parsed.get(BackgroundThreadsProp).asInstanceOf[Int], - queuedMaxRequests = parsed.get(QueuedMaxRequestsProp).asInstanceOf[Int], - - /** ********* Socket Server Configuration ***********/ - port = parsed.get(PortProp).asInstanceOf[Int], - hostName = parsed.get(HostNameProp).asInstanceOf[String], - _listeners = Option(parsed.get(ListenersProp)).map(_.asInstanceOf[String]), - _advertisedHostName = Option(parsed.get(AdvertisedHostNameProp)).map(_.asInstanceOf[String]), - _advertisedPort = Option(parsed.get(AdvertisedPortProp)).map(_.asInstanceOf[Int]), - _advertisedListeners = Option(parsed.get(AdvertisedListenersProp)).map(_.asInstanceOf[String]), - socketSendBufferBytes = parsed.get(SocketSendBufferBytesProp).asInstanceOf[Int], - socketReceiveBufferBytes = parsed.get(SocketReceiveBufferBytesProp).asInstanceOf[Int], - socketRequestMaxBytes = parsed.get(SocketRequestMaxBytesProp).asInstanceOf[Int], - maxConnectionsPerIp = parsed.get(MaxConnectionsPerIpProp).asInstanceOf[Int], - _maxConnectionsPerIpOverrides = parsed.get(MaxConnectionsPerIpOverridesProp).asInstanceOf[String], - connectionsMaxIdleMs = parsed.get(ConnectionsMaxIdleMsProp).asInstanceOf[Long], - - /** ********* Log Configuration ***********/ - numPartitions = parsed.get(NumPartitionsProp).asInstanceOf[Int], - _logDir = parsed.get(LogDirProp).asInstanceOf[String], - _logDirs = Option(parsed.get(LogDirsProp)).map(_.asInstanceOf[String]), - - logSegmentBytes = parsed.get(LogSegmentBytesProp).asInstanceOf[Int], - logRollTimeHours = parsed.get(LogRollTimeHoursProp).asInstanceOf[Int], - _logRollTimeMillis = Option(parsed.get(LogRollTimeMillisProp)).map(_.asInstanceOf[Long]), - - logRollTimeJitterHours = parsed.get(LogRollTimeJitterHoursProp).asInstanceOf[Int], - _logRollTimeJitterMillis = Option(parsed.get(LogRollTimeJitterMillisProp)).map(_.asInstanceOf[Long]), - - logRetentionTimeHours = parsed.get(LogRetentionTimeHoursProp).asInstanceOf[Int], - _logRetentionTimeMins = Option(parsed.get(LogRetentionTimeMinutesProp)).map(_.asInstanceOf[Int]), - _logRetentionTimeMillis = Option(parsed.get(LogRetentionTimeMillisProp)).map(_.asInstanceOf[Long]), - - logRetentionBytes = parsed.get(LogRetentionBytesProp).asInstanceOf[Long], - logCleanupIntervalMs = parsed.get(LogCleanupIntervalMsProp).asInstanceOf[Long], - logCleanupPolicy = parsed.get(LogCleanupPolicyProp).asInstanceOf[String], - logCleanerThreads = parsed.get(LogCleanerThreadsProp).asInstanceOf[Int], - logCleanerIoMaxBytesPerSecond = parsed.get(LogCleanerIoMaxBytesPerSecondProp).asInstanceOf[Double], - logCleanerDedupeBufferSize = parsed.get(LogCleanerDedupeBufferSizeProp).asInstanceOf[Long], - logCleanerIoBufferSize = parsed.get(LogCleanerIoBufferSizeProp).asInstanceOf[Int], - logCleanerDedupeBufferLoadFactor = parsed.get(LogCleanerDedupeBufferLoadFactorProp).asInstanceOf[Double], - logCleanerBackoffMs = parsed.get(LogCleanerBackoffMsProp).asInstanceOf[Long], - logCleanerMinCleanRatio = parsed.get(LogCleanerMinCleanRatioProp).asInstanceOf[Double], - logCleanerEnable = parsed.get(LogCleanerEnableProp).asInstanceOf[Boolean], - logCleanerDeleteRetentionMs = parsed.get(LogCleanerDeleteRetentionMsProp).asInstanceOf[Long], - logIndexSizeMaxBytes = parsed.get(LogIndexSizeMaxBytesProp).asInstanceOf[Int], - logIndexIntervalBytes = parsed.get(LogIndexIntervalBytesProp).asInstanceOf[Int], - logFlushIntervalMessages = parsed.get(LogFlushIntervalMessagesProp).asInstanceOf[Long], - logDeleteDelayMs = parsed.get(LogDeleteDelayMsProp).asInstanceOf[Long], - logFlushSchedulerIntervalMs = parsed.get(LogFlushSchedulerIntervalMsProp).asInstanceOf[Long], - _logFlushIntervalMs = Option(parsed.get(LogFlushIntervalMsProp)).map(_.asInstanceOf[Long]), - logFlushOffsetCheckpointIntervalMs = parsed.get(LogFlushOffsetCheckpointIntervalMsProp).asInstanceOf[Int], - numRecoveryThreadsPerDataDir = parsed.get(NumRecoveryThreadsPerDataDirProp).asInstanceOf[Int], - autoCreateTopicsEnable = parsed.get(AutoCreateTopicsEnableProp).asInstanceOf[Boolean], - minInSyncReplicas = parsed.get(MinInSyncReplicasProp).asInstanceOf[Int], - - /** ********* Replication configuration ***********/ - controllerSocketTimeoutMs = parsed.get(ControllerSocketTimeoutMsProp).asInstanceOf[Int], - defaultReplicationFactor = parsed.get(DefaultReplicationFactorProp).asInstanceOf[Int], - replicaLagTimeMaxMs = parsed.get(ReplicaLagTimeMaxMsProp).asInstanceOf[Long], - replicaSocketTimeoutMs = parsed.get(ReplicaSocketTimeoutMsProp).asInstanceOf[Int], - replicaSocketReceiveBufferBytes = parsed.get(ReplicaSocketReceiveBufferBytesProp).asInstanceOf[Int], - replicaFetchMaxBytes = parsed.get(ReplicaFetchMaxBytesProp).asInstanceOf[Int], - replicaFetchWaitMaxMs = parsed.get(ReplicaFetchWaitMaxMsProp).asInstanceOf[Int], - replicaFetchMinBytes = parsed.get(ReplicaFetchMinBytesProp).asInstanceOf[Int], - replicaFetchBackoffMs = parsed.get(ReplicaFetchBackoffMsProp).asInstanceOf[Int], - numReplicaFetchers = parsed.get(NumReplicaFetchersProp).asInstanceOf[Int], - replicaHighWatermarkCheckpointIntervalMs = parsed.get(ReplicaHighWatermarkCheckpointIntervalMsProp).asInstanceOf[Long], - fetchPurgatoryPurgeIntervalRequests = parsed.get(FetchPurgatoryPurgeIntervalRequestsProp).asInstanceOf[Int], - producerPurgatoryPurgeIntervalRequests = parsed.get(ProducerPurgatoryPurgeIntervalRequestsProp).asInstanceOf[Int], - autoLeaderRebalanceEnable = parsed.get(AutoLeaderRebalanceEnableProp).asInstanceOf[Boolean], - leaderImbalancePerBrokerPercentage = parsed.get(LeaderImbalancePerBrokerPercentageProp).asInstanceOf[Int], - leaderImbalanceCheckIntervalSeconds = parsed.get(LeaderImbalanceCheckIntervalSecondsProp).asInstanceOf[Int], - uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[Boolean], - interBrokerSecurityProtocol = SecurityProtocol.valueOf(parsed.get(InterBrokerSecurityProtocolProp).asInstanceOf[String]), - interBrokerProtocolVersion = ApiVersion(parsed.get(InterBrokerProtocolVersionProp).asInstanceOf[String]), - - /** ********* Controlled shutdown configuration ***********/ - controlledShutdownMaxRetries = parsed.get(ControlledShutdownMaxRetriesProp).asInstanceOf[Int], - controlledShutdownRetryBackoffMs = parsed.get(ControlledShutdownRetryBackoffMsProp).asInstanceOf[Int], - controlledShutdownEnable = parsed.get(ControlledShutdownEnableProp).asInstanceOf[Boolean], - - /** ********* Consumer coordinator configuration ***********/ - consumerMinSessionTimeoutMs = parsed.get(ConsumerMinSessionTimeoutMsProp).asInstanceOf[Int], - consumerMaxSessionTimeoutMs = parsed.get(ConsumerMaxSessionTimeoutMsProp).asInstanceOf[Int], - - /** ********* Offset management configuration ***********/ - offsetMetadataMaxSize = parsed.get(OffsetMetadataMaxSizeProp).asInstanceOf[Int], - offsetsLoadBufferSize = parsed.get(OffsetsLoadBufferSizeProp).asInstanceOf[Int], - offsetsTopicReplicationFactor = parsed.get(OffsetsTopicReplicationFactorProp).asInstanceOf[Short], - offsetsTopicPartitions = parsed.get(OffsetsTopicPartitionsProp).asInstanceOf[Int], - offsetsTopicSegmentBytes = parsed.get(OffsetsTopicSegmentBytesProp).asInstanceOf[Int], - offsetsTopicCompressionCodec = Option(parsed.get(OffsetsTopicCompressionCodecProp)).map(_.asInstanceOf[Int]).map(value => CompressionCodec.getCompressionCodec(value)).orNull, - offsetsRetentionMinutes = parsed.get(OffsetsRetentionMinutesProp).asInstanceOf[Int], - offsetsRetentionCheckIntervalMs = parsed.get(OffsetsRetentionCheckIntervalMsProp).asInstanceOf[Long], - offsetCommitTimeoutMs = parsed.get(OffsetCommitTimeoutMsProp).asInstanceOf[Int], - offsetCommitRequiredAcks = parsed.get(OffsetCommitRequiredAcksProp).asInstanceOf[Short], - deleteTopicEnable = parsed.get(DeleteTopicEnableProp).asInstanceOf[Boolean], - compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String], - metricNumSamples = parsed.get(MetricNumSamplesProp).asInstanceOf[Int], - metricSampleWindowMs = parsed.get(MetricSampleWindowMsProp).asInstanceOf[Long], - _metricReporterClasses = parsed.get(MetricReporterClassesProp).asInstanceOf[java.util.List[String]] - ) - } - - /** - * Create a log config instance using the given properties and defaults - */ - def fromProps(defaults: Properties, overrides: Properties): KafkaConfig = { - val props = new Properties(defaults) - props.putAll(overrides) - fromProps(props) - } - - /** * Check that property names are valid */ def validateNames(props: Properties) { @@ -662,171 +533,149 @@ object KafkaConfig { require(names.contains(name), "Unknown configuration \"%s\".".format(name)) } - /** - * Check that the given properties contain only valid kafka config names and that all values can be parsed and are valid - */ - def validate(props: Properties) { - validateNames(props) - configDef.parse(props) + def fromProps(props: Properties): KafkaConfig = { + KafkaConfig(props) + } - // to bootstrap KafkaConfig.validateValues() - KafkaConfig.fromProps(props) + def fromProps(defaults: Properties, overrides: Properties): KafkaConfig = { + val props = new Properties() + props.putAll(defaults) + props.putAll(overrides) + fromProps(props) } + } -class KafkaConfig (/** ********* Zookeeper Configuration ***********/ - val zkConnect: String, - val zkSessionTimeoutMs: Int = Defaults.ZkSessionTimeoutMs, - private val _zkConnectionTimeoutMs: Option[Int] = None, - val zkSyncTimeMs: Int = Defaults.ZkSyncTimeMs, - - /** ********* General Configuration ***********/ - val maxReservedBrokerId: Int = Defaults.MaxReservedBrokerId, - var brokerId: Int = Defaults.BrokerId, - val messageMaxBytes: Int = Defaults.MessageMaxBytes, - val numNetworkThreads: Int = Defaults.NumNetworkThreads, - val numIoThreads: Int = Defaults.NumIoThreads, - val backgroundThreads: Int = Defaults.BackgroundThreads, - val queuedMaxRequests: Int = Defaults.QueuedMaxRequests, - - /** ********* Socket Server Configuration ***********/ - val port: Int = Defaults.Port, - val hostName: String = Defaults.HostName, - private val _listeners: Option[String] = None, - private val _advertisedHostName: Option[String] = None, - private val _advertisedPort: Option[Int] = None, - private val _advertisedListeners: Option[String] = None, - val socketSendBufferBytes: Int = Defaults.SocketSendBufferBytes, - val socketReceiveBufferBytes: Int = Defaults.SocketReceiveBufferBytes, - val socketRequestMaxBytes: Int = Defaults.SocketRequestMaxBytes, - val maxConnectionsPerIp: Int = Defaults.MaxConnectionsPerIp, - private val _maxConnectionsPerIpOverrides: String = Defaults.MaxConnectionsPerIpOverrides, - val connectionsMaxIdleMs: Long = Defaults.ConnectionsMaxIdleMs, - - /** ********* Log Configuration ***********/ - val numPartitions: Int = Defaults.NumPartitions, - private val _logDir: String = Defaults.LogDir, - private val _logDirs: Option[String] = None, - - val logSegmentBytes: Int = Defaults.LogSegmentBytes, - - val logRollTimeHours: Int = Defaults.LogRollHours, - private val _logRollTimeMillis: Option[Long] = None, - - val logRollTimeJitterHours: Int = Defaults.LogRollJitterHours, - private val _logRollTimeJitterMillis: Option[Long] = None, - - val logRetentionTimeHours: Int = Defaults.LogRetentionHours, - private val _logRetentionTimeMins: Option[Int] = None, - private val _logRetentionTimeMillis: Option[Long] = None, - - val logRetentionBytes: Long = Defaults.LogRetentionBytes, - val logCleanupIntervalMs: Long = Defaults.LogCleanupIntervalMs, - val logCleanupPolicy: String = Defaults.LogCleanupPolicy, - val logCleanerThreads: Int = Defaults.LogCleanerThreads, - val logCleanerIoMaxBytesPerSecond: Double = Defaults.LogCleanerIoMaxBytesPerSecond, - val logCleanerDedupeBufferSize: Long = Defaults.LogCleanerDedupeBufferSize, - val logCleanerIoBufferSize: Int = Defaults.LogCleanerIoBufferSize, - val logCleanerDedupeBufferLoadFactor: Double = Defaults.LogCleanerDedupeBufferLoadFactor, - val logCleanerBackoffMs: Long = Defaults.LogCleanerBackoffMs, - val logCleanerMinCleanRatio: Double = Defaults.LogCleanerMinCleanRatio, - val logCleanerEnable: Boolean = Defaults.LogCleanerEnable, - val logCleanerDeleteRetentionMs: Long = Defaults.LogCleanerDeleteRetentionMs, - val logIndexSizeMaxBytes: Int = Defaults.LogIndexSizeMaxBytes, - val logIndexIntervalBytes: Int = Defaults.LogIndexIntervalBytes, - val logFlushIntervalMessages: Long = Defaults.LogFlushIntervalMessages, - val logDeleteDelayMs: Long = Defaults.LogDeleteDelayMs, - val logFlushSchedulerIntervalMs: Long = Defaults.LogFlushSchedulerIntervalMs, - private val _logFlushIntervalMs: Option[Long] = None, - val logFlushOffsetCheckpointIntervalMs: Int = Defaults.LogFlushOffsetCheckpointIntervalMs, - val numRecoveryThreadsPerDataDir: Int = Defaults.NumRecoveryThreadsPerDataDir, - val autoCreateTopicsEnable: Boolean = Defaults.AutoCreateTopicsEnable, - - val minInSyncReplicas: Int = Defaults.MinInSyncReplicas, - - /** ********* Replication configuration ***********/ - val controllerSocketTimeoutMs: Int = Defaults.ControllerSocketTimeoutMs, - val defaultReplicationFactor: Int = Defaults.DefaultReplicationFactor, - val replicaLagTimeMaxMs: Long = Defaults.ReplicaLagTimeMaxMs, - val replicaSocketTimeoutMs: Int = Defaults.ReplicaSocketTimeoutMs, - val replicaSocketReceiveBufferBytes: Int = Defaults.ReplicaSocketReceiveBufferBytes, - val replicaFetchMaxBytes: Int = Defaults.ReplicaFetchMaxBytes, - val replicaFetchWaitMaxMs: Int = Defaults.ReplicaFetchWaitMaxMs, - val replicaFetchMinBytes: Int = Defaults.ReplicaFetchMinBytes, - val replicaFetchBackoffMs: Int = Defaults.ReplicaFetchBackoffMs, - val numReplicaFetchers: Int = Defaults.NumReplicaFetchers, - val replicaHighWatermarkCheckpointIntervalMs: Long = Defaults.ReplicaHighWatermarkCheckpointIntervalMs, - val fetchPurgatoryPurgeIntervalRequests: Int = Defaults.FetchPurgatoryPurgeIntervalRequests, - val producerPurgatoryPurgeIntervalRequests: Int = Defaults.ProducerPurgatoryPurgeIntervalRequests, - val autoLeaderRebalanceEnable: Boolean = Defaults.AutoLeaderRebalanceEnable, - val leaderImbalancePerBrokerPercentage: Int = Defaults.LeaderImbalancePerBrokerPercentage, - val leaderImbalanceCheckIntervalSeconds: Int = Defaults.LeaderImbalanceCheckIntervalSeconds, - val uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable, - val interBrokerSecurityProtocol: SecurityProtocol = SecurityProtocol.valueOf(Defaults.InterBrokerSecurityProtocol), - val interBrokerProtocolVersion: ApiVersion = ApiVersion(Defaults.InterBrokerProtocolVersion), - - /** ********* Controlled shutdown configuration ***********/ - val controlledShutdownMaxRetries: Int = Defaults.ControlledShutdownMaxRetries, - val controlledShutdownRetryBackoffMs: Int = Defaults.ControlledShutdownRetryBackoffMs, - val controlledShutdownEnable: Boolean = Defaults.ControlledShutdownEnable, - - /** ********* Consumer coordinator configuration ***********/ - val consumerMinSessionTimeoutMs: Int = Defaults.ConsumerMinSessionTimeoutMs, - val consumerMaxSessionTimeoutMs: Int = Defaults.ConsumerMaxSessionTimeoutMs, - - /** ********* Offset management configuration ***********/ - val offsetMetadataMaxSize: Int = Defaults.OffsetMetadataMaxSize, - val offsetsLoadBufferSize: Int = Defaults.OffsetsLoadBufferSize, - val offsetsTopicReplicationFactor: Short = Defaults.OffsetsTopicReplicationFactor, - val offsetsTopicPartitions: Int = Defaults.OffsetsTopicPartitions, - val offsetsTopicSegmentBytes: Int = Defaults.OffsetsTopicSegmentBytes, - val offsetsTopicCompressionCodec: CompressionCodec = CompressionCodec.getCompressionCodec(Defaults.OffsetsTopicCompressionCodec), - val offsetsRetentionMinutes: Int = Defaults.OffsetsRetentionMinutes, - val offsetsRetentionCheckIntervalMs: Long = Defaults.OffsetsRetentionCheckIntervalMs, - val offsetCommitTimeoutMs: Int = Defaults.OffsetCommitTimeoutMs, - val offsetCommitRequiredAcks: Short = Defaults.OffsetCommitRequiredAcks, - - val deleteTopicEnable: Boolean = Defaults.DeleteTopicEnable, - val compressionType: String = Defaults.CompressionType, - - val metricSampleWindowMs: Long = Defaults.MetricSampleWindowMs, - val metricNumSamples: Int = Defaults.MetricNumSamples, - private val _metricReporterClasses: java.util.List[String] = util.Arrays.asList(Defaults.MetricReporterClasses) - ) { - - val zkConnectionTimeoutMs: Int = _zkConnectionTimeoutMs.getOrElse(zkSessionTimeoutMs) - - val listeners = getListeners() - val advertisedHostName: String = _advertisedHostName.getOrElse(hostName) - val advertisedPort: Int = _advertisedPort.getOrElse(port) - val advertisedListeners = getAdvertisedListeners() - val logDirs = CoreUtils.parseCsvList(_logDirs.getOrElse(_logDir)) - - val logRollTimeMillis = _logRollTimeMillis.getOrElse(60 * 60 * 1000L * logRollTimeHours) - val logRollTimeJitterMillis = _logRollTimeJitterMillis.getOrElse(60 * 60 * 1000L * logRollTimeJitterHours) - val logRetentionTimeMillis = getLogRetentionTimeMillis +case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(KafkaConfig.configDef, props) { + + /** ********* Zookeeper Configuration ***********/ + val zkConnect: String = getString(KafkaConfig.ZkConnectProp) + val zkSessionTimeoutMs: Int = getInt(KafkaConfig.ZkSessionTimeoutMsProp) + val zkConnectionTimeoutMs: java.lang.Integer = + Option(getInt(KafkaConfig.ZkConnectionTimeoutMsProp)).getOrElse(getInt(KafkaConfig.ZkSessionTimeoutMsProp)) + val zkSyncTimeMs: Int = getInt(KafkaConfig.ZkSyncTimeMsProp) - val logFlushIntervalMs = _logFlushIntervalMs.getOrElse(logFlushSchedulerIntervalMs) + /** ********* General Configuration ***********/ + val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) + var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) + val numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp) + val backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp) + val queuedMaxRequests = getInt(KafkaConfig.QueuedMaxRequestsProp) + val numIoThreads = getInt(KafkaConfig.NumIoThreadsProp) + val messageMaxBytes = getInt(KafkaConfig.MessageMaxBytesProp) + /** ********* Socket Server Configuration ***********/ + val hostName = getString(KafkaConfig.HostNameProp) + val port = getInt(KafkaConfig.PortProp) + val advertisedHostName = Option(getString(KafkaConfig.AdvertisedHostNameProp)).getOrElse(hostName) + val advertisedPort: java.lang.Integer = Option(getInt(KafkaConfig.AdvertisedPortProp)).getOrElse(port) + + val socketSendBufferBytes = getInt(KafkaConfig.SocketSendBufferBytesProp) + val socketReceiveBufferBytes = getInt(KafkaConfig.SocketReceiveBufferBytesProp) + val socketRequestMaxBytes = getInt(KafkaConfig.SocketRequestMaxBytesProp) + val maxConnectionsPerIp = getInt(KafkaConfig.MaxConnectionsPerIpProp) val maxConnectionsPerIpOverrides: Map[String, Int] = - getMap(KafkaConfig.MaxConnectionsPerIpOverridesProp, _maxConnectionsPerIpOverrides).map { case (k, v) => (k, v.toInt)} + getMap(KafkaConfig.MaxConnectionsPerIpOverridesProp, getString(KafkaConfig.MaxConnectionsPerIpOverridesProp)).map { case (k, v) => (k, v.toInt)} + val connectionsMaxIdleMs = getLong(KafkaConfig.ConnectionsMaxIdleMsProp) + - val metricReporterClasses: java.util.List[MetricsReporter] = getMetricClasses(_metricReporterClasses) + /** ********* Log Configuration ***********/ + val autoCreateTopicsEnable = getBoolean(KafkaConfig.AutoCreateTopicsEnableProp) + val numPartitions = getInt(KafkaConfig.NumPartitionsProp) + val logDirs = CoreUtils.parseCsvList( Option(getString(KafkaConfig.LogDirsProp)).getOrElse(getString(KafkaConfig.LogDirProp))) + val logSegmentBytes = getInt(KafkaConfig.LogSegmentBytesProp) + val logFlushIntervalMessages = getLong(KafkaConfig.LogFlushIntervalMessagesProp) + val logCleanerThreads = getInt(KafkaConfig.LogCleanerThreadsProp) + val numRecoveryThreadsPerDataDir = getInt(KafkaConfig.NumRecoveryThreadsPerDataDirProp) + val logFlushSchedulerIntervalMs = getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp) + val logFlushOffsetCheckpointIntervalMs = getInt(KafkaConfig.LogFlushOffsetCheckpointIntervalMsProp).toLong + val logCleanupIntervalMs = getLong(KafkaConfig.LogCleanupIntervalMsProp) + val logCleanupPolicy = getString(KafkaConfig.LogCleanupPolicyProp) + val offsetsRetentionMinutes = getInt(KafkaConfig.OffsetsRetentionMinutesProp) + val offsetsRetentionCheckIntervalMs = getLong(KafkaConfig.OffsetsRetentionCheckIntervalMsProp) + val logRetentionBytes = getLong(KafkaConfig.LogRetentionBytesProp) + val logCleanerDedupeBufferSize = getLong(KafkaConfig.LogCleanerDedupeBufferSizeProp) + val logCleanerDedupeBufferLoadFactor = getDouble(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp) + val logCleanerIoBufferSize = getInt(KafkaConfig.LogCleanerIoBufferSizeProp) + val logCleanerIoMaxBytesPerSecond = getDouble(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp) + val logCleanerDeleteRetentionMs = getLong(KafkaConfig.LogCleanerDeleteRetentionMsProp) + val logCleanerBackoffMs = getLong(KafkaConfig.LogCleanerBackoffMsProp) + val logCleanerMinCleanRatio = getDouble(KafkaConfig.LogCleanerMinCleanRatioProp) + val logCleanerEnable = getBoolean(KafkaConfig.LogCleanerEnableProp) + val logIndexSizeMaxBytes = getInt(KafkaConfig.LogIndexSizeMaxBytesProp) + val logIndexIntervalBytes = getInt(KafkaConfig.LogIndexIntervalBytesProp) + val logDeleteDelayMs = getLong(KafkaConfig.LogDeleteDelayMsProp) + val logRollTimeMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeHoursProp)) + val logRollTimeJitterMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeJitterMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeJitterHoursProp)) + val logFlushIntervalMs: java.lang.Long = Option(getLong(KafkaConfig.LogFlushIntervalMsProp)).getOrElse(getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp)) + val minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp) + + /** ********* Replication configuration ***********/ + val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp) + val defaultReplicationFactor: Int = getInt(KafkaConfig.DefaultReplicationFactorProp) + val replicaLagTimeMaxMs = getLong(KafkaConfig.ReplicaLagTimeMaxMsProp) + val replicaSocketTimeoutMs = getInt(KafkaConfig.ReplicaSocketTimeoutMsProp) + val replicaSocketReceiveBufferBytes = getInt(KafkaConfig.ReplicaSocketReceiveBufferBytesProp) + val replicaFetchMaxBytes = getInt(KafkaConfig.ReplicaFetchMaxBytesProp) + val replicaFetchWaitMaxMs = getInt(KafkaConfig.ReplicaFetchWaitMaxMsProp) + val replicaFetchMinBytes = getInt(KafkaConfig.ReplicaFetchMinBytesProp) + val replicaFetchBackoffMs = getInt(KafkaConfig.ReplicaFetchBackoffMsProp) + val numReplicaFetchers = getInt(KafkaConfig.NumReplicaFetchersProp) + val replicaHighWatermarkCheckpointIntervalMs = getLong(KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp) + val fetchPurgatoryPurgeIntervalRequests = getInt(KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp) + val producerPurgatoryPurgeIntervalRequests = getInt(KafkaConfig.ProducerPurgatoryPurgeIntervalRequestsProp) + val autoLeaderRebalanceEnable = getBoolean(KafkaConfig.AutoLeaderRebalanceEnableProp) + val leaderImbalancePerBrokerPercentage = getInt(KafkaConfig.LeaderImbalancePerBrokerPercentageProp) + val leaderImbalanceCheckIntervalSeconds = getLong(KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp) + val uncleanLeaderElectionEnable = getBoolean(KafkaConfig.UncleanLeaderElectionEnableProp) + val interBrokerSecurityProtocol = SecurityProtocol.valueOf(getString(KafkaConfig.InterBrokerSecurityProtocolProp)) + val interBrokerProtocolVersion = ApiVersion(getString(KafkaConfig.InterBrokerProtocolVersionProp)) + + /** ********* Controlled shutdown configuration ***********/ + val controlledShutdownMaxRetries = getInt(KafkaConfig.ControlledShutdownMaxRetriesProp) + val controlledShutdownRetryBackoffMs = getLong(KafkaConfig.ControlledShutdownRetryBackoffMsProp) + val controlledShutdownEnable = getBoolean(KafkaConfig.ControlledShutdownEnableProp) + + /** ********* Consumer coordinator configuration ***********/ + val consumerMinSessionTimeoutMs = getInt(KafkaConfig.ConsumerMinSessionTimeoutMsProp) + val consumerMaxSessionTimeoutMs = getInt(KafkaConfig.ConsumerMaxSessionTimeoutMsProp) + + /** ********* Offset management configuration ***********/ + val offsetMetadataMaxSize = getInt(KafkaConfig.OffsetMetadataMaxSizeProp) + val offsetsLoadBufferSize = getInt(KafkaConfig.OffsetsLoadBufferSizeProp) + val offsetsTopicReplicationFactor = getShort(KafkaConfig.OffsetsTopicReplicationFactorProp) + val offsetsTopicPartitions = getInt(KafkaConfig.OffsetsTopicPartitionsProp) + val offsetCommitTimeoutMs = getInt(KafkaConfig.OffsetCommitTimeoutMsProp) + val offsetCommitRequiredAcks = getShort(KafkaConfig.OffsetCommitRequiredAcksProp) + val offsetsTopicSegmentBytes = getInt(KafkaConfig.OffsetsTopicSegmentBytesProp) + val offsetsTopicCompressionCodec = Option(getInt(KafkaConfig.OffsetsTopicCompressionCodecProp)).map(value => CompressionCodec.getCompressionCodec(value)).orNull + + /** ********* Metric Configuration **************/ + val metricNumSamples = getInt(KafkaConfig.MetricNumSamplesProp) + val metricSampleWindowMs = getLong(KafkaConfig.MetricSampleWindowMsProp) + val metricReporterClasses: java.util.List[MetricsReporter] = getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter]) + + val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) + val compressionType = getString(KafkaConfig.CompressionTypeProp) + + + val listeners = getListeners + val advertisedListeners = getAdvertisedListeners + val logRetentionTimeMillis = getLogRetentionTimeMillis private def getLogRetentionTimeMillis: Long = { val millisInMinute = 60L * 1000L val millisInHour = 60L * millisInMinute - val millis = { - _logRetentionTimeMillis.getOrElse( - _logRetentionTimeMins match { - case Some(mins) => millisInMinute * mins - case None => millisInHour * logRetentionTimeHours - } - ) - } - if (millis < 0) return -1 - millis + val millis: java.lang.Long = + Option(getLong(KafkaConfig.LogRetentionTimeMillisProp)).getOrElse( + Option(getInt(KafkaConfig.LogRetentionTimeMinutesProp)) match { + case Some(mins) => millisInMinute * mins + case None => getInt(KafkaConfig.LogRetentionTimeHoursProp) * millisInHour + }) + + if (millis < 0) return -1 + millis } private def getMap(propName: String, propValue: String): Map[String, String] = { @@ -855,9 +704,9 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ // If the user did not define listeners but did define host or port, let's use them in backward compatible way // If none of those are defined, we default to PLAINTEXT://:9092 private def getListeners(): immutable.Map[SecurityProtocol, EndPoint] = { - if (_listeners.isDefined) { - validateUniquePortAndProtocol(_listeners.get) - CoreUtils.listenerListToEndPoints(_listeners.get) + if (getString(KafkaConfig.ListenersProp) != null) { + validateUniquePortAndProtocol(getString(KafkaConfig.ListenersProp)) + CoreUtils.listenerListToEndPoints(getString(KafkaConfig.ListenersProp)) } else { CoreUtils.listenerListToEndPoints("PLAINTEXT://" + hostName + ":" + port) } @@ -867,11 +716,12 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ // If he didn't but did define advertised host or port, we'll use those and fill in the missing value from regular host / port or defaults // If none of these are defined, we'll use the listeners private def getAdvertisedListeners(): immutable.Map[SecurityProtocol, EndPoint] = { - if (_advertisedListeners.isDefined) { - validateUniquePortAndProtocol(_advertisedListeners.get) - CoreUtils.listenerListToEndPoints(_advertisedListeners.get) - } else if (_advertisedHostName.isDefined || _advertisedPort.isDefined ) { - CoreUtils.listenerListToEndPoints("PLAINTEXT://" + advertisedHostName + ":" + advertisedPort) + if (getString(KafkaConfig.AdvertisedListenersProp) != null) { + validateUniquePortAndProtocol(getString(KafkaConfig.AdvertisedListenersProp)) + CoreUtils.listenerListToEndPoints(getString(KafkaConfig.AdvertisedListenersProp)) + } else if (getString(KafkaConfig.AdvertisedHostNameProp) != null || getInt(KafkaConfig.AdvertisedPortProp) != null) { + CoreUtils.listenerListToEndPoints("PLAINTEXT://" + + getString(KafkaConfig.AdvertisedHostNameProp) + ":" + getInt(KafkaConfig.AdvertisedPortProp)) } else { getListeners() } @@ -886,7 +736,7 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ val reporterName = iterator.next() if (!reporterName.isEmpty) { val reporter: MetricsReporter = CoreUtils.createObject[MetricsReporter](reporterName) - reporter.configure(toProps.asInstanceOf[java.util.Map[String, _]]) + reporter.configure(originals) reporterList.add(reporter) } } @@ -895,19 +745,13 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ } - - validateValues() private def validateValues() { require(brokerId >= -1 && brokerId <= maxReservedBrokerId, "broker.id must be equal or greater than -1 and not greater than reserved.broker.max.id") require(logRollTimeMillis >= 1, "log.roll.ms must be equal or greater than 1") require(logRollTimeJitterMillis >= 0, "log.roll.jitter.ms must be equal or greater than 0") - - require(_logRetentionTimeMins.forall(_ >= 1)|| _logRetentionTimeMins.forall(_ .equals(-1)), "log.retention.minutes must be unlimited (-1) or, equal or greater than 1") - require(logRetentionTimeHours >= 1 || logRetentionTimeHours == -1, "log.retention.hours must be unlimited (-1) or, equal or greater than 1") require(logRetentionTimeMillis >= 1 || logRetentionTimeMillis == -1, "log.retention.ms must be unlimited (-1) or, equal or greater than 1") - require(logDirs.size > 0) require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.") require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" + @@ -920,127 +764,4 @@ class KafkaConfig (/** ********* Zookeeper Configuration ***********/ require(BrokerCompressionCodec.isValid(compressionType), "compression.type : " + compressionType + " is not valid." + " Valid options are " + BrokerCompressionCodec.brokerCompressionOptions.mkString(",")) } - - def toProps: Properties = { - val props = new Properties() - import kafka.server.KafkaConfig._ - /** ********* Zookeeper Configuration ***********/ - props.put(ZkConnectProp, zkConnect) - props.put(ZkSessionTimeoutMsProp, zkSessionTimeoutMs.toString) - _zkConnectionTimeoutMs.foreach(value => props.put(ZkConnectionTimeoutMsProp, value.toString)) - props.put(ZkSyncTimeMsProp, zkSyncTimeMs.toString) - - /** ********* General Configuration ***********/ - props.put(MaxReservedBrokerIdProp, maxReservedBrokerId.toString) - props.put(BrokerIdProp, brokerId.toString) - props.put(MessageMaxBytesProp, messageMaxBytes.toString) - props.put(NumNetworkThreadsProp, numNetworkThreads.toString) - props.put(NumIoThreadsProp, numIoThreads.toString) - props.put(BackgroundThreadsProp, backgroundThreads.toString) - props.put(QueuedMaxRequestsProp, queuedMaxRequests.toString) - - /** ********* Socket Server Configuration ***********/ - props.put(PortProp, port.toString) - props.put(HostNameProp, hostName) - _listeners.foreach(props.put(ListenersProp, _)) - _advertisedHostName.foreach(props.put(AdvertisedHostNameProp, _)) - _advertisedPort.foreach(value => props.put(AdvertisedPortProp, value.toString)) - _advertisedListeners.foreach(props.put(AdvertisedListenersProp, _)) - props.put(SocketSendBufferBytesProp, socketSendBufferBytes.toString) - props.put(SocketReceiveBufferBytesProp, socketReceiveBufferBytes.toString) - props.put(SocketRequestMaxBytesProp, socketRequestMaxBytes.toString) - props.put(MaxConnectionsPerIpProp, maxConnectionsPerIp.toString) - props.put(MaxConnectionsPerIpOverridesProp, _maxConnectionsPerIpOverrides) - props.put(ConnectionsMaxIdleMsProp, connectionsMaxIdleMs.toString) - - /** ********* Log Configuration ***********/ - props.put(NumPartitionsProp, numPartitions.toString) - props.put(LogDirProp, _logDir) - _logDirs.foreach(value => props.put(LogDirsProp, value)) - props.put(LogSegmentBytesProp, logSegmentBytes.toString) - - props.put(LogRollTimeHoursProp, logRollTimeHours.toString) - _logRollTimeMillis.foreach(v => props.put(LogRollTimeMillisProp, v.toString)) - - props.put(LogRollTimeJitterHoursProp, logRollTimeJitterHours.toString) - _logRollTimeJitterMillis.foreach(v => props.put(LogRollTimeJitterMillisProp, v.toString)) - - - props.put(LogRetentionTimeHoursProp, logRetentionTimeHours.toString) - _logRetentionTimeMins.foreach(v => props.put(LogRetentionTimeMinutesProp, v.toString)) - _logRetentionTimeMillis.foreach(v => props.put(LogRetentionTimeMillisProp, v.toString)) - - props.put(LogRetentionBytesProp, logRetentionBytes.toString) - props.put(LogCleanupIntervalMsProp, logCleanupIntervalMs.toString) - props.put(LogCleanupPolicyProp, logCleanupPolicy) - props.put(LogCleanerThreadsProp, logCleanerThreads.toString) - props.put(LogCleanerIoMaxBytesPerSecondProp, logCleanerIoMaxBytesPerSecond.toString) - props.put(LogCleanerDedupeBufferSizeProp, logCleanerDedupeBufferSize.toString) - props.put(LogCleanerIoBufferSizeProp, logCleanerIoBufferSize.toString) - props.put(LogCleanerDedupeBufferLoadFactorProp, logCleanerDedupeBufferLoadFactor.toString) - props.put(LogCleanerBackoffMsProp, logCleanerBackoffMs.toString) - props.put(LogCleanerMinCleanRatioProp, logCleanerMinCleanRatio.toString) - props.put(LogCleanerEnableProp, logCleanerEnable.toString) - props.put(LogCleanerDeleteRetentionMsProp, logCleanerDeleteRetentionMs.toString) - props.put(LogIndexSizeMaxBytesProp, logIndexSizeMaxBytes.toString) - props.put(LogIndexIntervalBytesProp, logIndexIntervalBytes.toString) - props.put(LogFlushIntervalMessagesProp, logFlushIntervalMessages.toString) - props.put(LogDeleteDelayMsProp, logDeleteDelayMs.toString) - props.put(LogFlushSchedulerIntervalMsProp, logFlushSchedulerIntervalMs.toString) - _logFlushIntervalMs.foreach(v => props.put(LogFlushIntervalMsProp, v.toString)) - props.put(LogFlushOffsetCheckpointIntervalMsProp, logFlushOffsetCheckpointIntervalMs.toString) - props.put(NumRecoveryThreadsPerDataDirProp, numRecoveryThreadsPerDataDir.toString) - props.put(AutoCreateTopicsEnableProp, autoCreateTopicsEnable.toString) - props.put(MinInSyncReplicasProp, minInSyncReplicas.toString) - - /** ********* Replication configuration ***********/ - props.put(ControllerSocketTimeoutMsProp, controllerSocketTimeoutMs.toString) - props.put(DefaultReplicationFactorProp, defaultReplicationFactor.toString) - props.put(ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) - props.put(ReplicaSocketTimeoutMsProp, replicaSocketTimeoutMs.toString) - props.put(ReplicaSocketReceiveBufferBytesProp, replicaSocketReceiveBufferBytes.toString) - props.put(ReplicaFetchMaxBytesProp, replicaFetchMaxBytes.toString) - props.put(ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) - props.put(ReplicaFetchMinBytesProp, replicaFetchMinBytes.toString) - props.put(ReplicaFetchBackoffMsProp, replicaFetchBackoffMs.toString) - props.put(NumReplicaFetchersProp, numReplicaFetchers.toString) - props.put(ReplicaHighWatermarkCheckpointIntervalMsProp, replicaHighWatermarkCheckpointIntervalMs.toString) - props.put(FetchPurgatoryPurgeIntervalRequestsProp, fetchPurgatoryPurgeIntervalRequests.toString) - props.put(ProducerPurgatoryPurgeIntervalRequestsProp, producerPurgatoryPurgeIntervalRequests.toString) - props.put(AutoLeaderRebalanceEnableProp, autoLeaderRebalanceEnable.toString) - props.put(LeaderImbalancePerBrokerPercentageProp, leaderImbalancePerBrokerPercentage.toString) - props.put(LeaderImbalanceCheckIntervalSecondsProp, leaderImbalanceCheckIntervalSeconds.toString) - props.put(UncleanLeaderElectionEnableProp, uncleanLeaderElectionEnable.toString) - props.put(InterBrokerSecurityProtocolProp, interBrokerSecurityProtocol.toString) - props.put(InterBrokerProtocolVersionProp, interBrokerProtocolVersion.toString) - - - /** ********* Controlled shutdown configuration ***********/ - props.put(ControlledShutdownMaxRetriesProp, controlledShutdownMaxRetries.toString) - props.put(ControlledShutdownRetryBackoffMsProp, controlledShutdownRetryBackoffMs.toString) - props.put(ControlledShutdownEnableProp, controlledShutdownEnable.toString) - - /** ********* Consumer coordinator configuration ***********/ - props.put(ConsumerMinSessionTimeoutMsProp, consumerMinSessionTimeoutMs.toString) - props.put(ConsumerMaxSessionTimeoutMsProp, consumerMaxSessionTimeoutMs.toString) - - /** ********* Offset management configuration ***********/ - props.put(OffsetMetadataMaxSizeProp, offsetMetadataMaxSize.toString) - props.put(OffsetsLoadBufferSizeProp, offsetsLoadBufferSize.toString) - props.put(OffsetsTopicReplicationFactorProp, offsetsTopicReplicationFactor.toString) - props.put(OffsetsTopicPartitionsProp, offsetsTopicPartitions.toString) - props.put(OffsetsTopicSegmentBytesProp, offsetsTopicSegmentBytes.toString) - props.put(OffsetsTopicCompressionCodecProp, offsetsTopicCompressionCodec.codec.toString) - props.put(OffsetsRetentionMinutesProp, offsetsRetentionMinutes.toString) - props.put(OffsetsRetentionCheckIntervalMsProp, offsetsRetentionCheckIntervalMs.toString) - props.put(OffsetCommitTimeoutMsProp, offsetCommitTimeoutMs.toString) - props.put(OffsetCommitRequiredAcksProp, offsetCommitRequiredAcks.toString) - props.put(DeleteTopicEnableProp, deleteTopicEnable.toString) - props.put(CompressionTypeProp, compressionType.toString) - props.put(MetricNumSamplesProp, metricNumSamples.toString) - props.put(MetricSampleWindowMsProp, metricSampleWindowMs.toString) - props.put(MetricReporterClassesProp, JavaConversions.collectionAsScalaIterable(_metricReporterClasses).mkString(",")) - - props - } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index b320ce9..9de2a6f 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -17,6 +17,9 @@ package kafka.server +import java.util +import java.util.Properties + import kafka.admin._ import kafka.log.LogConfig import kafka.log.CleanerConfig @@ -388,23 +391,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def boundPort(): Int = socketServer.boundPort() private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = { - val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, - segmentMs = config.logRollTimeMillis, - segmentJitterMs = config.logRollTimeJitterMillis, - flushInterval = config.logFlushIntervalMessages, - flushMs = config.logFlushIntervalMs.toLong, - retentionSize = config.logRetentionBytes, - retentionMs = config.logRetentionTimeMillis, - maxMessageSize = config.messageMaxBytes, - maxIndexSize = config.logIndexSizeMaxBytes, - indexInterval = config.logIndexIntervalBytes, - deleteRetentionMs = config.logCleanerDeleteRetentionMs, - fileDeleteDelayMs = config.logDeleteDelayMs, - minCleanableRatio = config.logCleanerMinCleanRatio, - compact = config.logCleanupPolicy.trim.toLowerCase == "compact", - minInSyncReplicas = config.minInSyncReplicas, - compressionType = config.compressionType) - val defaultProps = defaultLogConfig.toProps + val defaultProps = copyKafkaConfigToLog(config.originals) + val defaultLogConfig = LogConfig(defaultProps) + val configs = AdminUtils.fetchAllTopicConfigs(zkClient).mapValues(LogConfig.fromProps(defaultProps, _)) // read the log configurations from zookeeper val cleanerConfig = CleanerConfig(numThreads = config.logCleanerThreads, @@ -428,6 +417,38 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg time = time) } + // Copy the subset of properties that are relevant to Logs + // I'm listing out individual properties here since the names are slightly different in each Config class... + private def copyKafkaConfigToLog(serverProps: java.util.Map[String, Object]): java.util.Map[String, Object] = { + + val logProps = new util.HashMap[String, Object]() + val entryset = serverProps.entrySet.iterator + while (entryset.hasNext) { + val entry = entryset.next + entry.getKey match { + case KafkaConfig.LogSegmentBytesProp => logProps.put(LogConfig.SegmentBytesProp, entry.getValue) + case KafkaConfig.LogRollTimeMillisProp => logProps.put(LogConfig.SegmentMsProp, entry.getValue) + case KafkaConfig.LogRollTimeJitterMillisProp => logProps.put(LogConfig.SegmentJitterMsProp, entry.getValue) + case KafkaConfig.LogIndexSizeMaxBytesProp => logProps.put(LogConfig.SegmentIndexBytesProp, entry.getValue) + case KafkaConfig.LogFlushIntervalMessagesProp => logProps.put(LogConfig.FlushMessagesProp, entry.getValue) + case KafkaConfig.LogFlushIntervalMsProp => logProps.put(LogConfig.FlushMsProp, entry.getValue) + case KafkaConfig.LogRetentionBytesProp => logProps.put(LogConfig.RetentionBytesProp, entry.getValue) + case KafkaConfig.LogRetentionTimeMillisProp => logProps.put(LogConfig.RetentionMsProp, entry.getValue) + case KafkaConfig.MessageMaxBytesProp => logProps.put(LogConfig.MaxMessageBytesProp, entry.getValue) + case KafkaConfig.LogIndexIntervalBytesProp => logProps.put(LogConfig.IndexIntervalBytesProp, entry.getValue) + case KafkaConfig.LogCleanerDeleteRetentionMsProp => logProps.put(LogConfig.DeleteRetentionMsProp, entry.getValue) + case KafkaConfig.LogDeleteDelayMsProp => logProps.put(LogConfig.FileDeleteDelayMsProp, entry.getValue) + case KafkaConfig.LogCleanerMinCleanRatioProp => logProps.put(LogConfig.MinCleanableDirtyRatioProp, entry.getValue) + case KafkaConfig.LogCleanupPolicyProp => logProps.put(LogConfig.CleanupPolicyProp, entry.getValue) + case KafkaConfig.MinInSyncReplicasProp => logProps.put(LogConfig.MinInSyncReplicasProp, entry.getValue) + case KafkaConfig.CompressionTypeProp => logProps.put(LogConfig.CompressionTypeProp, entry.getValue) + case KafkaConfig.UncleanLeaderElectionEnableProp => logProps.put(LogConfig.UncleanLeaderElectionEnableProp, entry.getValue) + case _ => // we just leave those out + } + } + logProps + } + private def createOffsetManager(): OffsetManager = { val offsetManagerConfig = OffsetManagerConfig( maxMetadataSize = config.offsetMetadataMaxSize, diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 181cbc1..c89d00b 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -90,7 +90,7 @@ class ReplicaFetcherThread(name:String, // Prior to truncating the follower's log, ensure that doing so is not disallowed by the configuration for unclean leader election. // This situation could only happen if the unclean election configuration for a topic changes while a replica is down. Otherwise, // we should never encounter this situation since a non-ISR leader cannot be elected if disallowed by the broker configuration. - if (!LogConfig.fromProps(brokerConfig.toProps, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, + if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, topicAndPartition.topic)).uncleanLeaderElectionEnable) { // Log a fatal error and shutdown the broker to ensure that data loss does not unexpectedly occur. fatal("Halting because log truncation is not allowed for topic %s,".format(topicAndPartition.topic) + @@ -120,6 +120,6 @@ class ReplicaFetcherThread(name:String, // any logic for partitions whose leader has changed def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) { - delayPartitions(partitions, brokerConfig.replicaFetchBackoffMs) + delayPartitions(partitions, brokerConfig.replicaFetchBackoffMs.toLong) } } diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala index b675a7e..01b1b0a 100644 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ b/core/src/main/scala/kafka/server/TopicConfigManager.scala @@ -101,9 +101,10 @@ class TopicConfigManager(private val zkClient: ZkClient, val topic = json.substring(1, json.length - 1) // hacky way to dequote if (logsByTopic.contains(topic)) { /* combine the default properties with the overrides in zk to create the new LogConfig */ - val props = new Properties(logManager.defaultConfig.toProps) + val props = new Properties() + props.putAll(logManager.defaultConfig.originals) props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) - val logConfig = LogConfig.fromProps(props) + val logConfig = LogConfig(props) for (log <- logsByTopic(topic)) log.config = logConfig info("Processed topic config change %d for topic %s, setting new config to %s.".format(changeId, topic, props)) diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index d0a8fa7..f5d704c 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -254,32 +254,6 @@ object CoreUtils extends Logging { } /** - * 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 a big-endian integer from a byte array */ def readInt(bytes: Array[Byte], offset: Int): Int = { diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala index c0e248d..225d77b 100755 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -17,6 +17,7 @@ package kafka +import java.util.Properties import java.util.concurrent.atomic._ import kafka.common._ import kafka.message._ @@ -33,10 +34,13 @@ object StressTestLog { def main(args: Array[String]) { val dir = TestUtils.tempDir() val time = new MockTime + val logProprties = new Properties() + logProprties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer) + logProprties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer) + logProprties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer) + val log = new Log(dir = dir, - config = LogConfig(segmentSize = 64*1024*1024, - maxMessageSize = Int.MaxValue, - maxIndexSize = 1024*1024), + config = LogConfig(logProprties), recoveryPoint = 0L, scheduler = time.scheduler, time = time) diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index 3034c4f..236d857 100755 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -20,7 +20,7 @@ package kafka import java.io._ import java.nio._ import java.nio.channels._ -import java.util.Random +import java.util.{Properties, Random} import kafka.log._ import kafka.utils._ import kafka.message._ @@ -110,7 +110,10 @@ object TestLinearWriteSpeed { writables(i) = new ChannelWritable(new File(dir, "kafka-test-" + i + ".dat"), buffer) } else if(options.has(logOpt)) { val segmentSize = rand.nextInt(512)*1024*1024 + 64*1024*1024 // vary size to avoid herd effect - writables(i) = new LogWritable(new File(dir, "kafka-test-" + i), new LogConfig(segmentSize=segmentSize, flushInterval = flushInterval), scheduler, messageSet) + val logProperties = new Properties() + logProperties.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProperties.put(LogConfig.FlushMessagesProp, flushInterval: java.lang.Long) + writables(i) = new LogWritable(new File(dir, "kafka-test-" + i), new LogConfig(logProperties), scheduler, messageSet) } else { System.err.println("Must specify what to write to with one of --log, --channel, or --mmap") System.exit(1) diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala index 375555f..6180b87 100755 --- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala @@ -26,7 +26,7 @@ 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 java.util.{Properties, Collection, ArrayList} import kafka.server.KafkaConfig import org.apache.kafka.common.record.CompressionType import scala.collection.JavaConversions._ @@ -54,9 +54,10 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin @Test def testBrokerSideCompression() { val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression) - + val logProps = new Properties() + logProps.put(LogConfig.CompressionTypeProp,brokerCompression) /*configure broker-side compression */ - val log = new Log(logDir, logConfig.copy(compressionType = brokerCompression), recoveryPoint = 0L, time.scheduler, time = time) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) /* append two messages */ log.append(new ByteBufferMessageSet(messageCompressionCode, new Message("hello".getBytes), new Message("there".getBytes))) diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index 8b8249a..0e2a6a1 100755 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -17,6 +17,8 @@ package kafka.log +import java.util.Properties + import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Test} @@ -35,7 +37,11 @@ import org.apache.kafka.common.utils.Utils class CleanerTest extends JUnitSuite { val dir = TestUtils.tempDir() - val logConfig = LogConfig(segmentSize=1024, maxIndexSize=1024, compact=true) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + val logConfig = LogConfig(logProps) val time = new MockTime() val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time) @@ -50,8 +56,11 @@ class CleanerTest extends JUnitSuite { @Test def testCleanSegments() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 1024)) - + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + // append messages to the log until we have four segments while(log.numberOfSegments < 4) log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) @@ -72,7 +81,10 @@ class CleanerTest extends JUnitSuite { @Test def testCleaningWithDeletes() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 1024)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append messages with the keys 0 through N while(log.numberOfSegments < 2) @@ -98,7 +110,11 @@ class CleanerTest extends JUnitSuite { 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)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Delete) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append unkeyed messages while(log.numberOfSegments < 2) @@ -114,7 +130,9 @@ class CleanerTest extends JUnitSuite { val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages // turn on compaction and compact the log - val compactedLog = makeLog(config = logConfig.copy(segmentSize = 1024)) + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val compactedLog = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0)) assertEquals("Log should only contain keyed messages after cleaning.", 0, unkeyedMessageCountInLog(log)) @@ -139,7 +157,10 @@ class CleanerTest extends JUnitSuite { @Test def testCleanSegmentsWithAbort() { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) - val log = makeLog(config = logConfig.copy(segmentSize = 1024)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append messages to the log until we have four segments while(log.numberOfSegments < 4) @@ -159,7 +180,11 @@ class CleanerTest extends JUnitSuite { @Test def testSegmentGrouping() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 300, indexInterval = 1)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append some messages to the log var i = 0 @@ -208,7 +233,12 @@ class CleanerTest extends JUnitSuite { @Test def testSegmentGroupingWithSparseOffsets() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 1024, indexInterval = 1)) + + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // fill up first segment while (log.numberOfSegments == 1) @@ -288,7 +318,12 @@ class CleanerTest extends JUnitSuite { @Test def testRecoveryAfterCrash() { val cleaner = makeCleaner(Int.MaxValue) - val config = logConfig.copy(segmentSize = 300, indexInterval = 1, fileDeleteDelayMs = 10) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer) + + val config = LogConfig.fromProps(logConfig.originals, logProps) def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = { // Recover log file and check that after recovery, keys are as expected diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 471ddff..381e9aa 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -18,6 +18,7 @@ package kafka.log import java.io.File +import java.util.Properties import kafka.common.TopicAndPartition import kafka.message._ @@ -127,8 +128,13 @@ class LogCleanerIntegrationTest(compressionCodec: String) extends JUnit3Suite { for(i <- 0 until parts) { val dir = new File(logDir, "log-" + i) dir.mkdirs() + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 100*1024: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, deleteDelay: java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) val log = new Log(dir = dir, - LogConfig(segmentSize = segmentSize, maxIndexSize = 100*1024, fileDeleteDelayMs = deleteDelay, compact = true), + LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 3fd5a53..c31f884 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -26,22 +26,9 @@ import org.scalatest.junit.JUnit3Suite 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) - Assert.assertEquals("producer", config.compressionType) - } - - @Test def testFromPropsEmpty() { val p = new Properties() - val config = LogConfig.fromProps(p) + val config = LogConfig(p) Assert.assertEquals(LogConfig(), config) } @@ -62,7 +49,7 @@ class LogConfigTest extends JUnit3Suite { } }) - val actual = LogConfig.fromProps(expected).toProps + val actual = LogConfig(expected).originals Assert.assertEquals(expected, actual) } @@ -86,7 +73,7 @@ class LogConfigTest extends JUnit3Suite { val props = new Properties props.setProperty(name, value.toString) intercept[ConfigException] { - LogConfig.fromProps(props) + LogConfig(props) } }) } diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 01dfbc4..a13f2be 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -18,6 +18,7 @@ package kafka.log import java.io._ +import java.util.Properties import junit.framework.Assert._ import org.junit.Test import org.scalatest.junit.JUnit3Suite @@ -30,7 +31,11 @@ class LogManagerTest extends JUnit3Suite { val time: MockTime = new MockTime() val maxRollInterval = 100 val maxLogAgeMs = 10*60*60*1000 - val logConfig = LogConfig(segmentSize = 1024, maxIndexSize = 4096, retentionMs = maxLogAgeMs) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) + logProps.put(LogConfig.RetentionMsProp, maxLogAgeMs: java.lang.Integer) + val logConfig = LogConfig(logProps) var logDir: File = null var logManager: LogManager = null val name = "kafka" @@ -113,8 +118,11 @@ class LogManagerTest extends JUnit3Suite { def testCleanupSegmentsToMaintainSize() { val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes logManager.shutdown() + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer) + logProps.put(LogConfig.RetentionBytesProp, 5L * 10L * setSize + 10L: java.lang.Long) + val config = LogConfig.fromProps(logConfig.originals, logProps) - val config = logConfig.copy(segmentSize = 10 * setSize, retentionSize = 5L * 10L * setSize + 10L) logManager = createLogManager() logManager.startup @@ -154,7 +162,10 @@ class LogManagerTest extends JUnit3Suite { @Test def testTimeBasedFlush() { logManager.shutdown() - val config = logConfig.copy(flushMs = 1000) + val logProps = new Properties() + logProps.put(LogConfig.FlushMsProp, 1000: java.lang.Integer) + val config = LogConfig.fromProps(logConfig.originals, logProps) + logManager = createLogManager() logManager.startup val log = logManager.createLog(TopicAndPartition(name, 0), config) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 8e095d6..a8e57c2 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -18,6 +18,7 @@ package kafka.log import java.io._ +import java.util.Properties import java.util.concurrent.atomic._ import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite @@ -61,9 +62,12 @@ class LogTest extends JUnitSuite { def testTimeBasedLogRoll() { val set = TestUtils.singleMessageSet("test".getBytes()) + val logProps = new Properties() + logProps.put(LogConfig.SegmentMsProp, 1 * 60 * 60: java.lang.Long) + // create a log val log = new Log(logDir, - logConfig.copy(segmentMs = 1 * 60 * 60L), + LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) @@ -96,9 +100,12 @@ class LogTest extends JUnitSuite { val set = TestUtils.singleMessageSet("test".getBytes()) val maxJitter = 20 * 60L + val logProps = new Properties() + logProps.put(LogConfig.SegmentMsProp, 1 * 60 * 60: java.lang.Long) + logProps.put(LogConfig.SegmentJitterMsProp, maxJitter: java.lang.Long) // create a log val log = new Log(logDir, - logConfig.copy(segmentMs = 1 * 60 * 60L, segmentJitterMs = maxJitter), + LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) @@ -123,8 +130,10 @@ class LogTest extends JUnitSuite { val msgPerSeg = 10 val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) // create a log - val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), recoveryPoint = 0L, time.scheduler, time = time) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) // segments expire in size @@ -149,7 +158,9 @@ class LogTest extends JUnitSuite { */ @Test def testAppendAndReadWithSequentialOffsets() { - val log = new Log(logDir, logConfig.copy(segmentSize = 71), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray for(i <- 0 until messages.length) @@ -168,7 +179,9 @@ class LogTest extends JUnitSuite { */ @Test def testAppendAndReadWithNonSequentialOffsets() { - val log = new Log(logDir, logConfig.copy(segmentSize = 71), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val messages = messageIds.map(id => new Message(id.toString.getBytes)) @@ -191,7 +204,9 @@ class LogTest extends JUnitSuite { */ @Test def testReadAtLogGap() { - val log = new Log(logDir, logConfig.copy(segmentSize = 300), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), 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) @@ -211,7 +226,9 @@ class LogTest extends JUnitSuite { @Test def testReadOutOfRange() { createEmptyLogs(logDir, 1024) - val log = new Log(logDir, logConfig.copy(segmentSize = 1024), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) assertEquals("Reading just beyond end of log should produce 0 byte read.", 0, log.read(1024, 1000).messageSet.sizeInBytes) try { log.read(0, 1024) @@ -234,7 +251,9 @@ class LogTest extends JUnitSuite { @Test def testLogRolls() { /* create a multipart log with 100 messages */ - val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val numMessages = 100 val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes)) messageSets.foreach(log.append(_)) @@ -263,7 +282,9 @@ class LogTest extends JUnitSuite { @Test def testCompressedMessages() { /* this log should roll after every messageset */ - val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), 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 +307,9 @@ class LogTest extends JUnitSuite { for(messagesToAppend <- List(0, 1, 25)) { logDir.mkdirs() // first test a log segment starting at 0 - val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) for(i <- 0 until messagesToAppend) log.append(TestUtils.singleMessageSet(i.toString.getBytes)) @@ -318,7 +341,9 @@ class LogTest extends JUnitSuite { 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) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, configSegmentSize: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) try { log.append(messageSet) @@ -342,7 +367,10 @@ class LogTest extends JUnitSuite { 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) + val logProps = new Properties() + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time) try { log.append(messageSetWithUnkeyedMessage) @@ -380,7 +408,9 @@ class LogTest extends JUnitSuite { // append messages to log val maxMessageSize = second.sizeInBytes - 1 - val log = new Log(logDir, logConfig.copy(maxMessageSize = maxMessageSize), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.MaxMessageBytesProp, maxMessageSize: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) // should be able to append the small message log.append(first) @@ -401,7 +431,11 @@ class LogTest extends JUnitSuite { val messageSize = 100 val segmentSize = 7 * messageSize val indexInterval = 3 * messageSize - val config = logConfig.copy(segmentSize = segmentSize, indexInterval = indexInterval, maxIndexSize = 4096) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, indexInterval: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) + val config = LogConfig(logProps) var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) for(i <- 0 until numMessages) log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(messageSize))) @@ -432,7 +466,11 @@ class LogTest extends JUnitSuite { def testIndexRebuild() { // publish the messages and close the log val numMessages = 200 - val config = logConfig.copy(segmentSize = 200, indexInterval = 1) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 200: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val config = LogConfig(logProps) var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) for(i <- 0 until numMessages) log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10))) @@ -460,8 +498,11 @@ class LogTest extends JUnitSuite { val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + // create a log - val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), recoveryPoint = 0L, scheduler = time.scheduler, time = time) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) for (i<- 1 to msgPerSeg) @@ -513,7 +554,9 @@ class LogTest extends JUnitSuite { val setSize = set.sizeInBytes val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages - val config = logConfig.copy(segmentSize = segmentSize) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + val config = LogConfig(logProps) val log = new Log(logDir, config, recoveryPoint = 0L, scheduler = time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) for (i<- 1 to msgPerSeg) @@ -540,10 +583,12 @@ class LogTest extends JUnitSuite { val bogusIndex2 = Log.indexFilename(logDir, 5) val set = TestUtils.singleMessageSet("test".getBytes()) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) val log = new Log(logDir, - logConfig.copy(segmentSize = set.sizeInBytes * 5, - maxIndexSize = 1000, - indexInterval = 1), + LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time) @@ -564,9 +609,11 @@ class LogTest extends JUnitSuite { @Test def testReopenThenTruncate() { val set = TestUtils.singleMessageSet("test".getBytes()) - val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, - maxIndexSize = 1000, - indexInterval = 10000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 10000: java.lang.Integer) + val config = LogConfig(logProps) // create a log var log = new Log(logDir, @@ -596,10 +643,13 @@ 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, - indexInterval = 10000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 10000: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, asyncDeleteMs: java.lang.Integer) + val config = LogConfig(logProps) + val log = new Log(logDir, config, recoveryPoint = 0L, @@ -634,7 +684,10 @@ class LogTest extends JUnitSuite { @Test def testOpenDeletesObsoleteFiles() { val set = TestUtils.singleMessageSet("test".getBytes()) - val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, maxIndexSize = 1000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + val config = LogConfig(logProps) var log = new Log(logDir, config, recoveryPoint = 0L, @@ -672,7 +725,11 @@ class LogTest extends JUnitSuite { @Test def testCorruptLog() { // append some messages to create some segments - val config = logConfig.copy(indexInterval = 1, maxMessageSize = 64*1024, segmentSize = 1000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer) + val config = LogConfig(logProps) val set = TestUtils.singleMessageSet("test".getBytes()) val recoveryPoint = 50L for(iteration <- 0 until 50) { @@ -704,7 +761,11 @@ class LogTest extends JUnitSuite { @Test def testCleanShutdownFile() { // append some messages to create some segments - val config = logConfig.copy(indexInterval = 1, maxMessageSize = 64*1024, segmentSize = 1000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + val config = LogConfig(logProps) val set = TestUtils.singleMessageSet("test".getBytes()) val parentLogDir = logDir.getParentFile assertTrue("Data directory %s must exist", parentLogDir.isDirectory) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 7877f6c..8a871cf 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -16,6 +16,8 @@ */ package kafka.server +import java.util.Properties + import junit.framework.Assert._ import org.junit.Test import kafka.integration.KafkaServerTestHarness @@ -30,16 +32,19 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testConfigChange() { - val oldVal = 100000 - val newVal = 200000 + val oldVal: java.lang.Long = 100000 + val newVal: java.lang.Long = 200000 val tp = TopicAndPartition("test", 0) - AdminUtils.createTopic(zkClient, tp.topic, 1, 1, LogConfig(flushInterval = oldVal).toProps) + val logProps = new Properties() + logProps.put(LogConfig.FlushMessagesProp, oldVal.toString) + AdminUtils.createTopic(zkClient, tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.servers(0).logManager.getLog(tp) assertTrue(logOpt.isDefined) assertEquals(oldVal, logOpt.get.config.flushInterval) } - AdminUtils.changeTopicConfig(zkClient, tp.topic, LogConfig(flushInterval = newVal).toProps) + logProps.put(LogConfig.FlushMessagesProp, newVal.toString) + AdminUtils.changeTopicConfig(zkClient, tp.topic, logProps) TestUtils.retry(10000) { assertEquals(newVal, this.servers(0).logManager.getLog(tp).get.config.flushInterval) } @@ -49,7 +54,9 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { def testConfigChangeOnNonExistingTopic() { val topic = TestUtils.tempTopic try { - AdminUtils.changeTopicConfig(zkClient, topic, LogConfig(flushInterval = 10000).toProps) + val logProps = new Properties() + logProps.put(LogConfig.FlushMessagesProp, 10000: java.lang.Integer) + AdminUtils.changeTopicConfig(zkClient, topic, logProps) fail("Should fail with AdminOperationException for topic doesn't exist") } catch { case e: AdminOperationException => // expected diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index c487f36..8268852 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -30,29 +30,13 @@ import scala.util.Random._ class KafkaConfigConfigDefTest extends JUnit3Suite { @Test - def testFromPropsDefaults() { - val defaults = new Properties() - defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") - - // some ordinary setting - defaults.put(KafkaConfig.AdvertisedPortProp, "1818") - - val props = new Properties(defaults) - - val config = KafkaConfig.fromProps(props) - - Assert.assertEquals(1818, config.advertisedPort) - Assert.assertEquals("KafkaConfig defaults should be retained", Defaults.ConnectionsMaxIdleMs, config.connectionsMaxIdleMs) - } - - @Test def testFromPropsEmpty() { // only required val p = new Properties() p.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") val actualConfig = KafkaConfig.fromProps(p) - val expectedConfig = new KafkaConfig(zkConnect = "127.0.0.1:2181") + val expectedConfig = new KafkaConfig(p) Assert.assertEquals(expectedConfig.zkConnect, actualConfig.zkConnect) Assert.assertEquals(expectedConfig.zkSessionTimeoutMs, actualConfig.zkSessionTimeoutMs) @@ -252,7 +236,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { } }) - val actual = KafkaConfig.fromProps(expected).toProps + val actual = KafkaConfig.fromProps(expected).originals Assert.assertEquals(expected, actual) } -- 1.7.12.4 From d9c0ad6855b4871694ddf17c9d4546b021302ee8 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Thu, 18 Jun 2015 15:59:11 -0700 Subject: [PATCH 29/59] kafka-2265; creating a topic with large number of partitions takes a long time; patched by Manikumar Reddy; reviewed by Jun Rao --- .../main/scala/kafka/controller/PartitionStateMachine.scala | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index 92fd92d..b4e7c88 100755 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -188,7 +188,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { case NewPartition => // pre: partition did not exist before this assertValidPreviousStates(topicAndPartition, List(NonExistentPartition), NewPartition) - assignReplicasToPartitions(topic, partition) partitionState.put(topicAndPartition, NewPartition) val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition).mkString(",") stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from %s to %s with assigned replicas %s" @@ -266,17 +265,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } /** - * Invoked on the NonExistentPartition->NewPartition state transition to update the controller's cache with the - * partition's replica assignment. - * @param topic The topic of the partition whose replica assignment is to be cached - * @param partition The partition whose replica assignment is to be cached - */ - private def assignReplicasToPartitions(topic: String, partition: Int) { - val assignedReplicas = ZkUtils.getReplicasForPartition(controllerContext.zkClient, topic, partition) - controllerContext.partitionReplicaAssignment += TopicAndPartition(topic, partition) -> assignedReplicas - } - - /** * Invoked on the NewPartition->OnlinePartition state change. When a partition is in the New state, it does not have * a leader and isr path in zookeeper. Once the partition moves to the OnlinePartition state, it's leader and isr * path gets initialized and it never goes back to the NewPartition state. From here, it can only go to the @@ -526,6 +514,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { else { if (partitionsToBeAdded.size > 0) { info("New partitions to be added %s".format(partitionsToBeAdded)) + controllerContext.partitionReplicaAssignment.++=(partitionsToBeAdded) controller.onNewPartitionCreation(partitionsToBeAdded.keySet.toSet) } } -- 1.7.12.4 From 5c2ca30f229c7f39fca65aed6bd45c382aacda77 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Thu, 18 Jun 2015 16:37:25 -0700 Subject: [PATCH 30/59] kafka-2234; Partition reassignment of a nonexistent topic prevents future reassignments; patched by Manikumar Reddy; reviewed by Jun Rao --- .../main/scala/kafka/admin/ReassignPartitionsCommand.scala | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 912b718..ea34589 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -208,9 +208,14 @@ class ReassignPartitionsCommand(zkClient: ZkClient, partitions: collection.Map[T def reassignPartitions(): Boolean = { try { val validPartitions = partitions.filter(p => validatePartition(zkClient, p._1.topic, p._1.partition)) - val jsonReassignmentData = ZkUtils.getPartitionReassignmentZkData(validPartitions) - ZkUtils.createPersistentPath(zkClient, ZkUtils.ReassignPartitionsPath, jsonReassignmentData) - true + if(validPartitions.isEmpty) { + false + } + else { + val jsonReassignmentData = ZkUtils.getPartitionReassignmentZkData(validPartitions) + ZkUtils.createPersistentPath(zkClient, ZkUtils.ReassignPartitionsPath, jsonReassignmentData) + true + } } catch { case ze: ZkNodeExistsException => val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient) -- 1.7.12.4 From 1c93bb16a91788d3489d4d784c13f1f0fddb6fb7 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Thu, 18 Jun 2015 17:00:34 -0700 Subject: [PATCH 31/59] trivial change to fix unit test failure introduced in kafka-2234 --- core/src/test/scala/unit/kafka/admin/AdminTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index efb2f8e..252ac81 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -242,7 +242,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val partitionToBeReassigned = 0 val topicAndPartition = TopicAndPartition(topic, partitionToBeReassigned) val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) - assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) + assertFalse("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) val reassignedPartitions = ZkUtils.getPartitionsBeingReassigned(zkClient) assertFalse("Partition should not be reassigned", reassignedPartitions.contains(topicAndPartition)) servers.foreach(_.shutdown()) -- 1.7.12.4 From 19c98cb8ed69d3d6da787dc04e1e88add5f6b9d7 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Thu, 18 Jun 2015 18:50:52 -0700 Subject: [PATCH 32/59] kafka-1758; corrupt recovery file prevents startup; patched by Manikumar Reddy; reviewed by Neha Narkhede and Jun Rao --- core/src/main/scala/kafka/log/LogManager.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 538fc83..69386c1 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -126,7 +126,15 @@ class LogManager(val logDirs: Array[File], brokerState.newState(RecoveringFromUncleanShutdown) } - val recoveryPoints = this.recoveryPointCheckpoints(dir).read + var recoveryPoints = Map[TopicAndPartition, Long]() + try { + recoveryPoints = this.recoveryPointCheckpoints(dir).read + } catch { + case e: Exception => { + warn("Error occured while reading recovery-point-offset-checkpoint file of directory " + dir, e) + warn("Resetting the recovery checkpoint to 0") + } + } val jobsForDir = for { dirContent <- Option(dir.listFiles).toList -- 1.7.12.4 From ca758252c5a524fe6135a585282dd4bf747afef2 Mon Sep 17 00:00:00 2001 From: Honghai Chen Date: Fri, 19 Jun 2015 07:52:37 -0700 Subject: [PATCH 33/59] kafka-1646; Improve consumer read performance for Windows; patched by Honghai Chen; reviewed by Jay Kreps and Jun Rao --- core/src/main/scala/kafka/log/FileMessageSet.scala | 58 ++++++++++++++++++++-- core/src/main/scala/kafka/log/Log.scala | 30 +++++++++-- core/src/main/scala/kafka/log/LogCleaner.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 6 +++ core/src/main/scala/kafka/log/LogSegment.scala | 4 +- core/src/main/scala/kafka/server/KafkaConfig.scala | 5 ++ core/src/main/scala/kafka/server/KafkaServer.scala | 1 + core/src/main/scala/kafka/utils/CoreUtils.scala | 10 ---- .../scala/unit/kafka/log/FileMessageSetTest.scala | 55 +++++++++++++++++++- .../test/scala/unit/kafka/log/LogConfigTest.scala | 1 + .../test/scala/unit/kafka/log/LogSegmentTest.scala | 54 +++++++++++++++++++- .../kafka/server/KafkaConfigConfigDefTest.scala | 1 + 12 files changed, 204 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index 2522604..39361fe 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -54,7 +54,7 @@ class FileMessageSet private[kafka](@volatile var file: File, /* if this is not a slice, update the file pointer to the end of the file */ if (!isSlice) /* set the file position to the last byte in the file */ - channel.position(channel.size) + channel.position(math.min(channel.size().toInt, end)) /** * Create a file message set with no slicing. @@ -66,12 +66,25 @@ class FileMessageSet private[kafka](@volatile var file: File, * Create a file message set with no slicing */ def this(file: File) = - this(file, CoreUtils.openChannel(file, mutable = true)) + this(file, FileMessageSet.openChannel(file, mutable = true)) + + /** + * Create a file message set with no slicing, and with initFileSize and preallocate. + * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize + * with one value (for example 512 * 1024 *1024 ) can improve the kafka produce performance. + * If it's new file and preallocate is true, end will be set to 0. Otherwise set to Int.MaxValue. + */ + def this(file: File, fileAlreadyExists: Boolean, initFileSize: Int, preallocate: Boolean) = + this(file, + channel = FileMessageSet.openChannel(file, mutable = true, fileAlreadyExists, initFileSize, preallocate), + start = 0, + end = ( if ( !fileAlreadyExists && preallocate ) 0 else Int.MaxValue), + isSlice = false) /** * Create a file message set with mutable option */ - def this(file: File, mutable: Boolean) = this(file, CoreUtils.openChannel(file, mutable)) + def this(file: File, mutable: Boolean) = this(file, FileMessageSet.openChannel(file, mutable)) /** * Create a slice view of the file message set that begins and ends at the given byte offsets @@ -223,10 +236,18 @@ class FileMessageSet private[kafka](@volatile var file: File, */ def close() { flush() + trim() channel.close() } /** + * Trim file when close or roll to next file + */ + def trim() { + truncateTo(sizeInBytes()) + } + + /** * Delete this message set from the filesystem * @return True iff this message set was deleted. */ @@ -272,6 +293,37 @@ class FileMessageSet private[kafka](@volatile var file: File, } } + +object FileMessageSet +{ + /** + * Open a channel for the given file + * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize + * with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance. + * @param file File path + * @param mutable mutable + * @param fileAlreadyExists File already exists or not + * @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024 + * @param preallocate Pre allocate file or not, gotten from configuration. + */ + def openChannel(file: File, mutable: Boolean, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): FileChannel = { + if (mutable) { + if (fileAlreadyExists) + new RandomAccessFile(file, "rw").getChannel() + else { + if (preallocate) { + val randomAccessFile = new RandomAccessFile(file, "rw") + randomAccessFile.setLength(initFileSize) + randomAccessFile.getChannel() + } + else + new RandomAccessFile(file, "rw").getChannel() + } + } + else + new FileInputStream(file).getChannel() + } +} object LogFlushStats extends KafkaMetricsGroup { val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 84e7b8f..6b9274d 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -78,6 +78,13 @@ class Log(val dir: File, /* last time it was flushed */ private val lastflushedTime = new AtomicLong(time.milliseconds) + def initFileSize() : Int = { + if (config.preallocate) + config.segmentSize + else + 0 + } + /* the actual segments of the log */ private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment] loadSegments() @@ -168,7 +175,8 @@ class Log(val dir: File, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, rollJitterMs = config.randomSegmentJitter, - time = time) + time = time, + fileAlreadyExists = true) if(!hasIndex) { error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath)) segment.recover(config.maxMessageSize) @@ -205,7 +213,10 @@ class Log(val dir: File, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, rollJitterMs = config.randomSegmentJitter, - time = time)) + time = time, + fileAlreadyExists = false, + initFileSize = this.initFileSize(), + preallocate = config.preallocate)) } else { recoverLog() // reset the index size of the currently active log segment to allow more entries @@ -586,14 +597,20 @@ class Log(val dir: File, segments.lastEntry() match { case null => - case entry => entry.getValue.index.trimToValidSize() + case entry => { + entry.getValue.index.trimToValidSize() + entry.getValue.log.trim() + } } val segment = new LogSegment(dir, startOffset = newOffset, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, rollJitterMs = config.randomSegmentJitter, - time = time) + time = time, + fileAlreadyExists = false, + initFileSize = initFileSize, + preallocate = config.preallocate) val prev = addSegment(segment) if(prev != null) throw new KafkaException("Trying to roll a new log segment for topic partition %s with start offset %d while it already exists.".format(name, newOffset)) @@ -687,7 +704,10 @@ class Log(val dir: File, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, rollJitterMs = config.randomSegmentJitter, - time = time)) + time = time, + fileAlreadyExists = false, + initFileSize = initFileSize, + preallocate = config.preallocate)) 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 c9ade72..d07a391 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -344,7 +344,7 @@ private[log] class Cleaner(val id: Int, logFile.delete() val indexFile = new File(segments.head.index.file.getPath + Log.CleanedFileSuffix) indexFile.delete() - val messages = new FileMessageSet(logFile) + val messages = new FileMessageSet(logFile, fileAlreadyExists = false, initFileSize = log.initFileSize(), preallocate = log.config.preallocate) val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize) val cleaned = new LogSegment(messages, index, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index e9af221..fc41132 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -43,6 +43,7 @@ object Defaults { val UncleanLeaderElectionEnable = kafka.server.Defaults.UncleanLeaderElectionEnable val MinInSyncReplicas = kafka.server.Defaults.MinInSyncReplicas val CompressionType = kafka.server.Defaults.CompressionType + val PreAllocateEnable = kafka.server.Defaults.LogPreAllocateEnable } case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props) { @@ -64,6 +65,7 @@ case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfi val uncleanLeaderElectionEnable = getBoolean(LogConfig.UncleanLeaderElectionEnableProp) val minInSyncReplicas = getInt(LogConfig.MinInSyncReplicasProp) val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase + val preallocate = getBoolean(LogConfig.PreAllocateEnableProp) def randomSegmentJitter: Long = if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs) @@ -95,6 +97,7 @@ object LogConfig { val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" val MinInSyncReplicasProp = "min.insync.replicas" val CompressionTypeProp = "compression.type" + val PreAllocateEnableProp = "preallocate" 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" @@ -118,6 +121,7 @@ object LogConfig { 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." + val PreAllocateEnableDoc ="Should pre allocate file when create new segment?" private val configDef = { import ConfigDef.Range._ @@ -149,6 +153,8 @@ object LogConfig { MEDIUM, UncleanLeaderElectionEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc) .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc) + .define(PreAllocateEnableProp, BOOLEAN, Defaults.PreAllocateEnable, + MEDIUM, PreAllocateEnableDoc) } def apply(): LogConfig = LogConfig(new Properties()) diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index ed03953..1377e8f 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -52,8 +52,8 @@ class LogSegment(val log: FileMessageSet, /* 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, rollJitterMs: Long, time: Time) = - this(new FileMessageSet(file = Log.logFilename(dir, startOffset)), + def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) = + this(new FileMessageSet(file = Log.logFilename(dir, startOffset), fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate), new OffsetIndex(file = Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), startOffset, indexIntervalBytes, diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index e0b2480..c1f0cca 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -83,6 +83,7 @@ object Defaults { val LogDeleteDelayMs = 60000 val LogFlushSchedulerIntervalMs = Long.MaxValue val LogFlushOffsetCheckpointIntervalMs = 60000 + val LogPreAllocateEnable = false val NumRecoveryThreadsPerDataDir = 1 val AutoCreateTopicsEnable = true val MinInSyncReplicas = 1 @@ -206,6 +207,7 @@ object KafkaConfig { val LogFlushSchedulerIntervalMsProp = "log.flush.scheduler.interval.ms" val LogFlushIntervalMsProp = "log.flush.interval.ms" val LogFlushOffsetCheckpointIntervalMsProp = "log.flush.offset.checkpoint.interval.ms" + val LogPreAllocateProp = "log.preallocate" val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir" val AutoCreateTopicsEnableProp = "auto.create.topics.enable" val MinInSyncReplicasProp = "min.insync.replicas" @@ -332,6 +334,7 @@ object KafkaConfig { val LogFlushSchedulerIntervalMsDoc = "The frequency in ms that the log flusher checks whether any log needs to be flushed to disk" val LogFlushIntervalMsDoc = "The maximum time in ms that a message in any topic is kept in memory before flushed to disk" val LogFlushOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point" + val LogPreAllocateEnableDoc = "Should pre allocate file when create new segment? If you are using Kafka on Windows, you probably need to set it to true." val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown" val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server" val MinInSyncReplicasDoc = "define the minimum number of replicas in ISR needed to satisfy a produce request with required.acks=-1 (or all)" @@ -466,6 +469,7 @@ object KafkaConfig { .define(LogFlushSchedulerIntervalMsProp, LONG, Defaults.LogFlushSchedulerIntervalMs, HIGH, LogFlushSchedulerIntervalMsDoc) .define(LogFlushIntervalMsProp, LONG, HIGH, LogFlushIntervalMsDoc, false) .define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LogFlushOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc) + .define(LogPreAllocateProp, BOOLEAN, Defaults.LogPreAllocateEnable, MEDIUM, LogPreAllocateEnableDoc) .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc) @@ -609,6 +613,7 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka val logRollTimeJitterMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeJitterMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeJitterHoursProp)) val logFlushIntervalMs: java.lang.Long = Option(getLong(KafkaConfig.LogFlushIntervalMsProp)).getOrElse(getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp)) val minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp) + val logPreAllocateEnable: Boolean = getBoolean(KafkaConfig.LogPreAllocateProp) /** ********* Replication configuration ***********/ val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 9de2a6f..52dc728 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -443,6 +443,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg case KafkaConfig.MinInSyncReplicasProp => logProps.put(LogConfig.MinInSyncReplicasProp, entry.getValue) case KafkaConfig.CompressionTypeProp => logProps.put(LogConfig.CompressionTypeProp, entry.getValue) case KafkaConfig.UncleanLeaderElectionEnableProp => logProps.put(LogConfig.UncleanLeaderElectionEnableProp, entry.getValue) + case KafkaConfig.LogPreAllocateProp => logProps.put(LogConfig.PreAllocateEnableProp, entry.getValue) case _ => // we just leave those out } } diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index f5d704c..168a18d 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -70,16 +70,6 @@ object CoreUtils extends Logging { Utils.daemonThread(name, runnable(fun)) /** - * Open a channel for the given file - */ - def openChannel(file: File, mutable: Boolean): FileChannel = { - if(mutable) - new RandomAccessFile(file, "rw").getChannel() - 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 * @param action The action to execute diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index cec1cae..02cf668 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -17,6 +17,7 @@ package kafka.log +import java.io._ import java.nio._ import java.util.concurrent.atomic._ import junit.framework.Assert._ @@ -146,5 +147,57 @@ class FileMessageSetTest extends BaseMessageSetTestCases { assertEquals(List(message), messageSet.toList) assertEquals(MessageSet.entrySize(message.message), messageSet.sizeInBytes) } - + + /** + * Test the new FileMessageSet with pre allocate as true + */ + @Test + def testPreallocateTrue() { + val temp = tempFile() + val set = new FileMessageSet(temp, false, 512 *1024 *1024, true) + val position = set.channel.position + val size = set.sizeInBytes() + assertEquals(0, position) + assertEquals(0, size) + assertEquals(512 *1024 *1024, temp.length) + } + + /** + * Test the new FileMessageSet with pre allocate as false + */ + @Test + def testPreallocateFalse() { + val temp = tempFile() + val set = new FileMessageSet(temp, false, 512 *1024 *1024, false) + val position = set.channel.position + val size = set.sizeInBytes() + assertEquals(0, position) + assertEquals(0, size) + assertEquals(0, temp.length) + } + + /** + * Test the new FileMessageSet with pre allocate as true and file has been clearly shut down, the file will be truncate to end of valid data. + */ + @Test + def testPreallocateClearShutdown() { + val temp = tempFile() + val set = new FileMessageSet(temp, false, 512 *1024 *1024, true) + set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*)) + val oldposition = set.channel.position + val oldsize = set.sizeInBytes() + assertEquals(messageSet.sizeInBytes, oldposition) + assertEquals(messageSet.sizeInBytes, oldsize) + set.close() + + val tempReopen = new File(temp.getAbsolutePath()) + val setReopen = new FileMessageSet(tempReopen, true, 512 *1024 *1024, true) + val position = setReopen.channel.position + val size = setReopen.sizeInBytes() + + assertEquals(oldposition, position) + assertEquals(oldposition, size) + assertEquals(oldposition, tempReopen.length) + } + } diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index c31f884..19dcb47 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -45,6 +45,7 @@ class LogConfigTest extends JUnit3Suite { case LogConfig.MinInSyncReplicasProp => expected.setProperty(name, (nextInt(Int.MaxValue - 1) + 1).toString) case LogConfig.RetentionBytesProp => expected.setProperty(name, nextInt().toString) case LogConfig.RetentionMsProp => expected.setProperty(name, nextLong().toString) + case LogConfig.PreAllocateEnableProp => expected.setProperty(name, randFrom("true", "false")) case positiveIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString) } }) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 03fb351..abcd1f0 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -226,5 +226,57 @@ class LogSegmentTest extends JUnit3Suite { seg.delete() } } - + + /* create a segment with pre allocate */ + def createSegment(offset: Long, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): LogSegment = { + val tempDir = TestUtils.tempDir() + val seg = new LogSegment(tempDir, offset, 10, 1000, 0, SystemTime, fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate) + segments += seg + seg + } + + /* create a segment with pre allocate, put message to it and verify */ + @Test + def testCreateWithInitFileSizeAppendMessage() { + val seg = createSegment(40, false, 512*1024*1024, true) + val ms = messages(50, "hello", "there") + seg.append(50, ms) + val ms2 = messages(60, "alpha", "beta") + seg.append(60, ms2) + val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, read.messageSet.toList) + } + + /* create a segment with pre allocate and clearly shut down*/ + @Test + def testCreateWithInitFileSizeClearShutdown() { + val tempDir = TestUtils.tempDir() + val seg = new LogSegment(tempDir, 40, 10, 1000, 0, SystemTime, false, 512*1024*1024, true) + + val ms = messages(50, "hello", "there") + seg.append(50, ms) + val ms2 = messages(60, "alpha", "beta") + seg.append(60, ms2) + val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, read.messageSet.toList) + val oldSize = seg.log.sizeInBytes() + val oldPosition = seg.log.channel.position + val oldFileSize = seg.log.file.length + assertEquals(512*1024*1024, oldFileSize) + seg.close() + //After close, file should be trimed + assertEquals(oldSize, seg.log.file.length) + + val segReopen = new LogSegment(tempDir, 40, 10, 1000, 0, SystemTime, true, 512*1024*1024, true) + segments += segReopen + + val readAgain = segReopen.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, readAgain.messageSet.toList) + val size = segReopen.log.sizeInBytes() + val position = segReopen.log.channel.position + val fileSize = segReopen.log.file.length + assertEquals(oldPosition, position) + assertEquals(oldSize, size) + assertEquals(size, fileSize) + } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 8268852..98a5b04 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -192,6 +192,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.MinInSyncReplicasProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.AutoLeaderRebalanceEnableProp => expected.setProperty(name, randFrom("true", "false")) case KafkaConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.LogPreAllocateProp => expected.setProperty(name, randFrom("true", "false")) case KafkaConfig.InterBrokerSecurityProtocolProp => expected.setProperty(name, SecurityProtocol.PLAINTEXT.toString) case KafkaConfig.InterBrokerProtocolVersionProp => expected.setProperty(name, ApiVersion.latestVersion.toString) -- 1.7.12.4 From 16ecf9806b286d9510103a4426bf0901d7dc8778 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Fri, 19 Jun 2015 09:34:22 -0700 Subject: [PATCH 34/59] kafka-2012; Broker should automatically handle corrupt index files; patched by Manikumar Reddy; reviewed by Jun Rao --- core/src/main/scala/kafka/log/Log.scala | 18 +++++++++---- core/src/test/scala/unit/kafka/log/LogTest.scala | 33 ++++++++++++++++++++++++ 2 files changed, 46 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 6b9274d..e5e8007 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -169,7 +169,7 @@ class Log(val dir: File, } else if(filename.endsWith(LogFileSuffix)) { // if its a log file, load the corresponding log segment val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong - val hasIndex = Log.indexFilename(dir, start).exists + val indexFile = Log.indexFilename(dir, start) val segment = new LogSegment(dir = dir, startOffset = start, indexIntervalBytes = config.indexInterval, @@ -177,7 +177,18 @@ class Log(val dir: File, rollJitterMs = config.randomSegmentJitter, time = time, fileAlreadyExists = true) - if(!hasIndex) { + + if(indexFile.exists()) { + try { + segment.index.sanityCheck() + } catch { + case e: java.lang.IllegalArgumentException => + warn("Found an corrupted index file, %s, deleting and rebuilding index...".format(indexFile.getAbsolutePath)) + indexFile.delete() + segment.recover(config.maxMessageSize) + } + } + else { error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath)) segment.recover(config.maxMessageSize) } @@ -223,9 +234,6 @@ class Log(val dir: File, activeSegment.index.resize(config.maxIndexSize) } - // sanity check the index file of every segment to ensure we don't proceed with a corrupt segment - for (s <- logSegments) - s.index.sanityCheck() } private def updateLogEndOffset(messageOffset: Long) { diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index a8e57c2..9e26190 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -489,6 +489,39 @@ class LogTest extends JUnitSuite { } /** + * Test that if we have corrupted an index segment it is rebuilt when the log is re-opened + */ + @Test + def testCorruptIndexRebuild() { + // publish the messages and close the log + val numMessages = 200 + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 200: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val config = LogConfig(logProps) + var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) + for(i <- 0 until numMessages) + log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10))) + val indexFiles = log.logSegments.map(_.index.file) + log.close() + + // corrupt all the index files + for( file <- indexFiles) { + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(" ") + bw.close() + } + + // reopen the log + log = new Log(logDir, config, recoveryPoint = 200L, 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) + log.close() + } + + /** * Test the Log truncate operations */ @Test -- 1.7.12.4 From cf28f893963c363bca43747e2f37fad3bb67d033 Mon Sep 17 00:00:00 2001 From: Chris Black Date: Mon, 22 Jun 2015 08:59:05 -0700 Subject: [PATCH 35/59] kafka-2290; OffsetIndex should open RandomAccessFile consistently; patched by Chris Black; reviewed by Jun Rao --- core/src/main/scala/kafka/log/OffsetIndex.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index a1082ae..332d5e2 100755 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -274,7 +274,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ def resize(newSize: Int) { inLock(lock) { - val raf = new RandomAccessFile(file, "rws") + val raf = new RandomAccessFile(file, "rw") val roundedNewSize = roundToExactMultiple(newSize, 8) val position = this.mmap.position -- 1.7.12.4 From dc54055d05742a4a7729a1fe1073c18e3d95cbb2 Mon Sep 17 00:00:00 2001 From: Ivan Simoneko Date: Mon, 22 Jun 2015 09:19:45 -0700 Subject: [PATCH 36/59] kafka-2235; LogCleaner offset map overflow; patched by Ivan Simoneko; reviewed by Jun Rao --- core/src/main/scala/kafka/log/LogCleaner.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index d07a391..b36ea0d 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -559,11 +559,17 @@ private[log] class Cleaner(val id: Int, // but we may be able to fit more (if there is lots of duplication in the dirty section of the log) var offset = dirty.head.baseOffset require(offset == start, "Last clean offset is %d but segment base offset is %d for log %s.".format(start, offset, log.name)) - val minStopOffset = (start + map.slots * this.dupBufferLoadFactor).toLong - for (segment <- dirty) { + val maxDesiredMapSize = (map.slots * this.dupBufferLoadFactor).toInt + var full = false + for (segment <- dirty if !full) { checkDone(log.topicAndPartition) - if(segment.baseOffset <= minStopOffset || map.utilization < this.dupBufferLoadFactor) + val segmentSize = segment.nextOffset() - segment.baseOffset + + require(segmentSize <= maxDesiredMapSize, "%d messages in segment %s/%s but offset map can fit only %d. You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads".format(segmentSize, log.name, segment.log.file.getName, maxDesiredMapSize)) + if (map.size + segmentSize <= maxDesiredMapSize) offset = buildOffsetMapForSegment(log.topicAndPartition, segment, map) + else + full = true } info("Offset map for log %s complete.".format(log.name)) offset -- 1.7.12.4 From 1eac3ceaf94b3e7583c7b6de2cfe13539ab06dd6 Mon Sep 17 00:00:00 2001 From: Onur Karaman Date: Mon, 22 Jun 2015 10:14:14 -0700 Subject: [PATCH 37/59] KAFKA-2245; Add response tests for consumer coordinator; reviewed by Joel Koshy --- .../kafka/coordinator/ConsumerCoordinator.scala | 4 +- .../kafka/coordinator/CoordinatorMetadata.scala | 4 +- .../ConsumerCoordinatorResponseTest.scala | 293 +++++++++++++++++++++ 3 files changed, 297 insertions(+), 4 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index 51e89c8..a385adb 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -46,8 +46,8 @@ class ConsumerCoordinator(val config: KafkaConfig, private var coordinatorMetadata: CoordinatorMetadata = null /** - * NOTE: If a group lock and coordinatorLock are simultaneously needed, - * be sure to acquire the group lock before coordinatorLock to prevent deadlock + * NOTE: If a group lock and metadataLock are simultaneously needed, + * be sure to acquire the group lock before metadataLock to prevent deadlock */ /** diff --git a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala index c39e6de..0cd5605 100644 --- a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala @@ -37,8 +37,8 @@ private[coordinator] class CoordinatorMetadata(config: KafkaConfig, maybePrepareRebalance: ConsumerGroupMetadata => Unit) { /** - * NOTE: If a group lock and coordinatorLock are simultaneously needed, - * be sure to acquire the group lock before coordinatorLock to prevent deadlock + * NOTE: If a group lock and metadataLock are simultaneously needed, + * be sure to acquire the group lock before metadataLock to prevent deadlock */ private val metadataLock = new ReentrantReadWriteLock() diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala new file mode 100644 index 0000000..a44fbd6 --- /dev/null +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala @@ -0,0 +1,293 @@ +/** + * 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.TimeUnit + +import junit.framework.Assert._ +import kafka.common.TopicAndPartition +import kafka.server.{KafkaConfig, OffsetManager} +import kafka.utils.TestUtils +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.JoinGroupRequest +import org.easymock.EasyMock +import org.junit.{After, Before, Test} +import org.scalatest.junit.JUnitSuite + +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, Future, Promise} + +/** + * Test ConsumerCoordinator responses + */ +class ConsumerCoordinatorResponseTest extends JUnitSuite { + type JoinGroupCallbackParams = (Set[TopicAndPartition], String, Int, Short) + type JoinGroupCallback = (Set[TopicAndPartition], String, Int, Short) => Unit + type HeartbeatCallbackParams = Short + type HeartbeatCallback = Short => Unit + + val ConsumerMinSessionTimeout = 10 + val ConsumerMaxSessionTimeout = 30 + val DefaultSessionTimeout = 20 + var offsetManager: OffsetManager = null + var consumerCoordinator: ConsumerCoordinator = null + + @Before + def setUp() { + val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") + props.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) + props.setProperty(KafkaConfig.ConsumerMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) + offsetManager = EasyMock.createStrictMock(classOf[OffsetManager]) + consumerCoordinator = new ConsumerCoordinator(KafkaConfig.fromProps(props), null, offsetManager) + consumerCoordinator.startup() + } + + @After + def tearDown() { + consumerCoordinator.shutdown() + } + + @Test + def testJoinGroupWrongCoordinator() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = false) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupUnknownPartitionAssignmentStrategy() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "foo" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupSessionTimeoutTooSmall() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, ConsumerMinSessionTimeout - 1, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupSessionTimeoutTooLarge() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, ConsumerMaxSessionTimeout + 1, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupUnknownConsumerNewGroup() { + val groupId = "groupId" + val consumerId = "consumerId" + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, joinGroupErrorCode) + } + + @Test + def testValidJoinGroup() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupInconsistentPartitionAssignmentStrategy() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + val otherPartitionAssignmentStrategy = "roundrobin" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, otherPartitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val otherJoinGroupErrorCode = otherJoinGroupResult._4 + assertEquals(Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code, otherJoinGroupErrorCode) + } + + @Test + def testJoinGroupUnknownConsumerExistingGroup() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = "consumerId" + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val otherJoinGroupErrorCode = otherJoinGroupResult._4 + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, otherJoinGroupErrorCode) + } + + @Test + def testHeartbeatWrongCoordinator() { + val groupId = "groupId" + val consumerId = "consumerId" + + val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = false) + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.code, heartbeatResult) + } + + @Test + def testHeartbeatUnknownGroup() { + val groupId = "groupId" + val consumerId = "consumerId" + + val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = true) + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, heartbeatResult) + } + + @Test + def testHeartbeatUnknownConsumerExistingGroup() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = "consumerId" + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, otherConsumerId, 1, isCoordinatorForGroup = true) + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, heartbeatResult) + } + + @Test + def testHeartbeatIllegalGeneration() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val assignedConsumerId = joinGroupResult._2 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, 2, isCoordinatorForGroup = true) + assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult) + } + + @Test + def testValidHeartbeat() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val assignedConsumerId = joinGroupResult._2 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1, isCoordinatorForGroup = true) + assertEquals(Errors.NONE.code, heartbeatResult) + } + + @Test + def testGenerationIdIncrementsOnRebalance() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val initialGenerationId = joinGroupResult._3 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(1, initialGenerationId) + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val nextGenerationId = otherJoinGroupResult._3 + val otherJoinGroupErrorCode = otherJoinGroupResult._4 + assertEquals(2, nextGenerationId) + assertEquals(Errors.NONE.code, otherJoinGroupErrorCode) + } + + private def setupJoinGroupCallback: (Future[JoinGroupCallbackParams], JoinGroupCallback) = { + val responsePromise = Promise[JoinGroupCallbackParams] + val responseFuture = responsePromise.future + val responseCallback: JoinGroupCallback = (partitions, consumerId, generationId, errorCode) => + responsePromise.success((partitions, consumerId, generationId, errorCode)) + (responseFuture, responseCallback) + } + + private def setupHeartbeatCallback: (Future[HeartbeatCallbackParams], HeartbeatCallback) = { + val responsePromise = Promise[HeartbeatCallbackParams] + val responseFuture = responsePromise.future + val responseCallback: HeartbeatCallback = errorCode => responsePromise.success(errorCode) + (responseFuture, responseCallback) + } + + private def joinGroup(groupId: String, + consumerId: String, + partitionAssignmentStrategy: String, + sessionTimeout: Int, + isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = { + val (responseFuture, responseCallback) = setupJoinGroupCallback + EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(offsetManager) + consumerCoordinator.handleJoinGroup(groupId, consumerId, Set.empty, sessionTimeout, partitionAssignmentStrategy, responseCallback) + Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) + } + + private def heartbeat(groupId: String, + consumerId: String, + generationId: Int, + isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = { + val (responseFuture, responseCallback) = setupHeartbeatCallback + EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(offsetManager) + consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback) + Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) + } +} -- 1.7.12.4 From 2270a7537f7734ec02b2391fae280624520a664d Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 22 Jun 2015 15:28:35 -0700 Subject: [PATCH 38/59] KAFKA-2293; Fix incorrect format specification in Partition.scala; reviewed by Joel Koshy --- core/src/main/scala/kafka/cluster/Partition.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 6cb6477..0990938 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -249,13 +249,12 @@ class Partition(val topic: String, 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]") + " is not recognized to be one of the assigned replicas %s for partition %s.") .format(localBrokerId, replicaId, - logReadResult.info.fetchOffsetMetadata, + logReadResult.info.fetchOffsetMetadata.messageOffset, assignedReplicas().map(_.brokerId).mkString(","), - topic, - partitionId)) + TopicAndPartition(topic, partitionId))) } } -- 1.7.12.4 From b6d326b0893e60b350608260fd1bd2542337cb5a Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 23 Jun 2015 00:07:19 -0400 Subject: [PATCH 39/59] kafka-2168; New consumer poll() can block other calls like position(), commit(), and close() indefinitely; patched by Jason Gustafson; reviewed by Jay Kreps, Ewen Cheslack-Postava, Guozhang Wang and Jun Rao --- .../apache/kafka/clients/consumer/Consumer.java | 5 + .../kafka/clients/consumer/ConsumerRecords.java | 7 + .../clients/consumer/ConsumerWakeupException.java | 20 + .../kafka/clients/consumer/KafkaConsumer.java | 715 ++++++++++++++++----- .../kafka/clients/consumer/MockConsumer.java | 9 +- .../clients/consumer/OffsetResetStrategy.java | 17 + .../clients/consumer/internals/Coordinator.java | 447 ++++++------- .../kafka/clients/consumer/internals/Fetcher.java | 159 ++--- .../clients/consumer/internals/Heartbeat.java | 10 + .../clients/consumer/internals/RequestFuture.java | 209 ++++++ .../consumer/internals/SubscriptionState.java | 41 +- .../java/org/apache/kafka/common/utils/Utils.java | 15 + .../kafka/clients/consumer/MockConsumerTest.java | 2 +- .../consumer/internals/CoordinatorTest.java | 148 ++++- .../clients/consumer/internals/FetcherTest.java | 32 +- .../clients/consumer/internals/HeartbeatTest.java | 9 + .../consumer/internals/SubscriptionStateTest.java | 19 +- .../org/apache/kafka/common/utils/UtilsTest.java | 8 + 18 files changed, 1330 insertions(+), 542 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java 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 8f587bc..fd98740 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 @@ -108,4 +108,9 @@ public interface Consumer extends Closeable { */ public void close(); + /** + * @see KafkaConsumer#wakeup() + */ + public void wakeup(); + } 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 1ca75f8..eb75d2e 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 @@ -27,6 +27,8 @@ import java.util.Map; * {@link Consumer#poll(long)} operation. */ public class ConsumerRecords implements Iterable> { + public static final ConsumerRecords EMPTY = + new ConsumerRecords(Collections.EMPTY_MAP); private final Map>> records; @@ -103,4 +105,9 @@ public class ConsumerRecords implements Iterable> { } } + @SuppressWarnings("unchecked") + public static ConsumerRecords empty() { + return (ConsumerRecords) EMPTY; + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java new file mode 100644 index 0000000..35f1ec9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java @@ -0,0 +1,20 @@ +/** + * 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; + +public class ConsumerWakeupException extends KafkaException { + private static final long serialVersionUID = 1L; + +} 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 951c34c..9be8fbc 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,44 +12,48 @@ */ package org.apache.kafka.clients.consumer; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedHashMap; -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 java.util.concurrent.atomic.AtomicReference; - import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.consumer.internals.Coordinator; import org.apache.kafka.clients.consumer.internals.Fetcher; +import org.apache.kafka.clients.consumer.internals.RequestFuture; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.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; import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.serialization.Deserializer; 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.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.ConcurrentModificationException; +import java.util.LinkedHashMap; +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.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.kafka.common.utils.Utils.min; + /** * A Kafka client that consumes records from a Kafka cluster. *

      @@ -298,10 +302,54 @@ import org.slf4j.LoggerFactory; * *

      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. + * The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application + * making the call. It is the responsibility of the user to ensure that multi-threaded access + * is properly synchronized. Un-synchronized access will result in {@link ConcurrentModificationException}. + * + *

      + * The only exception to this rule is {@link #wakeup()}, which can safely be used from an external thread to + * interrupt an active operation. In this case, a {@link ConsumerWakeupException} will be thrown from the thread + * blocking on the operation. This can be used to shutdown the consumer from another thread. The following + * snippet shows the typical pattern: + * + *

      + * public class KafkaConsumerRunner implements Runnable {
      + *     private final AtomicBoolean closed = new AtomicBoolean(false);
      + *     private final KafkaConsumer consumer;
      + *
      + *     public void run() {
      + *         try {
      + *             consumer.subscribe("topic");
      + *             while (!closed.get()) {
      + *                 ConsumerRecords records = consumer.poll(10000);
      + *                 // Handle new records
      + *             }
      + *         } catch (ConsumerWakeupException e) {
      + *             // Ignore exception if closing
      + *             if (!closed.get()) throw e;
      + *         } finally {
      + *             consumer.close();
      + *         }
      + *     }
      + *
      + *     public void shutdown() {
      + *         closed.set(true);
      + *         consumer.wakeup();
      + *     }
      + * }
      + * 
      + * + * Then in a separate thread, the consumer can be shutdown by setting the closed flag and waking up the consumer. + * + *
      + *     closed.set(true);
      + *     consumer.wakeup();
      + * 
      + * *

      - * This leaves several options for implementing multi-threaded processing of records. + * 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

      * @@ -363,6 +411,17 @@ public class KafkaConsumer implements Consumer { private final ConsumerRebalanceCallback rebalanceCallback; private long lastCommitAttemptMs; private boolean closed = false; + private final AtomicBoolean wakeup = new AtomicBoolean(false); + + // currentThread holds the threadId of the current thread accessing KafkaConsumer + // and is used to prevent multi-threaded access + private final AtomicReference currentThread = new AtomicReference(); + // refcount is used to allow reentrant access by the thread who has acquired currentThread + private int refcount = 0; // reference count for reentrant access + + // TODO: This timeout controls how long we should wait before retrying a request. We should be able + // to leverage the work of KAFKA-2120 to get this value from configuration. + private long requestTimeoutMs = 5000L; /** * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -480,13 +539,12 @@ public class KafkaConsumer implements Consumer { config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG), config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG), config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG)); - this.subscriptions = new SubscriptionState(); + OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase()); + this.subscriptions = new SubscriptionState(offsetResetStrategy); this.coordinator = new Coordinator(this.client, config.getString(ConsumerConfig.GROUP_ID_CONFIG), - this.retryBackoffMs, config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), - this.metadata, this.subscriptions, metrics, metricGrpPrefix, @@ -508,12 +566,10 @@ public class KafkaConsumer implements Consumer { this.valueDeserializer = valueDeserializer; } this.fetcher = new Fetcher(this.client, - this.retryBackoffMs, config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG), config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG), config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG), config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG), - config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase(), this.keyDeserializer, this.valueDeserializer, this.metadata, @@ -542,8 +598,13 @@ public class KafkaConsumer implements Consumer { * 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()); + public Set subscriptions() { + acquire(); + try { + return Collections.unmodifiableSet(this.subscriptions.assignedPartitions()); + } finally { + release(); + } } /** @@ -561,12 +622,16 @@ public class KafkaConsumer implements Consumer { * @param topics A variable list of topics that the consumer wants to subscribe to */ @Override - 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); + public void subscribe(String... topics) { + acquire(); + try { + log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", ")); + for (String topic : topics) + this.subscriptions.subscribe(topic); + metadata.addTopics(topics); + } finally { + release(); + } } /** @@ -574,16 +639,20 @@ public class KafkaConsumer implements Consumer { * 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 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()); + public void subscribe(TopicPartition... partitions) { + acquire(); + try { + log.debug("Subscribed to partitions(s): {}", Utils.join(partitions, ", ")); + for (TopicPartition tp : partitions) { + this.subscriptions.subscribe(tp); + metadata.addTopics(tp.topic()); + } + } finally { + release(); } } @@ -593,12 +662,16 @@ public class KafkaConsumer implements Consumer { * * @param topics Topics to unsubscribe from */ - 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) - this.subscriptions.unsubscribe(topic); + public void unsubscribe(String... topics) { + acquire(); + try { + log.debug("Unsubscribed from topic(s): {}", Utils.join(topics, ", ")); + // throw an exception if the topic was never subscribed to + for (String topic : topics) + this.subscriptions.unsubscribe(topic); + } finally { + release(); + } } /** @@ -607,12 +680,16 @@ public class KafkaConsumer implements Consumer { * * @param partitions Partitions to unsubscribe from */ - 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) - this.subscriptions.unsubscribe(partition); + public void unsubscribe(TopicPartition... partitions) { + acquire(); + try { + log.debug("Unsubscribed from partitions(s): {}", Utils.join(partitions, ", ")); + // throw an exception if the partition was never subscribed to + for (TopicPartition partition : partitions) + this.subscriptions.unsubscribe(partition); + } finally { + release(); + } } /** @@ -624,17 +701,65 @@ public class KafkaConsumer implements Consumer { * 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 + * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, returns + * immediately with any records available now. 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 synchronized ConsumerRecords poll(long timeout) { - ensureNotClosed(); - long now = time.milliseconds(); + public ConsumerRecords poll(long timeout) { + acquire(); + try { + if (timeout < 0) + throw new IllegalArgumentException("Timeout must not be negative"); + + // Poll for new data until the timeout expires + long remaining = timeout; + while (remaining >= 0) { + long start = time.milliseconds(); + long pollTimeout = min(remaining, timeToNextCommit(start), coordinator.timeToNextHeartbeat(start)); + + Map>> records = pollOnce(pollTimeout, start); + long end = time.milliseconds(); + + if (!records.isEmpty()) { + // If data is available, then return it, but first send off the + // next round of fetches to enable pipelining while the user is + // handling the fetched records. + fetcher.initFetches(metadata.fetch(), end); + pollClient(0, end); + return new ConsumerRecords(records); + } + + remaining -= end - start; + + // Nothing was available, so we should backoff before retrying + if (remaining > 0) { + Utils.sleep(min(remaining, retryBackoffMs)); + remaining -= time.milliseconds() - end; + } + } + + return ConsumerRecords.empty(); + } finally { + release(); + } + } + + + /** + * Do one round of polling. In addition to checking for new data, this does any needed + * heart-beating, auto-commits, and offset updates. + * @param timeout The maximum time to block in the underlying poll + * @param now Current time in millis + * @return The fetched records (may be empty) + */ + private Map>> pollOnce(long timeout, long now) { + Cluster cluster = this.metadata.fetch(); + + // TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) if (subscriptions.partitionsAutoAssigned()) { if (subscriptions.partitionAssignmentNeeded()) { @@ -649,26 +774,18 @@ public class KafkaConsumer implements Consumer { // fetch positions if we have partitions we're subscribed to that we // don't know the offset for if (!subscriptions.hasAllFetchPositions()) - updateFetchPositions(this.subscriptions.missingFetchPositions(), now); + updateFetchPositions(this.subscriptions.missingFetchPositions()); // 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(); + // Init any new fetches (won't resend pending fetches) fetcher.initFetches(cluster, now); - client.poll(timeout, now); - /* - * initiate a fetch request for any nodes that we just got a response from without blocking - */ - fetcher.initFetches(cluster, now); - client.poll(0, now); + pollClient(timeout, now); - return new ConsumerRecords(fetcher.fetchedRecords()); + return fetcher.fetchedRecords(); } /** @@ -686,18 +803,20 @@ public class KafkaConsumer implements Consumer { * @param commitType Control whether the commit is blocking */ @Override - public synchronized void commit(final Map offsets, CommitType commitType) { - ensureNotClosed(); - log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); + public void commit(final Map offsets, CommitType commitType) { + acquire(); + try { + log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); - long now = time.milliseconds(); - this.lastCommitAttemptMs = now; + this.lastCommitAttemptMs = time.milliseconds(); - // commit the offsets with the coordinator - boolean syncCommit = commitType.equals(CommitType.SYNC); - if (!syncCommit) - this.subscriptions.needRefreshCommits(); - coordinator.commitOffsets(offsets, syncCommit, now); + // commit the offsets with the coordinator + if (commitType == CommitType.ASYNC) + this.subscriptions.needRefreshCommits(); + commitOffsets(offsets, commitType); + } finally { + release(); + } } /** @@ -710,9 +829,13 @@ public class KafkaConsumer implements Consumer { * @param commitType Whether or not the commit should block until it is acknowledged. */ @Override - public synchronized void commit(CommitType commitType) { - ensureNotClosed(); - commit(this.subscriptions.allConsumed(), commitType); + public void commit(CommitType commitType) { + acquire(); + try { + commit(this.subscriptions.allConsumed(), commitType); + } finally { + release(); + } } /** @@ -721,35 +844,43 @@ public class KafkaConsumer implements Consumer { * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets */ @Override - public synchronized void seek(TopicPartition partition, long offset) { - ensureNotClosed(); - log.debug("Seeking to offset {} for partition {}", offset, partition); - this.subscriptions.seek(partition, offset); + public void seek(TopicPartition partition, long offset) { + acquire(); + try { + log.debug("Seeking to offset {} for partition {}", offset, partition); + this.subscriptions.seek(partition, offset); + } finally { + release(); + } } /** * Seek to the first offset for each of the given partitions */ - 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, fetcher.offsetBefore(tp, EARLIEST_OFFSET_TIMESTAMP)); + public void seekToBeginning(TopicPartition... partitions) { + acquire(); + try { + Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() + : Arrays.asList(partitions); + for (TopicPartition tp : parts) + subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); + } finally { + release(); } } /** * Seek to the last offset for each of the given partitions */ - 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, fetcher.offsetBefore(tp, LATEST_OFFSET_TIMESTAMP)); + public void seekToEnd(TopicPartition... partitions) { + acquire(); + try { + Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() + : Arrays.asList(partitions); + for (TopicPartition tp : parts) + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + } finally { + release(); } } @@ -761,16 +892,20 @@ public class KafkaConsumer implements Consumer { * @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) { - updateFetchPositions(Collections.singleton(partition), time.milliseconds()); - return this.subscriptions.consumed(partition); - } else { - return offset; + public long position(TopicPartition partition) { + acquire(); + try { + 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) { + updateFetchPositions(Collections.singleton(partition)); + return this.subscriptions.consumed(partition); + } else { + return offset; + } + } finally { + release(); } } @@ -787,22 +922,26 @@ public class KafkaConsumer implements Consumer { * partition. */ @Override - public synchronized long committed(TopicPartition partition) { - ensureNotClosed(); - Set partitionsToFetch; - if (subscriptions.assignedPartitions().contains(partition)) { + public long committed(TopicPartition partition) { + acquire(); + try { + 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); + } + refreshCommittedOffsets(partitionsToFetch); Long committed = this.subscriptions.committed(partition); - if (committed != null) - return committed; - partitionsToFetch = subscriptions.assignedPartitions(); - } else { - partitionsToFetch = Collections.singleton(partition); + if (committed == null) + throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); + return committed; + } finally { + release(); } - refreshCommittedOffsets(partitionsToFetch, time.milliseconds()); - Long committed = this.subscriptions.committed(partition); - if (committed == null) - throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); - return committed; } /** @@ -822,19 +961,41 @@ public class KafkaConsumer implements Consumer { */ @Override 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); + acquire(); + try { + Cluster cluster = this.metadata.fetch(); + List parts = cluster.partitionsForTopic(topic); + if (parts == null) { + metadata.add(topic); + awaitMetadataUpdate(); + parts = metadata.fetch().partitionsForTopic(topic); + } + return parts; + } finally { + release(); } - return parts; } @Override - public synchronized void close() { - close(false); + public void close() { + if (closed) return; + + acquire(); + try { + close(false); + } finally { + release(); + } + } + + /** + * Wakeup the consumer. This method is thread-safe and is useful in particular to abort a long poll. + * The thread which is blocking in an operation will throw {@link ConsumerWakeupException}. + */ + @Override + public void wakeup() { + this.wakeup.set(true); + this.client.wakeup(); } private void close(boolean swallowException) { @@ -856,6 +1017,15 @@ public class KafkaConsumer implements Consumer { return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs; } + private long timeToNextCommit(long now) { + if (!this.autoCommit) + return Long.MAX_VALUE; + long timeSinceLastCommit = now - this.lastCommitAttemptMs; + if (timeSinceLastCommit > this.autoCommitIntervalMs) + return 0; + return this.autoCommitIntervalMs - timeSinceLastCommit; + } + /** * Request a metadata update and wait until it has occurred */ @@ -863,7 +1033,7 @@ public class KafkaConsumer implements Consumer { int version = this.metadata.requestUpdate(); do { long now = time.milliseconds(); - this.client.poll(this.retryBackoffMs, now); + this.pollClient(this.retryBackoffMs, now); } while (this.metadata.version() == version); } @@ -881,8 +1051,7 @@ public class KafkaConsumer implements Consumer { } // get new assigned partitions from the coordinator - this.subscriptions.changePartitionAssignment(coordinator.assignPartitions( - new ArrayList(this.subscriptions.subscribedTopics()), now)); + assignPartitions(); // execute the user's callback after rebalance log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); @@ -899,25 +1068,73 @@ public class KafkaConsumer implements Consumer { * or reset it using the offset reset policy the user has configured. * * @param partitions The partitions that needs updating fetch positions - * @param now The current time * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is * defined */ - private void updateFetchPositions(Set partitions, long now) { + private void updateFetchPositions(Set partitions) { // first refresh the committed positions in case they are not up-to-date - refreshCommittedOffsets(partitions, now); + refreshCommittedOffsets(partitions); // reset the fetch position to the committed position for (TopicPartition tp : partitions) { - if (subscriptions.fetched(tp) == null) { - if (subscriptions.committed(tp) == null) { - // if the committed position is unknown reset the position - fetcher.resetOffset(tp); - } else { - log.debug("Resetting offset for partition {} to the committed offset {}", - tp, subscriptions.committed(tp)); - subscriptions.seek(tp, subscriptions.committed(tp)); - } + // Skip if we already have a fetch position + if (subscriptions.fetched(tp) != null) + continue; + + // TODO: If there are several offsets to reset, we could submit offset requests in parallel + if (subscriptions.isOffsetResetNeeded(tp)) { + resetOffset(tp); + } else if (subscriptions.committed(tp) == null) { + // There's no committed position, so we need to reset with the default strategy + subscriptions.needOffsetReset(tp); + resetOffset(tp); + } else { + log.debug("Resetting offset for partition {} to the committed offset {}", + tp, subscriptions.committed(tp)); + subscriptions.seek(tp, subscriptions.committed(tp)); + } + } + } + + /** + * Reset offsets for the given partition using the offset reset strategy. + * + * @param partition The given partition that needs reset offset + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined + */ + private void resetOffset(TopicPartition partition) { + OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); + final long timestamp; + if (strategy == OffsetResetStrategy.EARLIEST) + timestamp = EARLIEST_OFFSET_TIMESTAMP; + else if (strategy == OffsetResetStrategy.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, strategy.name().toLowerCase()); + long offset = listOffset(partition, timestamp); + this.subscriptions.seek(partition, offset); + } + + /** + * Fetch a single offset before the given timestamp for the partition. + * + * @param partition The partition that needs fetching offset. + * @param timestamp The timestamp for fetching offset. + * @return The offset of the message that is published before the given timestamp + */ + private long listOffset(TopicPartition partition, long timestamp) { + while (true) { + RequestFuture future = fetcher.listOffset(partition, timestamp); + + if (!future.isDone()) + pollFuture(future, requestTimeoutMs); + + if (future.isDone()) { + if (future.succeeded()) + return future.value(); + handleRequestFailure(future); } } } @@ -925,13 +1142,13 @@ public class KafkaConsumer implements Consumer { /** * Refresh the committed offsets for given set of partitions and update the cache */ - private void refreshCommittedOffsets(Set partitions, long now) { + private void refreshCommittedOffsets(Set partitions) { // we only need to fetch latest committed offset from coordinator if there // is some commit process in progress, otherwise our current // committed cache is up-to-date if (subscriptions.refreshCommitsNeeded()) { // contact coordinator to fetch committed offsets - Map offsets = coordinator.fetchOffsets(partitions, now); + Map offsets = fetchCommittedOffsets(partitions); // update the position with the offsets for (Map.Entry entry : offsets.entrySet()) { @@ -941,6 +1158,183 @@ public class KafkaConsumer implements Consumer { } } + /** + * Block until we have received a partition assignment from the coordinator. + */ + private void assignPartitions() { + // Ensure that there are no pending requests to the coordinator. This is important + // in particular to avoid resending a pending JoinGroup request. + awaitCoordinatorInFlightRequests(); + + while (subscriptions.partitionAssignmentNeeded()) { + RequestFuture future = coordinator.assignPartitions(time.milliseconds()); + + // Block indefinitely for the join group request (which can take as long as a session timeout) + if (!future.isDone()) + pollFuture(future); + + if (future.failed()) + handleRequestFailure(future); + } + } + + /** + * Block until the coordinator for this group is known. + */ + private void ensureCoordinatorKnown() { + while (coordinator.coordinatorUnknown()) { + RequestFuture future = coordinator.discoverConsumerCoordinator(); + + if (!future.isDone()) + pollFuture(future, requestTimeoutMs); + + if (future.failed()) + handleRequestFailure(future); + } + } + + /** + * Block until any pending requests to the coordinator have been handled. + */ + public void awaitCoordinatorInFlightRequests() { + while (coordinator.hasInFlightRequests()) { + long now = time.milliseconds(); + pollClient(-1, now); + } + } + + /** + * Lookup the committed offsets for a set of partitions. This will block until the coordinator has + * responded to the offset fetch request. + * @param partitions List of partitions to get offsets for + * @return Map from partition to its respective offset + */ + private Map fetchCommittedOffsets(Set partitions) { + while (true) { + long now = time.milliseconds(); + RequestFuture> future = coordinator.fetchOffsets(partitions, now); + + if (!future.isDone()) + pollFuture(future, requestTimeoutMs); + + if (future.isDone()) { + if (future.succeeded()) + return future.value(); + handleRequestFailure(future); + } + } + } + + /** + * Commit offsets. This call blocks (regardless of commitType) until the coordinator + * can receive the commit request. Once the request has been made, however, only the + * synchronous commits will wait for a successful response from the coordinator. + * @param offsets Offsets to commit. + * @param commitType Commit policy + */ + private void commitOffsets(Map offsets, CommitType commitType) { + if (commitType == CommitType.ASYNC) { + commitOffsetsAsync(offsets); + } else { + commitOffsetsSync(offsets); + } + } + + private void commitOffsetsAsync(Map offsets) { + while (true) { + long now = time.milliseconds(); + RequestFuture future = coordinator.commitOffsets(offsets, now); + + if (!future.isDone() || future.succeeded()) + return; + + handleRequestFailure(future); + } + } + + private void commitOffsetsSync(Map offsets) { + while (true) { + long now = time.milliseconds(); + RequestFuture future = coordinator.commitOffsets(offsets, now); + + if (!future.isDone()) + pollFuture(future, requestTimeoutMs); + + if (future.isDone()) { + if (future.succeeded()) + return; + else + handleRequestFailure(future); + } + } + } + + private void handleRequestFailure(RequestFuture future) { + if (future.hasException()) + throw future.exception(); + + switch (future.retryAction()) { + case BACKOFF: + Utils.sleep(retryBackoffMs); + break; + case POLL: + pollClient(retryBackoffMs, time.milliseconds()); + break; + case FIND_COORDINATOR: + ensureCoordinatorKnown(); + break; + case REFRESH_METADATA: + awaitMetadataUpdate(); + break; + case NOOP: + // Do nothing (retry now) + } + } + + /** + * Poll until a result is ready or timeout expires + * @param future The future to poll for + * @param timeout The time in milliseconds to wait for the result + */ + private void pollFuture(RequestFuture future, long timeout) { + // TODO: Update this code for KAFKA-2120, which adds request timeout to NetworkClient + // In particular, we must ensure that "timed out" requests will not have their callbacks + // invoked at a later time. + long remaining = timeout; + while (!future.isDone() && remaining >= 0) { + long start = time.milliseconds(); + pollClient(remaining, start); + if (future.isDone()) return; + remaining -= time.milliseconds() - start; + } + } + + /** + * Poll indefinitely until the result is ready. + * @param future The future to poll for. + */ + private void pollFuture(RequestFuture future) { + while (!future.isDone()) { + long now = time.milliseconds(); + pollClient(-1, now); + } + } + + /** + * Poll for IO. + * @param timeout The maximum time to wait for IO to become available + * @param now The current time in milliseconds + * @throws ConsumerWakeupException if {@link #wakeup()} is invoked while the poll is active + */ + private void pollClient(long timeout, long now) { + this.client.poll(timeout, now); + + if (wakeup.get()) { + wakeup.set(false); + throw new ConsumerWakeupException(); + } + } + /* * Check that the consumer hasn't been closed. */ @@ -948,4 +1342,27 @@ public class KafkaConsumer implements Consumer { if (this.closed) throw new IllegalStateException("This consumer has already been closed."); } + + /** + * Acquire the light lock protecting this consumer from multi-threaded access. Instead of blocking + * when the lock is not available, however, we just throw an exception (since multi-threaded usage is not + * supported). + * @throws IllegalStateException if the consumer has been closed + * @throws ConcurrentModificationException if another thread already has the lock + */ + private void acquire() { + ensureNotClosed(); + Long threadId = Thread.currentThread().getId(); + if (!threadId.equals(currentThread.get()) && !currentThread.compareAndSet(null, threadId)) + throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access"); + refcount++; + } + + /** + * Release the light lock protecting the consumer from multi-threaded access. + */ + private void release() { + if (--refcount == 0) + currentThread.set(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 f50da82..46e26a6 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 @@ -40,8 +40,8 @@ public class MockConsumer implements Consumer { private Map>> records; private boolean closed; - public MockConsumer() { - this.subscriptions = new SubscriptionState(); + public MockConsumer(OffsetResetStrategy offsetResetStrategy) { + this.subscriptions = new SubscriptionState(offsetResetStrategy); this.partitions = new HashMap>(); this.records = new HashMap>>(); this.closed = false; @@ -175,6 +175,11 @@ public class MockConsumer implements Consumer { this.closed = true; } + @Override + public void wakeup() { + + } + 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/OffsetResetStrategy.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java new file mode 100644 index 0000000..542da7f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java @@ -0,0 +1,17 @@ +/** + * 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 OffsetResetStrategy { + LATEST, EARLIEST, NONE +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index 41cb945..6c26667 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -15,7 +15,6 @@ package org.apache.kafka.clients.consumer.internals; 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.KafkaException; import org.apache.kafka.common.MetricName; @@ -57,7 +56,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; /** - * This class manage the coordination process with the consumer coordinator. + * This class manages the coordination process with the consumer coordinator. */ public final class Coordinator { @@ -67,13 +66,11 @@ public final class Coordinator { private final Time time; private final String groupId; - private final Metadata metadata; private final Heartbeat heartbeat; private final int sessionTimeoutMs; private final String assignmentStrategy; private final SubscriptionState subscriptions; private final CoordinatorMetrics sensors; - private final long retryBackoffMs; private Node consumerCoordinator; private String consumerId; private int generation; @@ -83,10 +80,8 @@ public final class Coordinator { */ public Coordinator(KafkaClient client, String groupId, - long retryBackoffMs, int sessionTimeoutMs, String assignmentStrategy, - Metadata metadata, SubscriptionState subscriptions, Metrics metrics, String metricGrpPrefix, @@ -98,10 +93,8 @@ public final class Coordinator { this.generation = -1; this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; this.groupId = groupId; - this.metadata = metadata; this.consumerCoordinator = null; this.subscriptions = subscriptions; - this.retryBackoffMs = retryBackoffMs; this.sessionTimeoutMs = sessionTimeoutMs; this.assignmentStrategy = assignmentStrategy; this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); @@ -109,84 +102,110 @@ public final class Coordinator { } /** - * Assign partitions for the subscribed topics. - * - * @param subscribedTopics The subscribed topics list - * @param now The current time - * @return The assigned partition info + * Send a request to get a new partition assignment. This is a non-blocking call which sends + * a JoinGroup request to the coordinator (if it is available). The returned future must + * be polled to see if the request completed successfully. + * @param now The current time in milliseconds + * @return A request future whose completion indicates the result of the JoinGroup request. */ - public List assignPartitions(List subscribedTopics, long now) { + public RequestFuture assignPartitions(final long now) { + final RequestFuture future = newCoordinatorRequestFuture(now); + if (future.isDone()) return future; // send a join group request to the coordinator + List subscribedTopics = new ArrayList(subscriptions.subscribedTopics()); log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); - // repeat processing the response until succeed or fatal error - do { - JoinGroupRequest request = new JoinGroupRequest(groupId, + JoinGroupRequest request = new JoinGroupRequest(groupId, this.sessionTimeoutMs, subscribedTopics, this.consumerId, this.assignmentStrategy); - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now); - JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); - short errorCode = response.errorCode(); + // create the request for the coordinator + log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); + + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + handleJoinResponse(resp, future); + } + }; + + sendCoordinator(ApiKeys.JOIN_GROUP, request.toStruct(), completionHandler, now); + return future; + } + + private void handleJoinResponse(ClientResponse response, RequestFuture future) { + if (response.wasDisconnected()) { + handleCoordinatorDisconnect(response); + future.retryWithNewCoordinator(); + } else { + // process the response + JoinGroupResponse joinResponse = new JoinGroupResponse(response.responseBody()); + short errorCode = joinResponse.errorCode(); if (errorCode == Errors.NONE.code()) { - this.consumerId = response.consumerId(); - this.generation = response.generationId(); + Coordinator.this.consumerId = joinResponse.consumerId(); + Coordinator.this.generation = joinResponse.generationId(); // set the flag to refresh last committed offsets - this.subscriptions.needRefreshCommits(); + subscriptions.needRefreshCommits(); log.debug("Joined group: {}", response); // record re-assignment time - this.sensors.partitionReassignments.record(time.milliseconds() - now); + this.sensors.partitionReassignments.record(response.requestLatencyMs()); - // return assigned partitions - return response.assignedPartitions(); + // update partition assignment + subscriptions.changePartitionAssignment(joinResponse.assignedPartitions()); + future.complete(null); } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { // reset the consumer id and retry immediately - this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", - groupId); + groupId); + + future.retryNow(); } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { // re-discover the coordinator and retry with backoff coordinatorDead(); - Utils.sleep(this.retryBackoffMs); - log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", - groupId); + groupId); + future.retryWithNewCoordinator(); } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code() || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code() || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) { // log the error and re-throw the exception + KafkaException e = Errors.forCode(errorCode).exception(); log.error("Attempt to join group {} failed due to: {}", - groupId, Errors.forCode(errorCode).exception().getMessage()); - Errors.forCode(errorCode).maybeThrow(); + groupId, e.getMessage()); + future.raise(e); } else { // unexpected error, throw the exception - throw new KafkaException("Unexpected error in join group response: " - + Errors.forCode(response.errorCode()).exception().getMessage()); + future.raise(new KafkaException("Unexpected error in join group response: " + + Errors.forCode(joinResponse.errorCode()).exception().getMessage())); } - } while (true); + } } /** - * Commit offsets for the specified list of topics and partitions. - * - * 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. + * Commit offsets for the specified list of topics and partitions. This is a non-blocking call + * which returns a request future that can be polled in the case of a synchronous commit or ignored in the + * asynchronous case. * * @param offsets The list of offsets per partition that should be committed. - * @param blocking Control whether the commit is blocking * @param now The current time + * @return A request future whose value indicates whether the commit was successful or not */ - public void commitOffsets(final Map offsets, boolean blocking, long now) { - if (!offsets.isEmpty()) { + public RequestFuture commitOffsets(final Map offsets, long now) { + final RequestFuture future = newCoordinatorRequestFuture(now); + if (future.isDone()) return future; + + if (offsets.isEmpty()) { + future.complete(null); + } else { // create the offset commit request Map offsetData; offsetData = new HashMap(offsets.size()); @@ -198,52 +217,63 @@ public final class Coordinator { OffsetCommitRequest.DEFAULT_RETENTION_TIME, offsetData); - // send request and possibly wait for response if it is blocking - RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets); + RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets, future); + sendCoordinator(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); + } - if (blocking) { - boolean done; - do { - ClientResponse response = blockingCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); + return future; + } - // 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); - } else { - this.client.send(initiateCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now)); - } + private RequestFuture newCoordinatorRequestFuture(long now) { + if (coordinatorUnknown()) + return RequestFuture.newCoordinatorNeeded(); + + if (client.ready(this.consumerCoordinator, now)) + // We have an open connection and we're ready to send + return new RequestFuture(); + + if (this.client.connectionFailed(this.consumerCoordinator)) { + coordinatorDead(); + return RequestFuture.newCoordinatorNeeded(); } + + // The connection has been initiated, so we need to poll to finish it + return RequestFuture.pollNeeded(); } /** - * Fetch the committed offsets of the given set of partitions. + * Fetch the committed offsets for a set of partitions. This is a non-blocking call. The + * returned future can be polled to get the actual offsets returned from the broker. * - * @param partitions The list of partitions which need to ask for committed offsets - * @param now The current time - * @return The fetched offset values + * @param partitions The set of partitions to get offsets for. + * @param now The current time in milliseconds + * @return A request future containing the committed offsets. */ - public Map fetchOffsets(Set partitions, long now) { - log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); - - while (true) { - // construct the request - OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); + public RequestFuture> fetchOffsets(Set partitions, long now) { + final RequestFuture> future = newCoordinatorRequestFuture(now); + if (future.isDone()) return future; - // send the request and block on waiting for response - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.OFFSET_FETCH, request.toStruct(), null, now); + log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); + // construct the request + OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); + + // send the request with a callback + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + handleOffsetResponse(resp, future); + } + }; + sendCoordinator(ApiKeys.OFFSET_FETCH, request.toStruct(), completionHandler, now); + return future; + } + private void handleOffsetResponse(ClientResponse resp, RequestFuture> future) { + if (resp.wasDisconnected()) { + handleCoordinatorDisconnect(resp); + future.retryWithNewCoordinator(); + } else { // parse the response to get the offsets - boolean offsetsReady = true; OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); Map offsets = new HashMap(response.responseData().size()); for (Map.Entry entry : response.responseData().entrySet()) { @@ -251,23 +281,21 @@ public final class Coordinator { OffsetFetchResponse.PartitionData data = entry.getValue(); if (data.hasError()) { log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) - .exception() - .getMessage()); + .exception() + .getMessage()); if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { // just retry - offsetsReady = false; - Utils.sleep(this.retryBackoffMs); + future.retryAfterBackoff(); } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { // re-discover the coordinator and retry coordinatorDead(); - offsetsReady = false; - Utils.sleep(this.retryBackoffMs); + future.retryWithNewCoordinator(); } else if (data.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { // just ignore this partition log.debug("Unknown topic or partition for " + tp); } else { - throw new KafkaException("Unexpected error in fetch offset response: " - + Errors.forCode(data.errorCode).exception().getMessage()); + future.raise(new KafkaException("Unexpected error in fetch offset response: " + + Errors.forCode(data.errorCode).exception().getMessage())); } } else if (data.offset >= 0) { // record the position with the offset (-1 indicates no committed offset to fetch) @@ -277,8 +305,8 @@ public final class Coordinator { } } - if (offsetsReady) - return offsets; + if (!future.isDone()) + future.complete(offsets); } } @@ -288,124 +316,105 @@ public final class Coordinator { * @param now The current time */ public void maybeHeartbeat(long now) { - if (heartbeat.shouldHeartbeat(now)) { + if (heartbeat.shouldHeartbeat(now) && coordinatorReady(now)) { HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); - this.client.send(initiateCoordinatorRequest(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now)); + sendCoordinator(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now); this.heartbeat.sentHeartbeat(now); } } - public boolean coordinatorUnknown() { - return this.consumerCoordinator == null; - } - /** - * Repeatedly attempt to send a request to the coordinator 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 + * Get the time until the next heartbeat is needed. + * @param now The current time + * @return The duration in milliseconds before the next heartbeat will be needed. */ - private ClientResponse blockingCoordinatorRequest(ApiKeys api, - Struct request, - RequestCompletionHandler handler, - long now) { - while (true) { - ClientRequest coordinatorRequest = initiateCoordinatorRequest(api, request, handler, now); - ClientResponse coordinatorResponse = sendAndReceive(coordinatorRequest, now); - if (coordinatorResponse.wasDisconnected()) { - handleCoordinatorDisconnect(coordinatorResponse); - Utils.sleep(this.retryBackoffMs); - } else { - return coordinatorResponse; - } - } + public long timeToNextHeartbeat(long now) { + return heartbeat.timeToNextHeartbeat(now); } /** - * Ensure the consumer coordinator is known and we have a ready connection to it. + * Check whether the coordinator has any in-flight requests. + * @return true if the coordinator has pending requests. */ - 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 coordinator, attempting to connect."); - this.client.poll(this.retryBackoffMs, time.milliseconds()); + public boolean hasInFlightRequests() { + return !coordinatorUnknown() && client.inFlightRequestCount(consumerCoordinator.idString()) > 0; + } - // if the coordinator connection has failed, we need to - // break the inner loop to re-discover the coordinator - if (this.client.connectionFailed(this.consumerCoordinator)) { - log.debug("Coordinator connection failed. Attempting to re-discover."); - coordinatorDead(); - break; - } - } - } - } + public boolean coordinatorUnknown() { + return this.consumerCoordinator == null; } - /** - * Mark the current coordinator as dead. - */ - private void coordinatorDead() { - if (this.consumerCoordinator != null) { - log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id()); - this.consumerCoordinator = null; - } + private boolean coordinatorReady(long now) { + return !coordinatorUnknown() && this.client.ready(this.consumerCoordinator, now); } /** - * Keep discovering the consumer coordinator until it is found. + * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to + * one of the brokers. The returned future should be polled to get the result of the request. + * @return A request future which indicates the completion of the metadata request */ - private void discoverCoordinator() { - while (this.consumerCoordinator == null) { - log.debug("No coordinator known, attempting to discover one."); - Node coordinator = fetchConsumerCoordinator(); - - if (coordinator == null) { - log.debug("No coordinator found, backing off."); - Utils.sleep(this.retryBackoffMs); + public RequestFuture discoverConsumerCoordinator() { + // initiate the consumer metadata request + // find a node to ask about the coordinator + long now = time.milliseconds(); + Node node = this.client.leastLoadedNode(now); + + if (node == null) { + return RequestFuture.metadataRefreshNeeded(); + } else if (!this.client.ready(node, now)) { + if (this.client.connectionFailed(node)) { + return RequestFuture.metadataRefreshNeeded(); } else { - log.debug("Found coordinator: " + coordinator); - this.consumerCoordinator = coordinator; + return RequestFuture.pollNeeded(); } + } else { + final RequestFuture future = new RequestFuture(); + + // create a consumer metadata request + log.debug("Issuing consumer metadata request to broker {}", node.id()); + ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId); + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + handleConsumerMetadataResponse(resp, future); + } + }; + send(node, ApiKeys.CONSUMER_METADATA, metadataRequest.toStruct(), completionHandler, now); + return future; } } - /** - * Get the current consumer coordinator information via consumer metadata request. - * - * @return the consumer coordinator node - */ - private Node fetchConsumerCoordinator() { - - // initiate the consumer metadata request - ClientRequest request = initiateConsumerMetadataRequest(); - - // send the request and wait for its response - ClientResponse response = sendAndReceive(request, request.createdTime()); + private void handleConsumerMetadataResponse(ClientResponse resp, RequestFuture future) { + log.debug("Consumer metadata response {}", resp); // parse the response to get the coordinator info if it is not disconnected, // otherwise we need to request metadata update - if (!response.wasDisconnected()) { - ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(response.responseBody()); + if (resp.wasDisconnected()) { + future.retryAfterMetadataRefresh(); + } else { + ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody()); // use MAX_VALUE - node.id as the coordinator id to mimic separate connections // for the coordinator in the underlying network client layer // TODO: this needs to be better handled in KAFKA-1935 - if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) - return new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), - consumerMetadataResponse.node().host(), - consumerMetadataResponse.node().port()); - } else { - this.metadata.requestUpdate(); + if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) { + this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), + consumerMetadataResponse.node().host(), + consumerMetadataResponse.node().port()); + future.complete(null); + } else { + future.retryAfterBackoff(); + } } + } - return null; + /** + * Mark the current coordinator as dead. + */ + private void coordinatorDead() { + if (this.consumerCoordinator != null) { + log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id()); + this.consumerCoordinator = null; + } } /** @@ -414,79 +423,23 @@ public final class Coordinator { private void handleCoordinatorDisconnect(ClientResponse response) { int correlation = response.request().request().header().correlationId(); log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected", - response.request(), - correlation, - response.request().request().destination()); + response.request(), + correlation, + response.request().request().destination()); // mark the coordinator as dead coordinatorDead(); } - /** - * Initiate a consumer metadata request to the least loaded node. - * - * @return The created request - */ - private ClientRequest initiateConsumerMetadataRequest() { - // find a node to ask about the coordinator - 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); - - // if there is no ready node, backoff before retry - if (node == null) - Utils.sleep(this.retryBackoffMs); - } - - // create a consumer metadata request - log.debug("Issuing consumer metadata request to broker {}", node.id()); - - ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.groupId); - RequestSend send = new RequestSend(node.idString(), - this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), - request.toStruct()); - long now = time.milliseconds(); - return new ClientRequest(now, true, send, null); + private void sendCoordinator(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { + send(this.consumerCoordinator, api, request, handler, now); } - /** - * Initiate a request to the coordinator. - */ - private ClientRequest initiateCoordinatorRequest(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { - - // first make sure the coordinator is known and ready - ensureCoordinatorReady(); - - // create the request for the coordinator - log.debug("Issuing request ({}: {}) to coordinator {}", api, request, this.consumerCoordinator.id()); - + private void send(Node node, ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { RequestHeader header = this.client.nextRequestHeader(api); - RequestSend send = new RequestSend(this.consumerCoordinator.idString(), header, request); - return new ClientRequest(now, true, send, handler); - } - - /** - * Attempt to send a request and receive its response. - * - * @return The response - */ - private ClientResponse sendAndReceive(ClientRequest clientRequest, long now) { - - // send the request - this.client.send(clientRequest); - - // drain all responses from the destination node - List responses = this.client.completeAll(clientRequest.request().destination(), now); - if (responses.isEmpty()) { - throw new IllegalStateException("This should not happen."); - } else { - // other requests should be handled by the callback, and - // we only care about the response of the last request - return responses.get(responses.size() - 1); - } + RequestSend send = new RequestSend(node.idString(), header, request); + this.client.send(new ClientRequest(now, true, send, handler)); } private class HeartbeatCompletionHandler implements RequestCompletionHandler { @@ -521,18 +474,21 @@ public final class Coordinator { private class CommitOffsetCompletionHandler implements RequestCompletionHandler { private final Map offsets; + private final RequestFuture future; - public CommitOffsetCompletionHandler(Map offsets) { + public CommitOffsetCompletionHandler(Map offsets, RequestFuture future) { this.offsets = offsets; + this.future = future; } @Override public void onComplete(ClientResponse resp) { if (resp.wasDisconnected()) { handleCoordinatorDisconnect(resp); + future.retryWithNewCoordinator(); } else { - OffsetCommitResponse response = new OffsetCommitResponse(resp.responseBody()); - for (Map.Entry entry : response.responseData().entrySet()) { + OffsetCommitResponse commitResponse = new OffsetCommitResponse(resp.responseBody()); + for (Map.Entry entry : commitResponse.responseData().entrySet()) { TopicPartition tp = entry.getKey(); short errorCode = entry.getValue(); long offset = this.offsets.get(tp); @@ -542,14 +498,19 @@ public final class Coordinator { } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { coordinatorDead(); + future.retryWithNewCoordinator(); } else { // do not need to throw the exception but just log the error + future.retryAfterBackoff(); log.error("Error committing partition {} at offset {}: {}", tp, offset, Errors.forCode(errorCode).exception().getMessage()); } } + + if (!future.isDone()) + future.complete(null); } sensors.commitLatency.record(resp.requestLatencyMs()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 56281ee..695eaf6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; @@ -61,9 +60,6 @@ import java.util.Map; public class Fetcher { private static final Logger log = LoggerFactory.getLogger(Fetcher.class); - private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; - private static final long LATEST_OFFSET_TIMESTAMP = -1L; - private final KafkaClient client; @@ -72,23 +68,19 @@ public class Fetcher { private final int maxWaitMs; private final int fetchSize; private final boolean checkCrcs; - private final long retryBackoffMs; private final Metadata metadata; private final FetchManagerMetrics sensors; private final SubscriptionState subscriptions; private final List> records; - private final AutoOffsetResetStrategy offsetResetStrategy; private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; public Fetcher(KafkaClient client, - long retryBackoffMs, int minBytes, int maxWaitMs, int fetchSize, boolean checkCrcs, - String offsetReset, Deserializer keyDeserializer, Deserializer valueDeserializer, Metadata metadata, @@ -102,17 +94,16 @@ public class Fetcher { this.client = client; this.metadata = metadata; this.subscriptions = subscriptions; - this.retryBackoffMs = retryBackoffMs; this.minBytes = minBytes; this.maxWaitMs = maxWaitMs; this.fetchSize = fetchSize; this.checkCrcs = checkCrcs; - this.offsetResetStrategy = AutoOffsetResetStrategy.valueOf(offsetReset); this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; this.records = new LinkedList>(); + this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags); } @@ -166,84 +157,76 @@ public class Fetcher { } /** - * Reset offsets for the given partition using the offset reset strategy. - * - * @param partition The given partition that needs reset offset - * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined - */ - public void resetOffset(TopicPartition partition) { - 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, offsetBefore(partition, timestamp)); - } - - /** * Fetch a single offset before the given timestamp for the partition. * * @param topicPartition The partition that needs fetching offset. * @param timestamp The timestamp for fetching offset. - * @return The offset of the message that is published before the given timestamp + * @return A response which can be polled to obtain the corresponding offset. */ - public long offsetBefore(TopicPartition topicPartition, long timestamp) { - log.debug("Fetching offsets for partition {}.", topicPartition); + public RequestFuture listOffset(final TopicPartition topicPartition, long timestamp) { Map partitions = new HashMap(1); partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1)); - while (true) { - long now = time.milliseconds(); - PartitionInfo info = metadata.fetch().partition(topicPartition); - if (info == null) { - metadata.add(topicPartition.topic()); - log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition); - awaitMetadataUpdate(); - } else if (info.leader() == null) { - log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); - awaitMetadataUpdate(); - } else if (this.client.ready(info.leader(), now)) { - Node node = info.leader(); - ListOffsetRequest request = new ListOffsetRequest(-1, partitions); - RequestSend send = new RequestSend(node.idString(), + long now = time.milliseconds(); + PartitionInfo info = metadata.fetch().partition(topicPartition); + if (info == null) { + metadata.add(topicPartition.topic()); + log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition); + return RequestFuture.metadataRefreshNeeded(); + } else if (info.leader() == null) { + log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); + return RequestFuture.metadataRefreshNeeded(); + } else if (this.client.ready(info.leader(), now)) { + final RequestFuture future = new RequestFuture(); + Node node = info.leader(); + ListOffsetRequest request = new ListOffsetRequest(-1, partitions); + RequestSend send = new RequestSend(node.idString(), 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.idString(), 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(topicPartition).errorCode; - if (errorCode == Errors.NONE.code()) { - List offsets = lor.responseData().get(topicPartition).offsets; - if (offsets.size() != 1) - throw new IllegalStateException("This should not happen."); - long offset = offsets.get(0); - log.debug("Fetched offset {} for partition {}", offset, topicPartition); - return offset; - } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() - || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { - log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", - topicPartition); - awaitMetadataUpdate(); - } else { - log.error("Attempt to fetch offsets for partition {} failed due to: {}", - topicPartition, Errors.forCode(errorCode).exception().getMessage()); - awaitMetadataUpdate(); - } + RequestCompletionHandler completionHandler = new RequestCompletionHandler() { + @Override + public void onComplete(ClientResponse resp) { + handleListOffsetResponse(topicPartition, resp, future); } + }; + ClientRequest clientRequest = new ClientRequest(now, true, send, completionHandler); + this.client.send(clientRequest); + return future; + } else { + // We initiated a connect to the leader, but we need to poll to finish it. + return RequestFuture.pollNeeded(); + } + } + + /** + * Callback for the response of the list offset call above. + * @param topicPartition The partition that was fetched + * @param clientResponse The response from the server. + */ + private void handleListOffsetResponse(TopicPartition topicPartition, + ClientResponse clientResponse, + RequestFuture future) { + if (clientResponse.wasDisconnected()) { + future.retryAfterMetadataRefresh(); + } else { + ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody()); + short errorCode = lor.responseData().get(topicPartition).errorCode; + if (errorCode == Errors.NONE.code()) { + List offsets = lor.responseData().get(topicPartition).offsets; + if (offsets.size() != 1) + throw new IllegalStateException("This should not happen."); + long offset = offsets.get(0); + log.debug("Fetched offset {} for partition {}", offset, topicPartition); + + future.complete(offset); + } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() + || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { + log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", + topicPartition); + future.retryAfterMetadataRefresh(); } else { - log.debug("Leader for partition {} is not ready, retry fetching offsets", topicPartition); - client.poll(this.retryBackoffMs, now); + log.error("Attempt to fetch offsets for partition {} failed due to: {}", + topicPartition, Errors.forCode(errorCode).exception().getMessage()); + future.retryAfterMetadataRefresh(); } } } @@ -257,8 +240,10 @@ public class Fetcher { 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.idString()) == 0) { + if (node == null) { + metadata.requestUpdate(); + } else if (this.client.inFlightRequestCount(node.idString()) == 0) { + // if there is a leader and no in-flight requests, issue a new fetch Map fetch = fetchable.get(node.id()); if (fetch == null) { fetch = new HashMap(); @@ -327,7 +312,7 @@ public class Fetcher { } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { // TODO: this could be optimized by grouping all out-of-range partitions log.info("Fetch offset {} is out of range, resetting offset", subscriptions.fetched(tp)); - resetOffset(tp); + subscriptions.needOffsetReset(tp); } else if (partition.errorCode == Errors.UNKNOWN.code()) { log.warn("Unknown error fetching data for topic-partition {}", tp); } else { @@ -356,17 +341,6 @@ public class Fetcher { return new ConsumerRecord(partition.topic(), partition.partition(), offset, key, value); } - /* - * 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); - } - private static class PartitionRecords { public long fetchOffset; public TopicPartition partition; @@ -379,9 +353,6 @@ public class Fetcher { } } - private static enum AutoOffsetResetStrategy { - LATEST, EARLIEST, NONE - } private class FetchManagerMetrics { public final Metrics metrics; 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 e7cfaaa..51eae19 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 @@ -42,4 +42,14 @@ public final class Heartbeat { public long lastHeartbeatSend() { return this.lastHeartbeatSend; } + + public long timeToNextHeartbeat(long now) { + long timeSinceLastHeartbeat = now - lastHeartbeatSend; + + long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL; + if (timeSinceLastHeartbeat > hbInterval) + return 0; + else + return hbInterval - timeSinceLastHeartbeat; + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java new file mode 100644 index 0000000..13fc9af --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java @@ -0,0 +1,209 @@ +/** + * 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; + +/** + * Result of an asynchronous request through {@link org.apache.kafka.clients.KafkaClient}. To get the + * result of the request, you must use poll using {@link org.apache.kafka.clients.KafkaClient#poll(long, long)} + * until {@link #isDone()} returns true. Typical usage might look like this: + * + *

      + *     RequestFuture future = sendRequest();
      + *     while (!future.isDone()) {
      + *         client.poll(timeout, now);
      + *     }
      + *
      + *     switch (future.outcome()) {
      + *     case SUCCESS:
      + *         // handle request success
      + *         break;
      + *     case NEED_RETRY:
      + *         // retry after taking possible retry action
      + *         break;
      + *     case EXCEPTION:
      + *         // handle exception
      +  *     }
      + * 
      + * + * When {@link #isDone()} returns true, there are three possible outcomes (obtained through {@link #outcome()}): + * + *
        + *
      1. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#SUCCESS}: If the request was + * successful, then you can use {@link #value()} to obtain the result.
      2. + *
      3. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#EXCEPTION}: If an unhandled exception + * was encountered, you can use {@link #exception()} to get it.
      4. + *
      5. {@link org.apache.kafka.clients.consumer.internals.RequestFuture.Outcome#NEED_RETRY}: The request may + * not have been successful, but the failure may be ephemeral and the caller just needs to try the request again. + * In this case, use {@link #retryAction()} to determine what action should be taken (if any) before + * retrying.
      6. + *
      + * + * @param Return type of the result (Can be Void if there is no response) + */ +public class RequestFuture { + public static final RequestFuture NEED_NEW_COORDINATOR = newRetryFuture(RetryAction.FIND_COORDINATOR); + public static final RequestFuture NEED_POLL = newRetryFuture(RetryAction.POLL); + public static final RequestFuture NEED_METADATA_REFRESH = newRetryFuture(RetryAction.REFRESH_METADATA); + + public enum RetryAction { + NOOP, // Retry immediately. + POLL, // Retry after calling poll (e.g. to finish a connection) + BACKOFF, // Retry after a delay + FIND_COORDINATOR, // Find a new coordinator before retrying + REFRESH_METADATA // Refresh metadata before retrying + } + + public enum Outcome { + SUCCESS, + NEED_RETRY, + EXCEPTION + } + + private Outcome outcome; + private RetryAction retryAction; + private T value; + private RuntimeException exception; + + /** + * Check whether the response is ready to be handled + * @return true if the response is ready, false otherwise + */ + public boolean isDone() { + return outcome != null; + } + + /** + * Get the value corresponding to this request (if it has one, as indicated by {@link #outcome()}). + * @return the value if it exists or null + */ + public T value() { + return value; + } + + /** + * Check if the request succeeded; + * @return true if a value is available, false otherwise + */ + public boolean succeeded() { + return outcome == Outcome.SUCCESS; + } + + /** + * Check if the request completed failed. + * @return true if the request failed (whether or not it can be retried) + */ + public boolean failed() { + return outcome != Outcome.SUCCESS; + } + + /** + * Return the error from this response (assuming {@link #succeeded()} has returned false. If the + * response is not ready or if there is no retryAction, null is returned. + * @return the error if it exists or null + */ + public RetryAction retryAction() { + return retryAction; + } + + /** + * Get the exception from a failed result. You should check that there is an exception + * with {@link #hasException()} before using this method. + * @return The exception if it exists or null + */ + public RuntimeException exception() { + return exception; + } + + /** + * Check whether there was an exception. + * @return true if this request failed with an exception + */ + public boolean hasException() { + return outcome == Outcome.EXCEPTION; + } + + /** + * Check the outcome of the future if it is ready. + * @return the outcome or null if the future is not finished + */ + public Outcome outcome() { + return outcome; + } + + /** + * The request failed, but should be retried using the provided retry action. + * @param retryAction The action that should be taken by the caller before retrying the request + */ + public void retry(RetryAction retryAction) { + this.outcome = Outcome.NEED_RETRY; + this.retryAction = retryAction; + } + + public void retryNow() { + retry(RetryAction.NOOP); + } + + public void retryAfterBackoff() { + retry(RetryAction.BACKOFF); + } + + public void retryWithNewCoordinator() { + retry(RetryAction.FIND_COORDINATOR); + } + + public void retryAfterMetadataRefresh() { + retry(RetryAction.REFRESH_METADATA); + } + + /** + * Complete the request successfully. After this call, {@link #succeeded()} will return true + * and the value can be obtained through {@link #value()}. + * @param value corresponding value (or null if there is none) + */ + public void complete(T value) { + this.outcome = Outcome.SUCCESS; + this.value = value; + } + + /** + * Raise an exception. The request will be marked as failed, and the caller can either + * handle the exception or throw it. + * @param e The exception that + */ + public void raise(RuntimeException e) { + this.outcome = Outcome.EXCEPTION; + this.exception = e; + } + + private static RequestFuture newRetryFuture(RetryAction retryAction) { + RequestFuture result = new RequestFuture(); + result.retry(retryAction); + return result; + } + + @SuppressWarnings("unchecked") + public static RequestFuture pollNeeded() { + return (RequestFuture) NEED_POLL; + } + + @SuppressWarnings("unchecked") + public static RequestFuture metadataRefreshNeeded() { + return (RequestFuture) NEED_METADATA_REFRESH; + } + + @SuppressWarnings("unchecked") + public static RequestFuture newCoordinatorNeeded() { + return (RequestFuture) NEED_NEW_COORDINATOR; + } + +} 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 cee7541..6837453 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 @@ -12,14 +12,15 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; + 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 */ @@ -49,7 +50,14 @@ public class SubscriptionState { /* do we need to request the latest committed offsets from the coordinator? */ private boolean needsFetchCommittedOffsets; - public SubscriptionState() { + /* Partitions that need to be reset before fetching */ + private Map resetPartitions; + + /* Default offset reset strategy */ + private OffsetResetStrategy offsetResetStrategy; + + public SubscriptionState(OffsetResetStrategy offsetResetStrategy) { + this.offsetResetStrategy = offsetResetStrategy; this.subscribedTopics = new HashSet(); this.subscribedPartitions = new HashSet(); this.assignedPartitions = new HashSet(); @@ -58,6 +66,7 @@ public class SubscriptionState { this.committed = new HashMap(); this.needsPartitionAssignment = false; this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up + this.resetPartitions = new HashMap(); } public void subscribe(String topic) { @@ -102,12 +111,14 @@ public class SubscriptionState { this.committed.remove(tp); this.fetched.remove(tp); this.consumed.remove(tp); + this.resetPartitions.remove(tp); } public void clearAssignment() { this.assignedPartitions.clear(); this.committed.clear(); this.fetched.clear(); + this.consumed.clear(); this.needsPartitionAssignment = !subscribedTopics().isEmpty(); } @@ -145,6 +156,7 @@ public class SubscriptionState { public void seek(TopicPartition tp, long offset) { fetched(tp, offset); consumed(tp, offset); + resetPartitions.remove(tp); } public Set assignedPartitions() { @@ -169,6 +181,28 @@ public class SubscriptionState { return this.consumed; } + public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) { + this.resetPartitions.put(partition, offsetResetStrategy); + this.fetched.remove(partition); + this.consumed.remove(partition); + } + + public void needOffsetReset(TopicPartition partition) { + needOffsetReset(partition, offsetResetStrategy); + } + + public boolean isOffsetResetNeeded(TopicPartition partition) { + return resetPartitions.containsKey(partition); + } + + public boolean isOffsetResetNeeded() { + return !resetPartitions.isEmpty(); + } + + public OffsetResetStrategy resetStrategy(TopicPartition partition) { + return resetPartitions.get(partition); + } + public boolean hasAllFetchPositions() { return this.fetched.size() >= this.assignedPartitions.size(); } @@ -192,4 +226,5 @@ public class SubscriptionState { this.needsPartitionAssignment = false; } + } \ No newline at end of file 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 f73eedb..af9993c 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -182,6 +182,21 @@ public class Utils { } /** + * Get the minimum of some long values. + * @param first Used to ensure at least one value + * @param rest The rest of longs to compare + * @return The minimum of all passed argument. + */ + public static long min(long first, long ... rest) { + long min = first; + for (int i = 0; i < rest.length; i++) { + if (rest[i] < min) + min = rest[i]; + } + return min; + } + + /** * Get the length for UTF8-encoding a string without encoding it first * * @param s The string to calculate the length for 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 677edd3..26b6b40 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 @@ -25,7 +25,7 @@ import org.junit.Test; public class MockConsumerTest { - private MockConsumer consumer = new MockConsumer(); + private MockConsumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST); @Test public void testSimpleMock() { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java index 1454ab7..613b192 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java @@ -17,10 +17,11 @@ package org.apache.kafka.clients.consumer.internals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -49,24 +50,20 @@ public class CoordinatorTest { private String topicName = "test"; private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); - private long retryBackoffMs = 0L; private int sessionTimeoutMs = 10; private String rebalanceStrategy = "not-matter"; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); - private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); - private SubscriptionState subscriptions = new SubscriptionState(); + private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); private Coordinator coordinator = new Coordinator(client, groupId, - retryBackoffMs, sessionTimeoutMs, rebalanceStrategy, - metadata, subscriptions, metrics, "consumer" + groupId, @@ -75,13 +72,14 @@ public class CoordinatorTest { @Before public void setup() { - metadata.update(cluster, time.milliseconds()); client.setNode(node); } @Test public void testNormalHeartbeat() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // normal heartbeat time.sleep(sessionTimeoutMs); @@ -94,6 +92,8 @@ public class CoordinatorTest { @Test public void testCoordinatorNotAvailable() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // consumer_coordinator_not_available will mark coordinator as unknown time.sleep(sessionTimeoutMs); @@ -108,6 +108,8 @@ public class CoordinatorTest { @Test public void testNotCoordinator() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // not_coordinator will mark coordinator as unknown time.sleep(sessionTimeoutMs); @@ -122,6 +124,8 @@ public class CoordinatorTest { @Test public void testIllegalGeneration() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // illegal_generation will cause re-partition subscriptions.subscribe(topicName); @@ -139,6 +143,8 @@ public class CoordinatorTest { @Test public void testCoordinatorDisconnect() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // coordinator disconnect will mark coordinator as unknown time.sleep(sessionTimeoutMs); @@ -152,39 +158,67 @@ public class CoordinatorTest { @Test public void testNormalJoinGroup() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // normal join group client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - assertEquals(Collections.singletonList(tp), - coordinator.assignPartitions(Collections.singletonList(topicName), time.milliseconds())); - assertEquals(0, client.inFlightRequestCount()); + coordinator.assignPartitions(time.milliseconds()); + client.poll(0, time.milliseconds()); + + assertFalse(subscriptions.partitionAssignmentNeeded()); + assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); } @Test public void testReJoinGroup() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + assertTrue(subscriptions.partitionAssignmentNeeded()); // diconnected from original coordinator will cause re-discover and join again client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()), true); + coordinator.assignPartitions(time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(subscriptions.partitionAssignmentNeeded()); + + // rediscover the coordinator client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + + // try assigning partitions again client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); - assertEquals(Collections.singletonList(tp), - coordinator.assignPartitions(Collections.singletonList(topicName), time.milliseconds())); - assertEquals(0, client.inFlightRequestCount()); + coordinator.assignPartitions(time.milliseconds()); + client.poll(0, time.milliseconds()); + assertFalse(subscriptions.partitionAssignmentNeeded()); + assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); } @Test public void testCommitOffsetNormal() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); - // sync commit + // With success flag client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), true, time.milliseconds()); + RequestFuture result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + assertEquals(1, client.poll(0, time.milliseconds()).size()); + assertTrue(result.isDone()); + assertTrue(result.succeeded()); - // async commit - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), false, time.milliseconds()); + // Without success flag + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); client.respond(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); assertEquals(1, client.poll(0, time.milliseconds()).size()); } @@ -192,34 +226,55 @@ public class CoordinatorTest { @Test public void testCommitOffsetError() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // async commit with coordinator not available client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), false, time.milliseconds()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(coordinator.coordinatorUnknown()); // resume client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // async commit with not coordinator client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), false, time.milliseconds()); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); assertEquals(1, client.poll(0, time.milliseconds()).size()); assertTrue(coordinator.coordinatorUnknown()); // resume client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // sync commit with not_coordinator client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), true, time.milliseconds()); + RequestFuture result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + assertEquals(1, client.poll(0, time.milliseconds()).size()); + assertTrue(result.isDone()); + assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); // sync commit with coordinator disconnected client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); - client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsets(Collections.singletonMap(tp, 100L), true, time.milliseconds()); + result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + + assertEquals(0, client.poll(0, time.milliseconds()).size()); + assertTrue(result.isDone()); + assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + + result = coordinator.commitOffsets(Collections.singletonMap(tp, 100L), time.milliseconds()); + assertEquals(1, client.poll(0, time.milliseconds()).size()); + assertTrue(result.isDone()); + assertTrue(result.succeeded()); } @@ -227,33 +282,70 @@ public class CoordinatorTest { public void testFetchOffset() { client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); // normal fetch client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - assertEquals(100L, (long) coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).get(tp)); + RequestFuture> result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertEquals(100L, (long) result.value().get(tp)); // fetch with loading in progress client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L)); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - assertEquals(100L, (long) coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).get(tp)); + + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.failed()); + assertEquals(RequestFuture.RetryAction.BACKOFF, result.retryAction()); + + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertEquals(100L, (long) result.value().get(tp)); // fetch with not coordinator client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code(), "", 100L)); client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); - assertEquals(100L, (long) coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).get(tp)); + + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.failed()); + assertEquals(RequestFuture.RetryAction.FIND_COORDINATOR, result.retryAction()); + + coordinator.discoverConsumerCoordinator(); + client.poll(0, time.milliseconds()); + + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertEquals(100L, (long) result.value().get(tp)); // fetch with no fetchable offsets client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - assertEquals(0, coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).size()); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.value().isEmpty()); // fetch with offset topic unknown client.prepareResponse(offsetFetchResponse(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), "", 100L)); - assertEquals(0, coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).size()); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.value().isEmpty()); // fetch with offset -1 client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); - assertEquals(0, coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()).size()); + result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); + client.poll(0, time.milliseconds()); + assertTrue(result.isDone()); + assertTrue(result.value().isEmpty()); } private Struct consumerMetadataResponse(Node node, short error) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 4195410..405efdc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -16,11 +16,10 @@ */ package org.apache.kafka.clients.consumer.internals; -import static org.junit.Assert.assertEquals; - import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -30,10 +29,11 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.FetchResponse; -import org.apache.kafka.common.requests.ListOffsetResponse; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; import java.nio.ByteBuffer; import java.util.Collections; @@ -41,37 +41,33 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import org.junit.Before; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class FetcherTest { private String topicName = "test"; private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); - private long retryBackoffMs = 0L; private int minBytes = 1; private int maxWaitMs = 0; private int fetchSize = 1000; - private String offsetReset = "EARLIEST"; private MockTime time = new MockTime(); private MockClient client = new MockClient(time); private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); - private SubscriptionState subscriptions = new SubscriptionState(); + private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); private Metrics metrics = new Metrics(time); private Map metricTags = new LinkedHashMap(); private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); private Fetcher fetcher = new Fetcher(client, - retryBackoffMs, minBytes, maxWaitMs, fetchSize, true, // check crc - offsetReset, new ByteArrayDeserializer(), new ByteArrayDeserializer(), metadata, @@ -140,11 +136,11 @@ public class FetcherTest { subscriptions.fetched(tp, 5); fetcher.initFetches(cluster, time.milliseconds()); client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.prepareResponse(listOffsetResponse(Collections.singletonList(0L), Errors.NONE.code())); client.poll(0, time.milliseconds()); + assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); - assertEquals(0L, (long) subscriptions.fetched(tp)); - assertEquals(0L, (long) subscriptions.consumed(tp)); + assertEquals(null, subscriptions.fetched(tp)); + assertEquals(null, subscriptions.consumed(tp)); } @Test @@ -157,11 +153,11 @@ public class FetcherTest { // fetch with out of range fetcher.initFetches(cluster, time.milliseconds()); client.respond(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); - client.prepareResponse(listOffsetResponse(Collections.singletonList(0L), Errors.NONE.code())); client.poll(0, time.milliseconds()); + assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertEquals(0, fetcher.fetchedRecords().size()); - assertEquals(0L, (long) subscriptions.fetched(tp)); - assertEquals(0L, (long) subscriptions.consumed(tp)); + assertEquals(null, subscriptions.fetched(tp)); + assertEquals(null, subscriptions.consumed(tp)); } private Struct fetchResponse(ByteBuffer buffer, short error, long hw) { @@ -169,9 +165,5 @@ public class FetcherTest { return response.toStruct(); } - private Struct listOffsetResponse(List offsets, short error) { - ListOffsetResponse response = new ListOffsetResponse(Collections.singletonMap(tp, new ListOffsetResponse.PartitionData(error, offsets))); - return response.toStruct(); - } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java index ecc78ce..ee1ede0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.MockTime; import org.junit.Test; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -42,4 +43,12 @@ public class HeartbeatTest { time.sleep(timeout / (2 * Heartbeat.HEARTBEATS_PER_SESSION_INTERVAL)); assertFalse(heartbeat.shouldHeartbeat(time.milliseconds())); } + + @Test + public void testTimeToNextHeartbeat() { + heartbeat.sentHeartbeat(0); + assertEquals(100, heartbeat.timeToNextHeartbeat(0)); + assertEquals(0, heartbeat.timeToNextHeartbeat(100)); + assertEquals(0, heartbeat.timeToNextHeartbeat(200)); + } } 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 e000cf8..319751c 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 @@ -22,12 +22,13 @@ import static java.util.Arrays.asList; import java.util.Collections; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; import org.junit.Test; public class SubscriptionStateTest { - private final SubscriptionState state = new SubscriptionState(); + private final SubscriptionState state = new SubscriptionState(OffsetResetStrategy.EARLIEST); private final TopicPartition tp0 = new TopicPartition("test", 0); private final TopicPartition tp1 = new TopicPartition("test", 1); @@ -43,7 +44,21 @@ public class SubscriptionStateTest { assertTrue(state.assignedPartitions().isEmpty()); assertAllPositions(tp0, null); } - + + @Test + public void partitionReset() { + state.subscribe(tp0); + state.seek(tp0, 5); + assertEquals(5L, (long) state.fetched(tp0)); + assertEquals(5L, (long) state.consumed(tp0)); + state.needOffsetReset(tp0); + assertTrue(state.isOffsetResetNeeded()); + assertTrue(state.isOffsetResetNeeded(tp0)); + assertEquals(null, state.fetched(tp0)); + assertEquals(null, state.consumed(tp0)); + } + + @Test public void topicSubscription() { state.subscribe("test"); assertEquals(1, state.subscribedTopics().size()); 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 2ebe3c2..e7951d8 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -100,4 +100,12 @@ public class UtilsTest { buffer = ByteBuffer.wrap(myvar).asReadOnlyBuffer(); this.subTest(buffer); } + + @Test + public void testMin() { + assertEquals(1, Utils.min(1)); + assertEquals(1, Utils.min(1, 2, 3)); + assertEquals(1, Utils.min(2, 1, 3)); + assertEquals(1, Utils.min(2, 3, 1)); + } } \ No newline at end of file -- 1.7.12.4 From 6d4991e312548bc81e0f6b790a370b6165a92df9 Mon Sep 17 00:00:00 2001 From: Jeff Maxwell Date: Tue, 23 Jun 2015 10:52:21 -0700 Subject: [PATCH 40/59] KAFKA-2294; javadoc compile error due to illegal

      , build failing (jdk 8); patched by Jeff Maxwell; reviewed by Jakob Homan --- .../src/main/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 5a37580..5671a3f 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 @@ -522,7 +522,8 @@ public class KafkaProducer implements Producer { * If close() is called from {@link Callback}, a warning message will be logged and close(0, TimeUnit.MILLISECONDS) * will be called instead. We do this because the sender thread would otherwise try to join itself and * block forever. - *

      + *

      + * * @throws InterruptException If the thread is interrupted while blocked */ @Override -- 1.7.12.4 From 9ff5b27bc572850863f58a3767da3c72aa2a9831 Mon Sep 17 00:00:00 2001 From: Tao Xiao Date: Mon, 29 Jun 2015 18:47:47 -0700 Subject: [PATCH 41/59] KAFKA-2281: avoid unnecessary value copying if logAsString is false; reviewed by Guozhang Wang --- .../clients/producer/internals/ErrorLoggingCallback.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java index 678d1c6..747e29f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java @@ -23,12 +23,18 @@ public class ErrorLoggingCallback implements Callback { private String topic; private byte[] key; private byte[] value; + private int valueLength; private boolean logAsString; public ErrorLoggingCallback(String topic, byte[] key, byte[] value, boolean logAsString) { this.topic = topic; this.key = key; - this.value = value; + + if (logAsString) { + this.value = value; + } + + this.valueLength = value == null ? -1 : value.length; this.logAsString = logAsString; } @@ -36,10 +42,10 @@ public class ErrorLoggingCallback implements Callback { if (e != null) { String keyString = (key == null) ? "null" : logAsString ? new String(key) : key.length + " bytes"; - String valueString = (value == null) ? "null" : - logAsString ? new String(value) : value.length + " bytes"; + String valueString = (valueLength == -1) ? "null" : + logAsString ? new String(value) : valueLength + " bytes"; log.error("Error when sending message to topic {} with key: {}, value: {} with error: {}", - topic, keyString, valueString, e.getMessage()); + topic, keyString, valueString, e.getMessage()); } } } -- 1.7.12.4 From 14e0ce0a47fb7f6ae6dab085b2ea9d5a1f644433 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 1 Jul 2015 15:28:11 -0700 Subject: [PATCH 42/59] KAFKA-2168: minor follow-up patch; reviewed by Guozhang Wang --- .../apache/kafka/clients/consumer/KafkaConsumer.java | 17 ++++++++++------- .../kafka/clients/consumer/internals/Coordinator.java | 10 +++++----- .../integration/kafka/api/ConsumerBounceTest.scala | 14 ++++++-------- 3 files changed, 21 insertions(+), 20 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 9be8fbc..1f0e515 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 @@ -42,6 +42,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.ConcurrentModificationException; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -332,6 +333,7 @@ import static org.apache.kafka.common.utils.Utils.min; * } * } * + * // Shutdown hook which can be called from a separate thread * public void shutdown() { * closed.set(true); * consumer.wakeup(); @@ -417,7 +419,7 @@ public class KafkaConsumer implements Consumer { // and is used to prevent multi-threaded access private final AtomicReference currentThread = new AtomicReference(); // refcount is used to allow reentrant access by the thread who has acquired currentThread - private int refcount = 0; // reference count for reentrant access + private final AtomicInteger refcount = new AtomicInteger(0); // TODO: This timeout controls how long we should wait before retrying a request. We should be able // to leverage the work of KAFKA-2120 to get this value from configuration. @@ -795,7 +797,7 @@ public class KafkaConsumer implements Consumer { * 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 non-blocking commit will attempt to commit offsets asynchronously. 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. * @@ -832,7 +834,9 @@ public class KafkaConsumer implements Consumer { public void commit(CommitType commitType) { acquire(); try { - commit(this.subscriptions.allConsumed(), commitType); + // Need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance) + Map allConsumed = new HashMap(this.subscriptions.allConsumed()); + commit(allConsumed, commitType); } finally { release(); } @@ -978,10 +982,9 @@ public class KafkaConsumer implements Consumer { @Override public void close() { - if (closed) return; - acquire(); try { + if (closed) return; close(false); } finally { release(); @@ -1355,14 +1358,14 @@ public class KafkaConsumer implements Consumer { Long threadId = Thread.currentThread().getId(); if (!threadId.equals(currentThread.get()) && !currentThread.compareAndSet(null, threadId)) throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access"); - refcount++; + refcount.incrementAndGet(); } /** * Release the light lock protecting the consumer from multi-threaded access. */ private void release() { - if (--refcount == 0) + if (refcount.decrementAndGet() == 0) currentThread.set(null); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index 6c26667..68b4cb1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -217,7 +217,7 @@ public final class Coordinator { OffsetCommitRequest.DEFAULT_RETENTION_TIME, offsetData); - RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets, future); + RequestCompletionHandler handler = new OffsetCommitCompletionHandler(offsets, future); sendCoordinator(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); } @@ -261,14 +261,14 @@ public final class Coordinator { RequestCompletionHandler completionHandler = new RequestCompletionHandler() { @Override public void onComplete(ClientResponse resp) { - handleOffsetResponse(resp, future); + handleOffsetFetchResponse(resp, future); } }; sendCoordinator(ApiKeys.OFFSET_FETCH, request.toStruct(), completionHandler, now); return future; } - private void handleOffsetResponse(ClientResponse resp, RequestFuture> future) { + private void handleOffsetFetchResponse(ClientResponse resp, RequestFuture> future) { if (resp.wasDisconnected()) { handleCoordinatorDisconnect(resp); future.retryWithNewCoordinator(); @@ -471,12 +471,12 @@ public final class Coordinator { } } - private class CommitOffsetCompletionHandler implements RequestCompletionHandler { + private class OffsetCommitCompletionHandler implements RequestCompletionHandler { private final Map offsets; private final RequestFuture future; - public CommitOffsetCompletionHandler(Map offsets, RequestFuture future) { + public OffsetCommitCompletionHandler(Map offsets, RequestFuture future) { this.offsets = offsets; this.future = future; } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index f56096b..b0750fa 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -14,14 +14,10 @@ package kafka.api import kafka.server.KafkaConfig -import org.apache.kafka.clients.producer.ProducerConfig -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.kafka.clients.consumer.CommitType +import kafka.utils.{Logging, ShutdownableThread, TestUtils} +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition - -import kafka.utils.{ShutdownableThread, TestUtils, Logging} - import org.junit.Assert._ import scala.collection.JavaConversions._ @@ -85,9 +81,11 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { assertEquals(consumed.toLong, record.offset()) consumed += 1 } + consumer.commit(CommitType.SYNC) + assertEquals(consumer.position(tp), consumer.committed(tp)) - if (consumed == numRecords) { + if (consumer.position(tp) == numRecords) { consumer.seekToBeginning() consumed = 0 } -- 1.7.12.4 From 3f8480ccfb011eb43da774737597c597f703e11b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 2 Jul 2015 11:41:51 -0700 Subject: [PATCH 43/59] KAFKA-1740: merge offset manager into consumer coordinator; reviewed by Onur Karaman and Jason Gustafson --- .../clients/consumer/internals/Coordinator.java | 27 +++- .../org/apache/kafka/common/protocol/Errors.java | 6 +- .../common/requests/OffsetCommitResponse.java | 8 +- .../kafka/common/requests/OffsetFetchRequest.java | 3 - .../kafka/common/requests/OffsetFetchResponse.java | 5 +- .../consumer/internals/CoordinatorTest.java | 7 - core/src/main/scala/kafka/admin/TopicCommand.scala | 4 +- core/src/main/scala/kafka/cluster/Partition.scala | 16 +- .../kafka/common/OffsetMetadataAndError.scala | 14 +- core/src/main/scala/kafka/common/Topic.scala | 4 +- .../kafka/coordinator/ConsumerCoordinator.scala | 170 +++++++++++++++++++-- .../kafka/coordinator/CoordinatorMetadata.scala | 4 +- core/src/main/scala/kafka/server/KafkaApis.scala | 69 +++++---- core/src/main/scala/kafka/server/KafkaServer.scala | 26 +--- .../main/scala/kafka/server/OffsetManager.scala | 52 +++---- .../main/scala/kafka/server/ReplicaManager.scala | 95 +++++++----- .../scala/integration/kafka/api/ConsumerTest.scala | 7 +- .../kafka/api/IntegrationTestHarness.scala | 9 +- .../scala/unit/kafka/admin/TopicCommandTest.scala | 8 +- .../unit/kafka/consumer/TopicFilterTest.scala | 9 +- .../ConsumerCoordinatorResponseTest.scala | 9 +- .../coordinator/CoordinatorMetadataTest.scala | 2 +- .../scala/unit/kafka/server/OffsetCommitTest.scala | 10 +- 23 files changed, 357 insertions(+), 207 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index 68b4cb1..c1c8172 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -290,9 +290,10 @@ public final class Coordinator { // re-discover the coordinator and retry coordinatorDead(); future.retryWithNewCoordinator(); - } else if (data.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { - // just ignore this partition - log.debug("Unknown topic or partition for " + tp); + } else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code() + || data.errorCode == Errors.ILLEGAL_GENERATION.code()) { + // need to re-join group + subscriptions.needReassignment(); } else { future.raise(new KafkaException("Unexpected error in fetch offset response: " + Errors.forCode(data.errorCode).exception().getMessage())); @@ -499,13 +500,23 @@ public final class Coordinator { || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { coordinatorDead(); future.retryWithNewCoordinator(); - } else { + } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code() + || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) { // do not need to throw the exception but just log the error - future.retryAfterBackoff(); log.error("Error committing partition {} at offset {}: {}", - tp, - offset, - Errors.forCode(errorCode).exception().getMessage()); + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code() + || errorCode == Errors.ILLEGAL_GENERATION.code()) { + // need to re-join group + subscriptions.needReassignment(); + } else { + // re-throw the exception as these should not happen + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); } } 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 5b898c8..4c0ecc3 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 @@ -77,7 +77,11 @@ public enum Errors { UNKNOWN_CONSUMER_ID(25, new ApiException("The coordinator is not aware of this consumer.")), INVALID_SESSION_TIMEOUT(26, - new ApiException("The session timeout is not within an acceptable range.")); + new ApiException("The session timeout is not within an acceptable range.")), + COMMITTING_PARTITIONS_NOT_ASSIGNED(27, + new ApiException("Some of the committing partitions are not assigned the committer")), + INVALID_COMMIT_OFFSET_SIZE(28, + new ApiException("The committing offset data size is not valid")); private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); 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 70844d6..a163333 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 @@ -41,7 +41,13 @@ public class OffsetCommitResponse extends AbstractRequestResponse { /** * Possible error code: * - * TODO + * OFFSET_METADATA_TOO_LARGE (12) + * CONSUMER_COORDINATOR_NOT_AVAILABLE (15) + * NOT_COORDINATOR_FOR_CONSUMER (16) + * ILLEGAL_GENERATION (22) + * UNKNOWN_CONSUMER_ID (25) + * COMMITTING_PARTITIONS_NOT_ASSIGNED (27) + * INVALID_COMMIT_OFFSET_SIZE (28) */ private final Map responseData; 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 b5e8a0f..6ee7597 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 @@ -42,9 +42,6 @@ public class OffsetFetchRequest extends AbstractRequest { // partition level field names private static final String PARTITION_KEY_NAME = "partition"; - public static final int DEFAULT_GENERATION_ID = -1; - public static final String DEFAULT_CONSUMER_ID = ""; - private final String groupId; private final List partitions; 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 512a0ef..3dc8521 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 @@ -47,10 +47,11 @@ public class OffsetFetchResponse extends AbstractRequestResponse { /** * Possible error code: * - * UNKNOWN_TOPIC_OR_PARTITION (3) + * UNKNOWN_TOPIC_OR_PARTITION (3) <- only for request v0 * OFFSET_LOAD_IN_PROGRESS (14) * NOT_COORDINATOR_FOR_CONSUMER (16) - * NO_OFFSETS_FETCHABLE (23) + * ILLEGAL_GENERATION (22) + * UNKNOWN_CONSUMER_ID (25) */ private final Map responseData; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java index 613b192..d085fe5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java @@ -333,13 +333,6 @@ public class CoordinatorTest { assertTrue(result.isDone()); assertTrue(result.value().isEmpty()); - // fetch with offset topic unknown - client.prepareResponse(offsetFetchResponse(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), "", 100L)); - result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); - client.poll(0, time.milliseconds()); - assertTrue(result.isDone()); - assertTrue(result.value().isEmpty()); - // fetch with offset -1 client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); result = coordinator.fetchOffsets(Collections.singleton(tp), time.milliseconds()); diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index dacbdd0..a2ecb96 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -27,8 +27,8 @@ import scala.collection._ import scala.collection.JavaConversions._ import kafka.log.LogConfig import kafka.consumer.Whitelist -import kafka.server.OffsetManager import org.apache.kafka.common.utils.Utils +import kafka.coordinator.ConsumerCoordinator object TopicCommand { @@ -111,7 +111,7 @@ object TopicCommand { println("Updated config for topic \"%s\".".format(topic)) } if(opts.options.has(opts.partitionsOpt)) { - if (topic == OffsetManager.OffsetsTopicName) { + if (topic == ConsumerCoordinator.OffsetsTopicName) { throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.") } println("WARNING: If partitions are increased for a topic that has a key, the partition " + diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 0990938..2649090 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -22,7 +22,7 @@ import kafka.utils.CoreUtils.{inReadLock,inWriteLock} import kafka.admin.AdminUtils import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig -import kafka.server.{TopicPartitionOperationKey, LogOffsetMetadata, OffsetManager, LogReadResult, ReplicaManager} +import kafka.server.{TopicPartitionOperationKey, LogOffsetMetadata, LogReadResult, ReplicaManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.message.ByteBufferMessageSet @@ -160,8 +160,7 @@ class Partition(val topic: String, * and setting the new leader and ISR */ def makeLeader(controllerId: Int, - partitionStateInfo: PartitionStateInfo, correlationId: Int, - offsetManager: OffsetManager): Boolean = { + partitionStateInfo: PartitionStateInfo, correlationId: Int): Boolean = { inWriteLock(leaderIsrUpdateLock) { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch @@ -186,8 +185,6 @@ class Partition(val topic: String, if (r.brokerId != localBrokerId) r.updateLogReadResult(LogReadResult.UnknownLogReadResult)) // we may need to increment high watermark since ISR could be down to 1 maybeIncrementLeaderHW(newLeaderReplica) - if (topic == OffsetManager.OffsetsTopicName) - offsetManager.loadOffsetsFromLog(partitionId) true } } @@ -198,7 +195,7 @@ class Partition(val topic: String, */ def makeFollower(controllerId: Int, partitionStateInfo: PartitionStateInfo, - correlationId: Int, offsetManager: OffsetManager): Boolean = { + correlationId: Int): Boolean = { inWriteLock(leaderIsrUpdateLock) { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch @@ -215,13 +212,6 @@ class Partition(val topic: String, leaderEpoch = leaderAndIsr.leaderEpoch zkVersion = leaderAndIsr.zkVersion - leaderReplicaIdOpt.foreach { leaderReplica => - if (topic == OffsetManager.OffsetsTopicName && - /* if we are making a leader->follower transition */ - leaderReplica == localBrokerId) - offsetManager.removeOffsetsFromCacheForPartition(partitionId) - } - if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) { false } diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 6b4242c..deb48b1 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -17,6 +17,8 @@ package kafka.common +import org.apache.kafka.common.protocol.Errors + case class OffsetMetadata(offset: Long, metadata: String = OffsetMetadata.NoMetadata) { override def toString = "OffsetMetadata[%d,%s]" .format(offset, @@ -51,7 +53,7 @@ object OffsetAndMetadata { def apply(offset: Long) = new OffsetAndMetadata(OffsetMetadata(offset, OffsetMetadata.NoMetadata)) } -case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short = ErrorMapping.NoError) { +case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short = Errors.NONE.code) { def offset = offsetMetadata.offset def metadata = offsetMetadata.metadata @@ -60,10 +62,12 @@ case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short = } object OffsetMetadataAndError { - val NoOffset = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.NoError) - val OffsetsLoading = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.OffsetsLoadInProgressCode) - val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.UnknownTopicOrPartitionCode) - val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.NotCoordinatorForConsumerCode) + val NoOffset = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.NONE.code) + val OffsetsLoading = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.OFFSET_LOAD_IN_PROGRESS.code) + val UnknownConsumer = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.UNKNOWN_CONSUMER_ID.code) + val NotCoordinatorForGroup = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.NOT_COORDINATOR_FOR_CONSUMER.code) + val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + val IllegalGroupGenerationId = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.ILLEGAL_GENERATION.code) def apply(offset: Long) = new OffsetMetadataAndError(OffsetMetadata(offset, OffsetMetadata.NoMetadata), ErrorMapping.NoError) diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index ad75978..32595d6 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -18,7 +18,7 @@ package kafka.common import util.matching.Regex -import kafka.server.OffsetManager +import kafka.coordinator.ConsumerCoordinator object Topic { @@ -26,7 +26,7 @@ object Topic { private val maxNameLength = 255 private val rgx = new Regex(legalChars + "+") - val InternalTopics = Set(OffsetManager.OffsetsTopicName) + val InternalTopics = Set(ConsumerCoordinator.OffsetsTopicName) def validate(topic: String) { if (topic.length <= 0) diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index a385adb..476973b 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -16,7 +16,9 @@ */ package kafka.coordinator -import kafka.common.TopicAndPartition +import kafka.common.{OffsetMetadataAndError, OffsetAndMetadata, TopicAndPartition} +import kafka.message.UncompressedCodec +import kafka.log.LogConfig import kafka.server._ import kafka.utils._ import org.apache.kafka.common.protocol.Errors @@ -24,7 +26,11 @@ import org.apache.kafka.common.requests.JoinGroupRequest import org.I0Itec.zkclient.ZkClient import java.util.concurrent.atomic.AtomicBoolean +import java.util.Properties +import scala.collection.{Map, Seq, immutable} +case class GroupManagerConfig(consumerMinSessionTimeoutMs: Int, + consumerMaxSessionTimeoutMs: Int) /** * ConsumerCoordinator handles consumer group and consumer offset management. @@ -33,11 +39,13 @@ import java.util.concurrent.atomic.AtomicBoolean * consumer groups. Consumer groups are assigned to coordinators based on their * group names. */ -class ConsumerCoordinator(val config: KafkaConfig, - val zkClient: ZkClient, - val offsetManager: OffsetManager) extends Logging { +class ConsumerCoordinator(val brokerId: Int, + val groupConfig: GroupManagerConfig, + val offsetConfig: OffsetManagerConfig, + private val offsetManager: OffsetManager, + zkClient: ZkClient) extends Logging { - this.logIdent = "[ConsumerCoordinator " + config.brokerId + "]: " + this.logIdent = "[ConsumerCoordinator " + brokerId + "]: " private val isActive = new AtomicBoolean(false) @@ -45,6 +53,22 @@ class ConsumerCoordinator(val config: KafkaConfig, private var rebalancePurgatory: DelayedOperationPurgatory[DelayedRebalance] = null private var coordinatorMetadata: CoordinatorMetadata = null + def this(brokerId: Int, + groupConfig: GroupManagerConfig, + offsetConfig: OffsetManagerConfig, + replicaManager: ReplicaManager, + zkClient: ZkClient, + scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig, + new OffsetManager(offsetConfig, replicaManager, zkClient, scheduler), zkClient) + + def offsetsTopicConfigs: Properties = { + val props = new Properties + props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + props.put(LogConfig.SegmentBytesProp, offsetConfig.offsetsTopicSegmentBytes.toString) + props.put(LogConfig.CompressionTypeProp, UncompressedCodec.name) + props + } + /** * NOTE: If a group lock and metadataLock are simultaneously needed, * be sure to acquire the group lock before metadataLock to prevent deadlock @@ -55,9 +79,9 @@ class ConsumerCoordinator(val config: KafkaConfig, */ def startup() { info("Starting up.") - heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", config.brokerId) - rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", config.brokerId) - coordinatorMetadata = new CoordinatorMetadata(config, zkClient, maybePrepareRebalance) + heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", brokerId) + rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", brokerId) + coordinatorMetadata = new CoordinatorMetadata(brokerId, zkClient, maybePrepareRebalance) isActive.set(true) info("Startup complete.") } @@ -69,6 +93,7 @@ class ConsumerCoordinator(val config: KafkaConfig, def shutdown() { info("Shutting down.") isActive.set(false) + offsetManager.shutdown() coordinatorMetadata.shutdown() heartbeatPurgatory.shutdown() rebalancePurgatory.shutdown() @@ -87,7 +112,8 @@ class ConsumerCoordinator(val config: KafkaConfig, responseCallback(Set.empty, consumerId, 0, Errors.NOT_COORDINATOR_FOR_CONSUMER.code) } else if (!PartitionAssignor.strategies.contains(partitionAssignmentStrategy)) { responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code) - } else if (sessionTimeoutMs < config.consumerMinSessionTimeoutMs || sessionTimeoutMs > config.consumerMaxSessionTimeoutMs) { + } else if (sessionTimeoutMs < groupConfig.consumerMinSessionTimeoutMs || + sessionTimeoutMs > groupConfig.consumerMaxSessionTimeoutMs) { responseCallback(Set.empty, consumerId, 0, Errors.INVALID_SESSION_TIMEOUT.code) } else { // only try to create the group if the group is not unknown AND @@ -196,6 +222,75 @@ class ConsumerCoordinator(val config: KafkaConfig, } } + def handleCommitOffsets(groupId: String, + consumerId: String, + generationId: Int, + offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], + responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { + if (!isActive.get) { + responseCallback(offsetMetadata.mapValues(_ => Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code)) + } else if (!isCoordinatorForGroup(groupId)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.NOT_COORDINATOR_FOR_CONSUMER.code)) + } else { + val group = coordinatorMetadata.getGroup(groupId) + if (group == null) { + // if the group does not exist, it means this group is not relying + // on Kafka for partition management, and hence never send join-group + // request to the coordinator before; in this case blindly commit the offsets + offsetManager.storeOffsets(groupId, consumerId, generationId, offsetMetadata, responseCallback) + } else { + group synchronized { + if (group.is(Dead)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_CONSUMER_ID.code)) + } else if (!group.has(consumerId)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_CONSUMER_ID.code)) + } else if (generationId != group.generationId) { + responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code)) + } else if (!offsetMetadata.keySet.subsetOf(group.get(consumerId).assignedTopicPartitions)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code)) + } else { + offsetManager.storeOffsets(groupId, consumerId, generationId, offsetMetadata, responseCallback) + } + } + } + } + } + + def handleFetchOffsets(groupId: String, + partitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = { + if (!isActive.get) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap + } else if (!isCoordinatorForGroup(groupId)) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap + } else { + val group = coordinatorMetadata.getGroup(groupId) + if (group == null) { + // if the group does not exist, it means this group is not relying + // on Kafka for partition management, and hence never send join-group + // request to the coordinator before; in this case blindly fetch the offsets + offsetManager.getOffsets(groupId, partitions) + } else { + group synchronized { + if (group.is(Dead)) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownConsumer)}.toMap + } else { + offsetManager.getOffsets(groupId, partitions) + } + } + } + } + } + + def handleGroupImmigration(offsetTopicPartitionId: Int) = { + // TODO we may need to add more logic in KAFKA-2017 + offsetManager.loadOffsetsFromLog(offsetTopicPartitionId) + } + + def handleGroupEmigration(offsetTopicPartitionId: Int) = { + // TODO we may need to add more logic in KAFKA-2017 + offsetManager.removeOffsetsFromCacheForPartition(offsetTopicPartitionId) + } + /** * Complete existing DelayedHeartbeats for the given consumer and schedule the next one */ @@ -246,8 +341,7 @@ class ConsumerCoordinator(val config: KafkaConfig, private def prepareRebalance(group: ConsumerGroupMetadata) { group.transitionTo(PreparingRebalance) - group.generationId += 1 - info("Preparing to rebalance group %s generation %s".format(group.groupId, group.generationId)) + info("Preparing to rebalance group %s with old generation %s".format(group.groupId, group.generationId)) val rebalanceTimeout = group.rebalanceTimeout val delayedRebalance = new DelayedRebalance(this, group, rebalanceTimeout) @@ -259,7 +353,9 @@ class ConsumerCoordinator(val config: KafkaConfig, assert(group.notYetRejoinedConsumers == List.empty[ConsumerMetadata]) group.transitionTo(Rebalancing) - info("Rebalancing group %s generation %s".format(group.groupId, group.generationId)) + group.generationId += 1 + + info("Rebalancing group %s with new generation %s".format(group.groupId, group.generationId)) val assignedPartitionsPerConsumer = reassignPartitions(group) trace("Rebalance for group %s generation %s has assigned partitions: %s" @@ -275,8 +371,6 @@ class ConsumerCoordinator(val config: KafkaConfig, maybePrepareRebalance(group) } - private def isCoordinatorForGroup(groupId: String) = offsetManager.leaderIsLocal(offsetManager.partitionFor(groupId)) - private def reassignPartitions(group: ConsumerGroupMetadata) = { val assignor = PartitionAssignor.createInstance(group.partitionAssignmentStrategy) val topicsPerConsumer = group.topicsPerConsumer @@ -345,8 +439,54 @@ class ConsumerCoordinator(val config: KafkaConfig, } } - def onCompleteHeartbeat() {} + def onCompleteHeartbeat() { + // TODO: add metrics for complete heartbeats + } + + def partitionFor(group: String): Int = offsetManager.partitionFor(group) private def shouldKeepConsumerAlive(consumer: ConsumerMetadata, heartbeatDeadline: Long) = consumer.awaitingRebalanceCallback != null || consumer.latestHeartbeat + consumer.sessionTimeoutMs > heartbeatDeadline + + private def isCoordinatorForGroup(groupId: String) = offsetManager.leaderIsLocal(offsetManager.partitionFor(groupId)) +} + +object ConsumerCoordinator { + + val OffsetsTopicName = "__consumer_offsets" + + def create(config: KafkaConfig, + zkClient: ZkClient, + replicaManager: ReplicaManager, + kafkaScheduler: KafkaScheduler): ConsumerCoordinator = { + val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + loadBufferSize = config.offsetsLoadBufferSize, + offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, + offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions = config.offsetsTopicPartitions, + offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, + offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, + offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) + val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs, + consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs) + + new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkClient, kafkaScheduler) + } + + def create(config: KafkaConfig, + zkClient: ZkClient, + offsetManager: OffsetManager): ConsumerCoordinator = { + val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + loadBufferSize = config.offsetsLoadBufferSize, + offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, + offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions = config.offsetsTopicPartitions, + offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, + offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, + offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) + val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs, + consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs) + + new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, offsetManager, zkClient) + } } diff --git a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala index 0cd5605..2920320 100644 --- a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala @@ -32,7 +32,7 @@ import scala.collection.mutable * It delegates all group logic to the callers. */ @threadsafe -private[coordinator] class CoordinatorMetadata(config: KafkaConfig, +private[coordinator] class CoordinatorMetadata(brokerId: Int, zkClient: ZkClient, maybePrepareRebalance: ConsumerGroupMetadata => Unit) { @@ -179,7 +179,7 @@ private[coordinator] class CoordinatorMetadata(config: KafkaConfig, * Zookeeper listener to handle topic partition changes */ class TopicPartitionChangeListener extends IZkDataListener with Logging { - this.logIdent = "[TopicPartitionChangeListener on Coordinator " + config.brokerId + "]: " + this.logIdent = "[TopicPartitionChangeListener on Coordinator " + brokerId + "]: " override def handleDataChange(dataPath: String, data: Object) { info("Handling data change for path: %s data: %s".format(dataPath, data)) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index ad6f058..18f5b5b 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -37,7 +37,6 @@ 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, @@ -95,8 +94,23 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest] try { - val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest, offsetManager) - val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, response, error) + // call replica manager to handle updating partitions to become leader or follower + val result = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest) + val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, result.responseMap, result.errorCode) + // for each new leader or follower, call coordinator to handle + // consumer group migration + result.updatedLeaders.foreach { case partition => + if (partition.topic == ConsumerCoordinator.OffsetsTopicName) + coordinator.handleGroupImmigration(partition.partitionId) + } + result.updatedFollowers.foreach { case partition => + partition.leaderReplicaIdOpt.foreach { leaderReplica => + if (partition.topic == ConsumerCoordinator.OffsetsTopicName && + leaderReplica == brokerId) + coordinator.handleGroupEmigration(partition.partitionId) + } + } + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, leaderAndIsrResponse))) } catch { case e: KafkaStorageException => @@ -142,6 +156,12 @@ class KafkaApis(val requestChannel: RequestChannel, def handleOffsetCommitRequest(request: RequestChannel.Request) { val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] + // filter non-exist topics + val invalidRequestsInfo = offsetCommitRequest.requestInfo.filter { case (topicAndPartition, offsetMetadata) => + !metadataCache.contains(topicAndPartition.topic) + } + val filteredRequestInfo = (offsetCommitRequest.requestInfo -- invalidRequestsInfo.keys) + // the callback for sending an offset commit response def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { commitStatus.foreach { case (topicAndPartition, errorCode) => @@ -154,14 +174,14 @@ class KafkaApis(val requestChannel: RequestChannel, topicAndPartition, ErrorMapping.exceptionNameFor(errorCode))) } } - - val response = OffsetCommitResponse(commitStatus, offsetCommitRequest.correlationId) + val combinedCommitStatus = commitStatus ++ invalidRequestsInfo.map(_._1 -> ErrorMapping.UnknownTopicOrPartitionCode) + val response = OffsetCommitResponse(combinedCommitStatus, offsetCommitRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } if (offsetCommitRequest.versionId == 0) { // for version 0 always store offsets to ZK - val responseInfo = offsetCommitRequest.requestInfo.map { + val responseInfo = filteredRequestInfo.map { case (topicAndPartition, metaAndError) => { val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicAndPartition.topic) try { @@ -189,7 +209,7 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetRetention = if (offsetCommitRequest.versionId <= 1 || offsetCommitRequest.retentionMs == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) { - offsetManager.config.offsetsRetentionMs + coordinator.offsetConfig.offsetsRetentionMs } else { offsetCommitRequest.retentionMs } @@ -203,7 +223,7 @@ class KafkaApis(val requestChannel: RequestChannel, val currentTimestamp = SystemTime.milliseconds val defaultExpireTimestamp = offsetRetention + currentTimestamp - val offsetData = offsetCommitRequest.requestInfo.mapValues(offsetAndMetadata => + val offsetData = filteredRequestInfo.mapValues(offsetAndMetadata => offsetAndMetadata.copy( commitTimestamp = currentTimestamp, expireTimestamp = { @@ -215,8 +235,8 @@ class KafkaApis(val requestChannel: RequestChannel, ) ) - // call offset manager to store offsets - offsetManager.storeOffsets( + // call coordinator to handle commit offset + coordinator.handleCommitOffsets( offsetCommitRequest.groupId, offsetCommitRequest.consumerId, offsetCommitRequest.groupGenerationId, @@ -422,9 +442,9 @@ class KafkaApis(val requestChannel: RequestChannel, if (topics.size > 0 && topicResponses.size != topics.size) { val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet val responsesForNonExistentTopics = nonExistentTopics.map { topic => - if (topic == OffsetManager.OffsetsTopicName || config.autoCreateTopicsEnable) { + if (topic == ConsumerCoordinator.OffsetsTopicName || config.autoCreateTopicsEnable) { try { - if (topic == OffsetManager.OffsetsTopicName) { + if (topic == ConsumerCoordinator.OffsetsTopicName) { val aliveBrokers = metadataCache.getAliveBrokers val offsetsTopicReplicationFactor = if (aliveBrokers.length > 0) @@ -433,7 +453,7 @@ class KafkaApis(val requestChannel: RequestChannel, config.offsetsTopicReplicationFactor.toInt AdminUtils.createTopic(zkClient, topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor, - offsetManager.offsetsTopicConfig) + coordinator.offsetsTopicConfigs) info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" .format(topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor)) } @@ -496,26 +516,19 @@ class KafkaApis(val requestChannel: RequestChannel, OffsetFetchResponse(collection.immutable.Map(responseInfo: _*), offsetFetchRequest.correlationId) } else { - // version 1 reads offsets from Kafka - 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 = - if (knownTopicPartitions.size > 0) - offsetManager.getOffsets(offsetFetchRequest.groupId, knownTopicPartitions).toMap - else - Map.empty[TopicAndPartition, OffsetMetadataAndError] - val status = unknownStatus ++ knownStatus + // version 1 reads offsets from Kafka; + val offsets = coordinator.handleFetchOffsets(offsetFetchRequest.groupId, offsetFetchRequest.requestInfo).toMap - OffsetFetchResponse(status, offsetFetchRequest.correlationId) + // Note that we do not need to filter the partitions in the + // metadata cache as the topic partitions will be filtered + // in coordinator's offset manager through the offset cache + OffsetFetchResponse(offsets, offsetFetchRequest.correlationId) } trace("Sending offset fetch response %s for correlation id %d to client %s." .format(response, offsetFetchRequest.correlationId, offsetFetchRequest.clientId)) requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) - } /* @@ -524,10 +537,10 @@ class KafkaApis(val requestChannel: RequestChannel, def handleConsumerMetadataRequest(request: RequestChannel.Request) { val consumerMetadataRequest = request.requestObj.asInstanceOf[ConsumerMetadataRequest] - val partition = offsetManager.partitionFor(consumerMetadataRequest.group) + val partition = coordinator.partitionFor(consumerMetadataRequest.group) // get metadata (and create the topic if necessary) - val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName), request.securityProtocol).head + val offsetsTopicMetadata = getTopicMetadata(Set(ConsumerCoordinator.OffsetsTopicName), request.securityProtocol).head val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 52dc728..18917bc 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -41,7 +41,7 @@ import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBroker import kafka.network.{BlockingChannel, SocketServer} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge -import kafka.coordinator.ConsumerCoordinator +import kafka.coordinator.{GroupManagerConfig, ConsumerCoordinator} /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required @@ -75,8 +75,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var logManager: LogManager = null - var offsetManager: OffsetManager = null - var replicaManager: ReplicaManager = null var topicConfigManager: TopicConfigManager = null @@ -157,19 +155,16 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) replicaManager.startup() - /* start offset manager */ - offsetManager = createOffsetManager() - /* start kafka controller */ kafkaController = new KafkaController(config, zkClient, brokerState) kafkaController.startup() /* start kafka coordinator */ - consumerCoordinator = new ConsumerCoordinator(config, zkClient, offsetManager) + consumerCoordinator = ConsumerCoordinator.create(config, zkClient, replicaManager, kafkaScheduler) consumerCoordinator.startup() /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, + apis = new KafkaApis(socketServer.requestChannel, replicaManager, consumerCoordinator, kafkaController, zkClient, config.brokerId, config, metadataCache) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) @@ -349,8 +344,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg CoreUtils.swallow(socketServer.shutdown()) if(requestHandlerPool != null) CoreUtils.swallow(requestHandlerPool.shutdown()) - if(offsetManager != null) - offsetManager.shutdown() CoreUtils.swallow(kafkaScheduler.shutdown()) if(apis != null) CoreUtils.swallow(apis.close()) @@ -450,19 +443,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg logProps } - private def createOffsetManager(): OffsetManager = { - val offsetManagerConfig = OffsetManagerConfig( - maxMetadataSize = config.offsetMetadataMaxSize, - loadBufferSize = config.offsetsLoadBufferSize, - offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, - offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, - offsetsTopicNumPartitions = config.offsetsTopicPartitions, - offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, - offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, - offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) - new OffsetManager(offsetManagerConfig, replicaManager, zkClient, kafkaScheduler, metadataCache) - } - /** * Generates new brokerId or reads from meta.properties based on following conditions *

        diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 5cca85c..47b6ce9 100755 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -17,6 +17,7 @@ package kafka.server +import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.types.{Struct, Schema, Field} import org.apache.kafka.common.protocol.types.Type.STRING import org.apache.kafka.common.protocol.types.Type.INT32 @@ -25,19 +26,19 @@ import org.apache.kafka.common.utils.Utils import kafka.utils._ import kafka.common._ -import kafka.log.{FileMessageSet, LogConfig} +import kafka.log.FileMessageSet import kafka.message._ import kafka.metrics.KafkaMetricsGroup import kafka.common.TopicAndPartition import kafka.tools.MessageFormatter import kafka.api.ProducerResponseStatus +import kafka.coordinator.ConsumerCoordinator import scala.Some import scala.collection._ import java.io.PrintStream import java.util.concurrent.atomic.AtomicBoolean import java.nio.ByteBuffer -import java.util.Properties import java.util.concurrent.TimeUnit import com.yammer.metrics.core.Gauge @@ -87,8 +88,7 @@ object OffsetManagerConfig { class OffsetManager(val config: OffsetManagerConfig, replicaManager: ReplicaManager, zkClient: ZkClient, - scheduler: Scheduler, - metadataCache: MetadataCache) extends Logging with KafkaMetricsGroup { + scheduler: Scheduler) extends Logging with KafkaMetricsGroup { /* offsets and metadata cache */ private val offsetsCache = new Pool[GroupTopicPartition, OffsetAndMetadata] @@ -143,9 +143,9 @@ class OffsetManager(val config: OffsetManagerConfig, // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say, // if we crash or leaders move) since the new leaders will get rid of expired offsets during their own purge cycles. tombstonesForPartition.flatMap { case (offsetsPartition, tombstones) => - val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + val partitionOpt = replicaManager.getPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) partitionOpt.map { partition => - val appendPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + val appendPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) val messages = tombstones.map(_._2).toSeq trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) @@ -170,14 +170,6 @@ class OffsetManager(val config: OffsetManagerConfig, } - def offsetsTopicConfig: Properties = { - val props = new Properties - props.put(LogConfig.SegmentBytesProp, config.offsetsTopicSegmentBytes.toString) - props.put(LogConfig.CleanupPolicyProp, "compact") - props.put(LogConfig.CompressionTypeProp, "uncompressed") - props - } - def partitionFor(group: String): Int = Utils.abs(group.hashCode) % config.offsetsTopicNumPartitions /** @@ -214,22 +206,14 @@ 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(groupId: String, consumerId: String, 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 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 + // first filter out partitions with offset metadata size exceeding limit val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => - validateOffsetMetadataLength(offsetAndMetadata.metadata) || nonExistentTopics.contains(topicAndPartition) + validateOffsetMetadataLength(offsetAndMetadata.metadata) } // construct the message set to append @@ -240,7 +224,7 @@ class OffsetManager(val config: OffsetManagerConfig, ) }.toSeq - val offsetTopicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, partitionFor(groupId)) + val offsetTopicPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, partitionFor(groupId)) val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) @@ -271,6 +255,10 @@ class OffsetManager(val config: OffsetManagerConfig, ErrorMapping.ConsumerCoordinatorNotAvailableCode else if (status.error == ErrorMapping.NotLeaderForPartitionCode) ErrorMapping.NotCoordinatorForConsumerCode + else if (status.error == ErrorMapping.MessageSizeTooLargeCode + || status.error == ErrorMapping.MessageSetSizeTooLargeCode + || status.error == ErrorMapping.InvalidFetchSizeCode) + Errors.INVALID_COMMIT_OFFSET_SIZE.code else status.error } @@ -278,9 +266,7 @@ class OffsetManager(val config: OffsetManagerConfig, // compute the final error codes for the commit response val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => - if (nonExistentTopics.contains(topicAndPartition)) - (topicAndPartition, ErrorMapping.UnknownTopicOrPartitionCode) - else if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) + if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) (topicAndPartition, responseCode) else (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) @@ -338,7 +324,7 @@ class OffsetManager(val config: OffsetManagerConfig, debug("Could not fetch offsets for group %s (not offset coordinator).".format(group)) topicPartitions.map { topicAndPartition => val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) - (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotOffsetManagerForGroup) + (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotCoordinatorForGroup) }.toMap } } @@ -349,7 +335,7 @@ class OffsetManager(val config: OffsetManagerConfig, */ def loadOffsetsFromLog(offsetsPartition: Int) { - val topicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + val topicPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) loadingPartitions synchronized { if (loadingPartitions.contains(offsetsPartition)) { @@ -421,7 +407,7 @@ class OffsetManager(val config: OffsetManagerConfig, } private def getHighWatermark(partitionId: Int): Long = { - val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, partitionId) + val partitionOpt = replicaManager.getPartition(ConsumerCoordinator.OffsetsTopicName, partitionId) val hw = partitionOpt.map { partition => partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L) @@ -449,7 +435,7 @@ class OffsetManager(val config: OffsetManagerConfig, } if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." - .format(numRemoved, TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition))) + .format(numRemoved, TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition))) } @@ -461,8 +447,6 @@ class OffsetManager(val config: OffsetManagerConfig, object OffsetManager { - val OffsetsTopicName = "__consumer_offsets" - private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) private val CURRENT_OFFSET_SCHEMA_VERSION = 1.toShort diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 59c9bc3..795220e 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -23,19 +23,19 @@ import kafka.cluster.{BrokerEndPoint, Partition, Replica} import kafka.log.{LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController -import kafka.common.TopicAndPartition import kafka.message.{ByteBufferMessageSet, MessageSet} +import kafka.api.ProducerResponseStatus +import kafka.common.TopicAndPartition +import kafka.api.PartitionFetchInfo + +import org.apache.kafka.common.protocol.Errors 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.Some import scala.collection._ -import scala.collection.mutable.HashMap -import scala.collection.Map -import scala.collection.Set import org.I0Itec.zkclient.ZkClient import com.yammer.metrics.core.Gauge @@ -84,6 +84,17 @@ object LogReadResult { false) } +case class BecomeLeaderOrFollowerResult(responseMap: collection.Map[(String, Int), Short], + updatedLeaders: Set[Partition], + updatedFollowers: Set[Partition], + errorCode: Short) { + + override def toString = { + "updated leaders: [%s], updated followers: [%s], update results: [%s], global error: [%d]" + .format(updatedLeaders, updatedFollowers, responseMap, errorCode) + } +} + object ReplicaManager { val HighWatermarkFilename = "replication-offset-checkpoint" } @@ -393,10 +404,10 @@ class ReplicaManager(val config: KafkaConfig, (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(utpe))) case nle: NotLeaderForPartitionException => (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(nle))) - case mtl: MessageSizeTooLargeException => - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mtl))) - case mstl: MessageSetSizeTooLargeException => - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mstl))) + case mtle: MessageSizeTooLargeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mtle))) + case mstle: MessageSetSizeTooLargeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mstle))) case imse : InvalidMessageSizeException => (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(imse))) case t: Throwable => @@ -416,7 +427,7 @@ class ReplicaManager(val config: KafkaConfig, def fetchMessages(timeout: Long, replicaId: Int, fetchMinBytes: Int, - fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], + fetchInfo: immutable.Map[TopicAndPartition, PartitionFetchInfo], responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { val isFromFollower = replicaId >= 0 @@ -544,30 +555,29 @@ class ReplicaManager(val config: KafkaConfig, } } - def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest, - offsetManager: OffsetManager): (collection.Map[(String, Int), Short], Short) = { + def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): BecomeLeaderOrFollowerResult = { leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partition), stateInfo) => stateChangeLogger.trace("Broker %d received LeaderAndIsr request %s correlation id %d from controller %d epoch %d for partition [%s,%d]" .format(localBrokerId, stateInfo, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, leaderAndISRRequest.controllerEpoch, topic, partition)) } replicaStateChangeLock synchronized { - val responseMap = new collection.mutable.HashMap[(String, Int), Short] - if(leaderAndISRRequest.controllerEpoch < controllerEpoch) { + val responseMap = new mutable.HashMap[(String, Int), Short] + if (leaderAndISRRequest.controllerEpoch < controllerEpoch) { leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partition), stateInfo) => stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d since " + "its controller epoch %d is old. Latest known controller epoch is %d").format(localBrokerId, leaderAndISRRequest.controllerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerEpoch, controllerEpoch)) } - (responseMap, ErrorMapping.StaleControllerEpochCode) + BecomeLeaderOrFollowerResult(responseMap, Set.empty[Partition], Set.empty[Partition], ErrorMapping.StaleControllerEpochCode) } else { val controllerId = leaderAndISRRequest.controllerId val correlationId = leaderAndISRRequest.correlationId controllerEpoch = leaderAndISRRequest.controllerEpoch // First check partition's leader epoch - val partitionState = new HashMap[Partition, PartitionStateInfo]() - leaderAndISRRequest.partitionStateInfos.foreach{ case ((topic, partitionId), partitionStateInfo) => + val partitionState = new mutable.HashMap[Partition, PartitionStateInfo]() + leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partitionId), partitionStateInfo) => val partition = getOrCreatePartition(topic, partitionId) val partitionLeaderEpoch = partition.getLeaderEpoch() // If the leader epoch is valid record the epoch of the controller that made the leadership decision. @@ -591,14 +601,19 @@ class ReplicaManager(val config: KafkaConfig, } } - val partitionsTobeLeader = partitionState - .filter{ case (partition, partitionStateInfo) => partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId} + val partitionsTobeLeader = partitionState.filter { case (partition, partitionStateInfo) => + partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId + } val partitionsToBeFollower = (partitionState -- partitionsTobeLeader.keys) - if (!partitionsTobeLeader.isEmpty) - makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap, offsetManager) - if (!partitionsToBeFollower.isEmpty) - makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap, offsetManager) + val partitionsBecomeLeader = if (!partitionsTobeLeader.isEmpty) + makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap) + else + Set.empty[Partition] + val partitionsBecomeFollower = if (!partitionsToBeFollower.isEmpty) + makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap) + else + Set.empty[Partition] // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions // have been completely populated before starting the checkpointing there by avoiding weird race conditions @@ -607,7 +622,7 @@ class ReplicaManager(val config: KafkaConfig, hwThreadInitialized = true } replicaFetcherManager.shutdownIdleFetcherThreads() - (responseMap, ErrorMapping.NoError) + BecomeLeaderOrFollowerResult(responseMap, partitionsBecomeLeader, partitionsBecomeFollower, ErrorMapping.NoError) } } } @@ -623,10 +638,11 @@ class ReplicaManager(val config: KafkaConfig, * the error message will be set on each partition since we do not know which partition caused it * TODO: the above may need to be fixed later */ - private def makeLeaders(controllerId: Int, epoch: Int, + private def makeLeaders(controllerId: Int, + epoch: Int, partitionState: Map[Partition, PartitionStateInfo], - correlationId: Int, responseMap: mutable.Map[(String, Int), Short], - offsetManager: OffsetManager) = { + correlationId: Int, + responseMap: mutable.Map[(String, Int), Short]): Set[Partition] = { partitionState.foreach(state => stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + "starting the become-leader transition for partition %s") @@ -645,7 +661,7 @@ class ReplicaManager(val config: KafkaConfig, } // Update the partition information to be the leader partitionState.foreach{ case (partition, partitionStateInfo) => - partition.makeLeader(controllerId, partitionStateInfo, correlationId, offsetManager)} + partition.makeLeader(controllerId, partitionStateInfo, correlationId)} } catch { case e: Throwable => @@ -664,6 +680,8 @@ class ReplicaManager(val config: KafkaConfig, "for the become-leader transition for partition %s") .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) } + + partitionState.keySet } /* @@ -682,9 +700,12 @@ class ReplicaManager(val config: KafkaConfig, * If an unexpected error is thrown in this function, it will be propagated to KafkaApis where * the error message will be set on each partition since we do not know which partition caused it */ - private def makeFollowers(controllerId: Int, epoch: Int, partitionState: Map[Partition, PartitionStateInfo], - leaders: Set[BrokerEndPoint], correlationId: Int, responseMap: mutable.Map[(String, Int), Short], - offsetManager: OffsetManager) { + private def makeFollowers(controllerId: Int, + epoch: Int, + partitionState: Map[Partition, PartitionStateInfo], + leaders: Set[BrokerEndPoint], + correlationId: Int, + responseMap: mutable.Map[(String, Int), Short]) : Set[Partition] = { partitionState.foreach { state => stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + "starting the become-follower transition for partition %s") @@ -694,18 +715,18 @@ class ReplicaManager(val config: KafkaConfig, for (partition <- partitionState.keys) responseMap.put((partition.topic, partition.partitionId), ErrorMapping.NoError) - try { + val partitionsToMakeFollower: mutable.Set[Partition] = mutable.Set() - var partitionsToMakeFollower: Set[Partition] = Set() + try { - // TODO: Delete leaders from LeaderAndIsrRequest in 0.8.1 + // TODO: Delete leaders from LeaderAndIsrRequest partitionState.foreach{ case (partition, partitionStateInfo) => 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)) + if (partition.makeFollower(controllerId, partitionStateInfo, correlationId)) partitionsToMakeFollower += partition else stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " + @@ -775,6 +796,8 @@ class ReplicaManager(val config: KafkaConfig, "for the become-follower transition for partition %s") .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) } + + partitionsToMakeFollower } private def maybeShrinkIsr(): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 17b17b9..92ffb91 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -25,12 +25,13 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.clients.consumer.NoOffsetForPartitionException import kafka.utils.{TestUtils, Logging} -import kafka.server.{KafkaConfig, OffsetManager} +import kafka.server.KafkaConfig import java.util.ArrayList import org.junit.Assert._ import scala.collection.JavaConversions._ +import kafka.coordinator.ConsumerCoordinator /** @@ -158,9 +159,9 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumer0.poll(50) // get metadata for the topic - var parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + var parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName) while(parts == null) - parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName) assertEquals(1, parts.size) assertNotNull(parts(0).leader()) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 07b1ff4..afcc349 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -26,6 +26,7 @@ import org.apache.kafka.clients.producer.KafkaProducer import kafka.server.{OffsetManager, KafkaConfig} import kafka.integration.KafkaServerTestHarness import scala.collection.mutable.Buffer +import kafka.coordinator.ConsumerCoordinator /** * A helper class for writing integration tests that involve producers, consumers, and servers @@ -63,11 +64,11 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { consumers += new KafkaConsumer(consumerConfig) // create the consumer offset topic - TestUtils.createTopic(zkClient, OffsetManager.OffsetsTopicName, - serverConfig.getProperty("offsets.topic.num.partitions").toInt, - serverConfig.getProperty("offsets.topic.replication.factor").toInt, + TestUtils.createTopic(zkClient, ConsumerCoordinator.OffsetsTopicName, + serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, + serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt, servers, - servers(0).offsetManager.offsetsTopicConfig) + servers(0).consumerCoordinator.offsetsTopicConfigs) } override def tearDown() { diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index c7136f2..dcd6988 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -22,9 +22,9 @@ import org.scalatest.junit.JUnit3Suite import kafka.utils.Logging import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness -import kafka.server.{OffsetManager, KafkaConfig} import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils +import kafka.coordinator.ConsumerCoordinator class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { @@ -87,12 +87,12 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin // create the offset topic val createOffsetTopicOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString, "--replication-factor", "1", - "--topic", OffsetManager.OffsetsTopicName)) + "--topic", ConsumerCoordinator.OffsetsTopicName)) TopicCommand.createTopic(zkClient, createOffsetTopicOpts) // try to delete the OffsetManager.OffsetsTopicName and make sure it doesn't - val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", OffsetManager.OffsetsTopicName)) - val deleteOffsetTopicPath = ZkUtils.getDeleteTopicPath(OffsetManager.OffsetsTopicName) + val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", ConsumerCoordinator.OffsetsTopicName)) + val deleteOffsetTopicPath = ZkUtils.getDeleteTopicPath(ConsumerCoordinator.OffsetsTopicName) assertFalse("Delete path for topic shouldn't exist before deletion.", zkClient.exists(deleteOffsetTopicPath)) intercept[AdminOperationException] { TopicCommand.deleteTopic(zkClient, deleteOffsetTopicOpts) diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index 4f124af..4b326d0 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -22,6 +22,7 @@ import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test import kafka.server.OffsetManager +import kafka.coordinator.ConsumerCoordinator class TopicFilterTest extends JUnitSuite { @@ -37,8 +38,8 @@ class TopicFilterTest extends JUnitSuite { val topicFilter2 = new Whitelist(".+") assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true)) - assertFalse(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false)) + assertFalse(topicFilter2.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter2.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = false)) val topicFilter3 = new Whitelist("white_listed-topic.+") assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true)) @@ -57,8 +58,8 @@ class TopicFilterTest extends JUnitSuite { assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) - assertFalse(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false)) + assertFalse(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = false)) } @Test diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala index a44fbd6..3cd726d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala @@ -22,8 +22,8 @@ import java.util.concurrent.TimeUnit import junit.framework.Assert._ import kafka.common.TopicAndPartition -import kafka.server.{KafkaConfig, OffsetManager} -import kafka.utils.TestUtils +import kafka.server.{OffsetManager, ReplicaManager, KafkaConfig} +import kafka.utils.{KafkaScheduler, TestUtils} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.JoinGroupRequest import org.easymock.EasyMock @@ -45,8 +45,8 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { val ConsumerMinSessionTimeout = 10 val ConsumerMaxSessionTimeout = 30 val DefaultSessionTimeout = 20 - var offsetManager: OffsetManager = null var consumerCoordinator: ConsumerCoordinator = null + var offsetManager : OffsetManager = null @Before def setUp() { @@ -54,12 +54,13 @@ class ConsumerCoordinatorResponseTest extends JUnitSuite { props.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) props.setProperty(KafkaConfig.ConsumerMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) offsetManager = EasyMock.createStrictMock(classOf[OffsetManager]) - consumerCoordinator = new ConsumerCoordinator(KafkaConfig.fromProps(props), null, offsetManager) + consumerCoordinator = ConsumerCoordinator.create(KafkaConfig.fromProps(props), null, offsetManager) consumerCoordinator.startup() } @After def tearDown() { + EasyMock.reset(offsetManager) consumerCoordinator.shutdown() } diff --git a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala index 08854c5..2cbf6e2 100644 --- a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala @@ -40,7 +40,7 @@ class CoordinatorMetadataTest extends JUnitSuite { def setUp() { val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") zkClient = EasyMock.createStrictMock(classOf[ZkClient]) - coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props), zkClient, null) + coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props).brokerId, zkClient, null) } @Test diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 528525b..39a6852 100755 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -120,7 +120,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val fetchRequest2 = OffsetFetchRequest(group, Seq(unknownTopicAndPartition)) val fetchResponse2 = simpleConsumer.fetchOffsets(fetchRequest2) - assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse2.requestInfo.get(unknownTopicAndPartition).get) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse2.requestInfo.get(unknownTopicAndPartition).get) assertEquals(1, fetchResponse2.requestInfo.size) } @@ -166,14 +166,14 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) - assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) - assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) - assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) - assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) + assertEquals(OffsetMetadataAndError.NoOffset, 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) -- 1.7.12.4 From fd612a2d50f1ee13009395f082357403c4277164 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Mon, 6 Jul 2015 15:47:40 -0700 Subject: [PATCH 44/59] kafka-2248; Use Apache Rat to enforce copyright headers; patched by Ewen Cheslack-Postava; reviewed by Gwen Shapira, Joel Joshy and Jun Rao --- .rat-excludes | 26 --- build.gradle | 27 ++- .../test/scala/other/kafka/TestOffsetManager.scala | 17 ++ gradle/buildscript.gradle | 19 +- gradle/license.gradle | 9 - gradle/rat.gradle | 115 ++++++++++++ gradle/resources/rat-output-to-html.xsl | 206 +++++++++++++++++++++ kafka-patch-review.py | 17 ++ scala.gradle | 15 ++ topics.json | 4 - 10 files changed, 409 insertions(+), 46 deletions(-) delete mode 100644 .rat-excludes delete mode 100644 gradle/license.gradle create mode 100644 gradle/rat.gradle create mode 100644 gradle/resources/rat-output-to-html.xsl delete mode 100644 topics.json diff --git a/.rat-excludes b/.rat-excludes deleted file mode 100644 index 01d6298..0000000 --- a/.rat-excludes +++ /dev/null @@ -1,26 +0,0 @@ -.rat-excludes -rat.out -sbt -sbt.boot.lock -README* -.gitignore -.git -.svn -build.properties -target -src_managed -update.log -clients/target -core/target -contrib/target -project/plugins/target -project/build/target -*.iml -*.csproj -TODO -Makefile* -*.html -*.xml -*expected.out -*.kafka - diff --git a/build.gradle b/build.gradle index 30d1cf2..727d7c5 100644 --- a/build.gradle +++ b/build.gradle @@ -13,11 +13,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +import org.ajoberstar.grgit.Grgit + buildscript { repositories { mavenCentral() } apply from: file('gradle/buildscript.gradle'), to: buildscript + + dependencies { + // For Apache Rat plugin to ignore non-Git files, need ancient version for Java 6 compatibility + classpath group: 'org.ajoberstar', name: 'grgit', version: '0.2.3' + } } def slf4jlog4j='org.slf4j:slf4j-log4j12:1.7.6' @@ -41,8 +48,24 @@ ext { } apply from: file('wrapper.gradle') -apply from: file('gradle/license.gradle') apply from: file('scala.gradle') +apply from: file('gradle/rat.gradle') + +rat { + // Exclude everything under the directory that git should be ignoring via .gitignore or that isn't checked in. These + // restrict us only to files that are checked in or are staged. + def repo = Grgit.open(project.file('.')) + excludes = new ArrayList(repo.clean(ignore: false, directories: true, dryRun: true)) + // And some of the files that we have checked in should also be excluded from this check + excludes.addAll([ + '**/.git/**', + 'gradlew', + 'gradlew.bat', + '**/README.md', + '.reviewboardrc', + 'system_test/**', + ]) +} subprojects { apply plugin: 'java' @@ -52,8 +75,6 @@ subprojects { sourceCompatibility = 1.6 - licenseTest.onlyIf { isVerificationRequired(project) } - uploadArchives { repositories { signing { diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala index 8047da4..e0e46c8 100644 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -1,3 +1,20 @@ +/** + * 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 other.kafka import org.I0Itec.zkclient.ZkClient diff --git a/gradle/buildscript.gradle b/gradle/buildscript.gradle index 5e45c06..047632b 100644 --- a/gradle/buildscript.gradle +++ b/gradle/buildscript.gradle @@ -1,3 +1,18 @@ +// 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. + repositories { repositories { // For license plugin. @@ -6,7 +21,3 @@ repositories { } } } - -dependencies { - classpath 'nl.javadude.gradle.plugins:license-gradle-plugin:0.10.0' -} diff --git a/gradle/license.gradle b/gradle/license.gradle deleted file mode 100644 index b4b62eb..0000000 --- a/gradle/license.gradle +++ /dev/null @@ -1,9 +0,0 @@ -subprojects { - apply plugin: 'license' - - license { - header rootProject.file('HEADER') - // Skip Twitter bootstrap JS and CSS. - skipExistingHeaders = true - } -} diff --git a/gradle/rat.gradle b/gradle/rat.gradle new file mode 100644 index 0000000..d62b372 --- /dev/null +++ b/gradle/rat.gradle @@ -0,0 +1,115 @@ +/* + * 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. + */ + +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.Task +import org.gradle.api.internal.project.IsolatedAntBuilder + +apply plugin: RatPlugin + +class RatTask extends DefaultTask { + @Input + List excludes + + def reportPath = 'build/rat' + def stylesheet = 'gradle/resources/rat-output-to-html.xsl' + def xmlReport = reportPath + '/rat-report.xml' + def htmlReport = reportPath + '/rat-report.html' + + def generateXmlReport(File reportDir) { + def antBuilder = services.get(IsolatedAntBuilder) + def ratClasspath = project.configurations.rat + antBuilder.withClasspath(ratClasspath).execute { + ant.taskdef(resource: 'org/apache/rat/anttasks/antlib.xml') + ant.report(format: 'xml', reportFile: xmlReport) { + fileset(dir: ".") { + patternset { + excludes.each { + exclude(name: it) + } + } + } + } + } + } + + def printUnknownFiles() { + def ratXml = new XmlParser().parse(xmlReport) + def unknownLicenses = 0 + ratXml.resource.each { resource -> + if (resource.'license-approval'.@name[0] == "false") { + println('Unknown license: ' + resource.@name) + unknownLicenses++ + } + } + if (unknownLicenses > 0) { + throw new GradleException("Found " + unknownLicenses + " files with " + + "unknown licenses.") + } + } + + def generateHtmlReport() { + def antBuilder = services.get(IsolatedAntBuilder) + def ratClasspath = project.configurations.rat + antBuilder.withClasspath(ratClasspath).execute { + ant.xslt( + in: xmlReport, + style: stylesheet, + out: htmlReport, + classpath: ratClasspath) + } + println('Rat report: ' + htmlReport) + } + + @TaskAction + def rat() { + File reportDir = new File(reportPath) + if (!reportDir.exists()) { + reportDir.mkdirs() + } + generateXmlReport(reportDir) + printUnknownFiles() + generateHtmlReport() + } +} + +class RatPlugin implements Plugin { + void apply(Project project) { + configureDependencies(project) + project.plugins.apply(JavaPlugin); + Task ratTask = project.task("rat", + type: RatTask, + group: 'Build', + description: 'Runs Apache Rat checks.') + project.tasks[JavaPlugin.TEST_TASK_NAME].dependsOn ratTask + } + + void configureDependencies(final Project project) { + project.configurations { + rat + } + project.repositories { + mavenCentral() + } + project.dependencies { + rat 'org.apache.rat:apache-rat-tasks:0.11' + } + } +} diff --git a/gradle/resources/rat-output-to-html.xsl b/gradle/resources/rat-output-to-html.xsl new file mode 100644 index 0000000..97ea7a1 --- /dev/null +++ b/gradle/resources/rat-output-to-html.xsl @@ -0,0 +1,206 @@ + + + + + + + + + + + + + + + + + + + + + + +

        Rat Report

        +

        This HTML version (yes, it is!) is generated from the RAT xml reports using Saxon9B. All the outputs required are displayed below, similar to the .txt version. + This is obviously a work in progress; and a prettier, easier to read and manage version will be available soon

        +
        + + + + + + + + + + + + + + + + + + + + + + + + + + +
        +Table 1: A snapshot summary of this rat report. +
        Notes: Binaries: Archives: Standards:
        Apache Licensed: Generated Documents:
        Note: JavaDocs are generated and so license header is optionalNote: Generated files do not require license headers
        Unknown Licenses - or files without a license. Unknown Licenses - or files without a license.
        +
        +
        +

        Unapproved Licenses:

        + + + +
        + + +
        +
        + +

        Archives:

        + + + + +
        +
        +
        + +

        + Files with Apache License headers will be marked AL
        + Binary files (which do not require AL headers) will be marked B
        + Compressed archives will be marked A
        + Notices, licenses etc will be marked N
        +

        + + + + ! + + + + N + A + B + + !!!!! + + +
        + + +
        +
        + +

        Printing headers for files without AL header...

        + + + +

        + +
        +
        +
        + + + +
        + + +
        +

        Resource:

        + +
        +
        + + + +

        First few lines of non-compliant file

        +

        + +

        +
        +

        Other Info:

        +
        + + + Header Type: +
        +
        + + + License Family: +
        +
        + + + License Approval: +
        +
        + + + Type: +
        +
        + + + +
        diff --git a/kafka-patch-review.py b/kafka-patch-review.py index b5a2e95..94873c3 100644 --- a/kafka-patch-review.py +++ b/kafka-patch-review.py @@ -1,4 +1,21 @@ #!/usr/bin/env python +# +# 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. import argparse import sys diff --git a/scala.gradle b/scala.gradle index cabb59c..5eb2a65 100644 --- a/scala.gradle +++ b/scala.gradle @@ -1,3 +1,18 @@ +// 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. + if (!hasProperty('scalaVersion')) { ext.scalaVersion = '2.10.5' } diff --git a/topics.json b/topics.json deleted file mode 100644 index ff011ed..0000000 --- a/topics.json +++ /dev/null @@ -1,4 +0,0 @@ -{"topics": - [{"topic": "foo"}], - "version":1 - } -- 1.7.12.4 From 2d96da05a0af7847aca5edc6d003a18be7f5216a Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Mon, 6 Jul 2015 16:36:20 -0700 Subject: [PATCH 45/59] kafka-2132; Move Log4J appender to a separate module; patched by Ashish Singh; reviewed by Gwen Shapira, Aditya Auradkar and Jun Rao --- build.gradle | 60 ++++++-- checkstyle/import-control.xml | 9 +- .../scala/kafka/producer/KafkaLog4jAppender.scala | 97 ------------ .../unit/kafka/log4j/KafkaLog4jAppenderTest.scala | 143 ------------------ .../kafka/log4jappender/KafkaLog4jAppender.java | 167 +++++++++++++++++++++ .../log4jappender/KafkaLog4jAppenderTest.java | 98 ++++++++++++ .../log4jappender/MockKafkaLog4jAppender.java | 47 ++++++ settings.gradle | 2 +- 8 files changed, 370 insertions(+), 253 deletions(-) delete mode 100644 core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala delete mode 100755 core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala create mode 100644 log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java create mode 100644 log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java create mode 100644 log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java diff --git a/build.gradle b/build.gradle index 727d7c5..ab86987 100644 --- a/build.gradle +++ b/build.gradle @@ -132,7 +132,7 @@ subprojects { archives srcJar archives javadocJar } - + plugins.withType(ScalaPlugin) { //source jar should also contain scala source: srcJar.from sourceSets.main.scala @@ -202,20 +202,20 @@ for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_6'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_6', '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_5', 'jar_core_2_11_6', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar']) { } -tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_6', '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_5', 'srcJar_2_11_6', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar']) { } -tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_6', '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_5', 'docsJar_2_11_6', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar']) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_6', 'clients:test']) { +tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_6', 'clients:test', 'log4j-appender:test']) { } tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_5', 'releaseTarGz_2_11_6']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_6', '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_5', 'uploadCoreArchives_2_11_6', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives']) { } project(':core') { @@ -228,6 +228,7 @@ project(':core') { dependencies { compile project(':clients') + compile project(':log4j-appender') compile "org.scala-lang:scala-library:$scalaVersion" compile 'org.apache.zookeeper:zookeeper:3.4.6' compile 'com.101tec:zkclient:0.5' @@ -237,7 +238,6 @@ project(':core') { testCompile 'junit:junit:4.6' testCompile 'org.easymock:easymock:3.0' testCompile 'org.objenesis:objenesis:1.2' - testCompile project(':clients') if (scalaVersion.startsWith('2.10')) { testCompile 'org.scalatest:scalatest_2.10:1.9.1' } else if (scalaVersion.startsWith('2.11')) { @@ -273,9 +273,9 @@ project(':core') { into "$buildDir/dependant-libs-${scalaVersion}" } - tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { + tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { into "kafka_${baseScalaVersion}-${version}" - compression = Compression.GZIP + compression = Compression.GZIP from(project.file("../bin")) { into "bin/" } from(project.file("../config")) { into "config/" } from '../LICENSE' @@ -378,7 +378,7 @@ project(':clients') { compile 'org.xerial.snappy:snappy-java:1.1.1.7' compile 'net.jpountz.lz4:lz4:1.2.0' - testCompile 'com.novocode:junit-interface:0.9' + testCompile 'junit:junit:4.6' testRuntime "$slf4jlog4j" } @@ -405,7 +405,45 @@ project(':clients') { artifacts { archives testJar } - + + configurations { + archives.extendsFrom (testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':log4j-appender') { + apply plugin: 'checkstyle' + archivesBaseName = "kafka-log4j-appender" + + dependencies { + compile project(':clients') + compile "$slf4jlog4j" + + testCompile 'junit:junit:4.6' + testCompile project(path: ':clients', configuration: 'archives') + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/log4jappender/*" + } + checkstyle { configFile = new File(rootDir, "checkstyle/checkstyle.xml") } diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index f2e6cec..19e0659 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -95,8 +95,15 @@ + + + + + + + - + diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala deleted file mode 100644 index 5d36a01..0000000 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ /dev/null @@ -1,97 +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.producer - -import async.MissingConfigException -import org.apache.log4j.spi.LoggingEvent -import org.apache.log4j.AppenderSkeleton -import org.apache.log4j.helpers.LogLog -import kafka.utils.Logging -import java.util.{Properties, Date} -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} - -class KafkaLog4jAppender extends AppenderSkeleton with Logging { - var topic: String = null - var brokerList: String = null - var compressionType: String = null - var retries: Int = 0 - var requiredNumAcks: Int = Int.MaxValue - var syncSend: Boolean = false - - private var producer: KafkaProducer[Array[Byte],Array[Byte]] = null - - def getTopic: String = topic - def setTopic(topic: String) { this.topic = topic } - - def getBrokerList: String = brokerList - def setBrokerList(brokerList: String) { this.brokerList = brokerList } - - def getCompressionType: String = compressionType - def setCompressionType(compressionType: String) { this.compressionType = compressionType } - - def getRequiredNumAcks: Int = requiredNumAcks - def setRequiredNumAcks(requiredNumAcks: Int) { this.requiredNumAcks = requiredNumAcks } - - def getSyncSend: Boolean = syncSend - def setSyncSend(syncSend: Boolean) { this.syncSend = syncSend } - - def getRetries: Int = retries - def setRetries(retries: Int) { this.retries = retries } - - override def activateOptions() { - // check for config parameter validity - val props = new Properties() - if(brokerList != null) - props.put(org.apache.kafka.clients.producer.ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) - if(props.isEmpty) - throw new MissingConfigException("The bootstrap servers property should be specified") - if(topic == null) - 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) - if(retries > 0) props.put(org.apache.kafka.clients.producer.ProducerConfig.RETRIES_CONFIG, retries.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) - } - - override def append(event: LoggingEvent) { - val message = subAppend(event) - LogLog.debug("[" + new Date(event.getTimeStamp).toString + "]" + message) - val response = producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, message.getBytes())) - if (syncSend) response.get - } - - def subAppend(event: LoggingEvent): String = { - if(this.layout == null) - event.getRenderedMessage - else - this.layout.format(event) - } - - override def close() { - if(!this.closed) { - this.closed = true - producer.close() - } - } - - override def requiresLayout: Boolean = true -} diff --git a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala deleted file mode 100755 index 41366a1..0000000 --- a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala +++ /dev/null @@ -1,143 +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.log4j - -import kafka.consumer.SimpleConsumer -import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.{TestUtils, CoreUtils, Logging} -import kafka.api.FetchRequestBuilder -import kafka.producer.async.MissingConfigException -import kafka.serializer.Encoder -import kafka.zk.ZooKeeperTestHarness - -import java.util.Properties -import java.io.File - -import org.apache.log4j.spi.LoggingEvent -import org.apache.log4j.{PropertyConfigurator, Logger} -import org.junit.{After, Before, Test} -import org.scalatest.junit.JUnit3Suite - -import junit.framework.Assert._ - -class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { - - var logDirZk: File = null - var config: KafkaConfig = null - var server: KafkaServer = null - - var simpleConsumerZk: SimpleConsumer = null - - val tLogger = Logger.getLogger(getClass()) - - private val brokerZk = 0 - - @Before - override def setUp() { - super.setUp() - - val propsZk = TestUtils.createBrokerConfig(brokerZk, zkConnect) - val logDirZkPath = propsZk.getProperty("log.dir") - logDirZk = new File(logDirZkPath) - config = KafkaConfig.fromProps(propsZk) - server = TestUtils.createServer(config) - simpleConsumerZk = new SimpleConsumer("localhost", server.boundPort(), 1000000, 64 * 1024, "") - } - - @After - override def tearDown() { - simpleConsumerZk.close - server.shutdown - CoreUtils.rm(logDirZk) - super.tearDown() - } - - @Test - def testKafkaLog4jConfigs() { - // host missing - var props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - - try { - PropertyConfigurator.configure(props) - fail("Missing properties exception was expected !") - } catch { - case e: MissingConfigException => - } - - // topic missing - props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.brokerList", TestUtils.getBrokerListStrFromServers(Seq(server))) - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - - try { - PropertyConfigurator.configure(props) - fail("Missing properties exception was expected !") - } catch { - case e: MissingConfigException => - } - } - - @Test - def testLog4jAppends() { - PropertyConfigurator.configure(getLog4jConfig) - - for(i <- 1 to 5) - info("test") - - val response = simpleConsumerZk.fetch(new FetchRequestBuilder().addFetch("test-topic", 0, 0L, 1024*1024).build()) - val fetchMessage = response.messageSet("test-topic", 0) - - var count = 0 - for(message <- fetchMessage) { - count = count + 1 - } - - assertEquals(5, count) - } - - private def getLog4jConfig: Properties = { - val props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.BrokerList", TestUtils.getBrokerListStrFromServers(Seq(server))) - props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.appender.KAFKA.RequiredNumAcks", "1") - props.put("log4j.appender.KAFKA.SyncSend", "true") - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - props - } -} - -class AppenderStringEncoder(encoding: String = "UTF-8") extends Encoder[LoggingEvent] { - def toBytes(event: LoggingEvent): Array[Byte] = { - event.getMessage.toString.getBytes(encoding) - } -} - diff --git a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java new file mode 100644 index 0000000..628ff53 --- /dev/null +++ b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java @@ -0,0 +1,167 @@ +/** + * 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.log4jappender; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.config.ConfigException; +import org.apache.log4j.AppenderSkeleton; +import org.apache.log4j.helpers.LogLog; +import org.apache.log4j.spi.LoggingEvent; + +import java.util.Date; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * A log4j appender that produces log messages to Kafka + */ +public class KafkaLog4jAppender extends AppenderSkeleton { + + private static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + private static final String COMPRESSION_TYPE_CONFIG = "compression.type"; + private static final String ACKS_CONFIG = "acks"; + private static final String RETRIES_CONFIG = "retries"; + private static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; + private static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; + + private String brokerList = null; + private String topic = null; + private String compressionType = null; + + private int retries = 0; + private int requiredNumAcks = Integer.MAX_VALUE; + private boolean syncSend = false; + private Producer producer = null; + + public Producer getProducer() { + return producer; + } + + public String getBrokerList() { + return brokerList; + } + + public void setBrokerList(String brokerList) { + this.brokerList = brokerList; + } + + public int getRequiredNumAcks() { + return requiredNumAcks; + } + + public void setRequiredNumAcks(int requiredNumAcks) { + this.requiredNumAcks = requiredNumAcks; + } + + public int getRetries() { + return retries; + } + + public void setRetries(int retries) { + this.retries = retries; + } + + public String getCompressionType() { + return compressionType; + } + + public void setCompressionType(String compressionType) { + this.compressionType = compressionType; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public boolean getSyncSend() { + return syncSend; + } + + public void setSyncSend(boolean syncSend) { + this.syncSend = syncSend; + } + + @Override + public void activateOptions() { + // check for config parameter validity + Properties props = new Properties(); + if (brokerList != null) + props.put(BOOTSTRAP_SERVERS_CONFIG, brokerList); + if (props.isEmpty()) + throw new ConfigException("The bootstrap servers property should be specified"); + if (topic == null) + throw new ConfigException("Topic must be specified by the Kafka log4j appender"); + if (compressionType != null) + props.put(COMPRESSION_TYPE_CONFIG, compressionType); + if (requiredNumAcks != Integer.MAX_VALUE) + props.put(ACKS_CONFIG, requiredNumAcks); + if (retries > 0) + props.put(RETRIES_CONFIG, retries); + + props.put(KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + props.put(VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + this.producer = getKafkaProducer(props); + LogLog.debug("Kafka producer connected to " + brokerList); + LogLog.debug("Logging for topic: " + topic); + } + + protected Producer getKafkaProducer(Properties props) { + return new KafkaProducer(props); + } + + @Override + protected void append(LoggingEvent event) { + String message = subAppend(event); + LogLog.debug("[" + new Date(event.getTimeStamp()) + "]" + message); + Future response = producer.send(new ProducerRecord(topic, message.getBytes())); + if (syncSend) { + try { + response.get(); + } catch (InterruptedException ex) { + throw new RuntimeException(ex); + } catch (ExecutionException ex) { + throw new RuntimeException(ex); + } + } + } + + private String subAppend(LoggingEvent event) { + return (this.layout == null) ? event.getRenderedMessage() : this.layout.format(event); + } + + @Override + public void close() { + if (!this.closed) { + this.closed = true; + producer.close(); + } + } + + @Override + public boolean requiresLayout() { + return true; + } +} diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java new file mode 100644 index 0000000..71bdd94 --- /dev/null +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java @@ -0,0 +1,98 @@ +/** + * 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.log4jappender; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.log4j.Logger; +import org.apache.log4j.PropertyConfigurator; +import org.junit.Assert; +import org.junit.Test; + +import java.io.UnsupportedEncodingException; +import java.util.Properties; + +public class KafkaLog4jAppenderTest { + + Logger logger = Logger.getLogger(KafkaLog4jAppenderTest.class); + + @Test + public void testKafkaLog4jConfigs() { + // host missing + Properties props = new Properties(); + props.put("log4j.rootLogger", "INFO"); + props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender"); + props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.put("log4j.appender.KAFKA.Topic", "test-topic"); + props.put("log4j.logger.kafka.log4j", "INFO, KAFKA"); + + try { + PropertyConfigurator.configure(props); + Assert.fail("Missing properties exception was expected !"); + } catch (ConfigException ex) { + // It's OK! + } + + // topic missing + props = new Properties(); + props.put("log4j.rootLogger", "INFO"); + props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender"); + props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.put("log4j.appender.KAFKA.brokerList", "127.0.0.1:9093"); + props.put("log4j.logger.kafka.log4j", "INFO, KAFKA"); + + try { + PropertyConfigurator.configure(props); + Assert.fail("Missing properties exception was expected !"); + } catch (ConfigException ex) { + // It's OK! + } + } + + + @Test + public void testLog4jAppends() throws UnsupportedEncodingException { + PropertyConfigurator.configure(getLog4jConfig()); + + for (int i = 1; i <= 5; ++i) { + logger.error(getMessage(i)); + } + + Assert.assertEquals( + 5, ((MockKafkaLog4jAppender) (logger.getRootLogger().getAppender("KAFKA"))).getHistory().size()); + } + + private byte[] getMessage(int i) throws UnsupportedEncodingException { + return ("test_" + i).getBytes("UTF-8"); + } + + private Properties getLog4jConfig() { + Properties props = new Properties(); + props.put("log4j.rootLogger", "INFO, KAFKA"); + props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.MockKafkaLog4jAppender"); + props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.put("log4j.appender.KAFKA.BrokerList", "127.0.0.1:9093"); + props.put("log4j.appender.KAFKA.Topic", "test-topic"); + props.put("log4j.appender.KAFKA.RequiredNumAcks", "1"); + props.put("log4j.appender.KAFKA.SyncSend", "false"); + props.put("log4j.logger.kafka.log4j", "INFO, KAFKA"); + return props; + } +} + diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java new file mode 100644 index 0000000..c35f26a --- /dev/null +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.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.log4jappender; + +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.test.MockSerializer; +import org.apache.log4j.spi.LoggingEvent; + +import java.util.Properties; + +public class MockKafkaLog4jAppender extends KafkaLog4jAppender { + private MockProducer mockProducer = + new MockProducer(false, new MockSerializer(), new MockSerializer()); + + @Override + protected Producer getKafkaProducer(Properties props) { + return mockProducer; + } + + @Override + protected void append(LoggingEvent event) { + if (super.getProducer() == null) { + activateOptions(); + } + super.append(event); + } + + protected java.util.List> getHistory() { + return mockProducer.history(); + } +} diff --git a/settings.gradle b/settings.gradle index 83f764e..3b6a952 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,4 +14,4 @@ // limitations under the License. apply from: file('scala.gradle') -include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients' +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'log4j-appender' -- 1.7.12.4 From 271b18d119fdc37952c36c573ba185aa672e3f96 Mon Sep 17 00:00:00 2001 From: Kostya Golikov Date: Tue, 7 Jul 2015 09:12:57 -0700 Subject: [PATCH 46/59] KAFKA-2314: proper MirrorMaker's message handler help message; reviewed by Guozhang Wang --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 459aaec..797b4bb 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -131,9 +131,9 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { .ofType(classOf[String]) val messageHandlerOpt = parser.accepts("message.handler", - "The consumer rebalance listener to use for mirror maker consumer.") + "Message handler which will process every record in-between consumer and producer.") .withRequiredArg() - .describedAs("A custom rebalance listener of type MirrorMakerMessageHandler") + .describedAs("A custom message handler of type MirrorMakerMessageHandler") .ofType(classOf[String]) val messageHandlerArgsOpt = parser.accepts("message.handler.args", -- 1.7.12.4 From f77dc386c099da5ff0bac4d2a12b04f7f17f07d3 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 7 Jul 2015 09:45:26 -0700 Subject: [PATCH 47/59] kafka-1367; Broker topic metadata not kept in sync with ZooKeeper; patched by Ashish Singh; reviewed by Jun Rao --- .../scala/kafka/common/TopicAndPartition.scala | 6 +- .../scala/kafka/controller/KafkaController.scala | 69 ++++++++++++++++-- .../main/scala/kafka/utils/ReplicationUtils.scala | 16 ++++- core/src/main/scala/kafka/utils/ZkUtils.scala | 1 + .../unit/kafka/integration/TopicMetadataTest.scala | 84 +++++++++++++++++++--- .../unit/kafka/utils/ReplicationUtilsTest.scala | 2 + 6 files changed, 158 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/common/TopicAndPartition.scala b/core/src/main/scala/kafka/common/TopicAndPartition.scala index df3db91..13a3f28 100644 --- a/core/src/main/scala/kafka/common/TopicAndPartition.scala +++ b/core/src/main/scala/kafka/common/TopicAndPartition.scala @@ -1,6 +1,7 @@ package kafka.common import kafka.cluster.{Replica, Partition} +import kafka.utils.Json /** * Licensed to the Apache Software Foundation (ASF) under one or more @@ -24,6 +25,8 @@ import kafka.cluster.{Replica, Partition} */ case class TopicAndPartition(topic: String, partition: Int) { + private val version: Long = 1L + def this(tuple: (String, Int)) = this(tuple._1, tuple._2) def this(partition: Partition) = this(partition.topic, partition.partitionId) @@ -33,5 +36,6 @@ case class TopicAndPartition(topic: String, partition: Int) { def asTuple = (topic, partition) override def toString = "[%s,%d]".format(topic, partition) -} + def toJson = Json.encode(Map("version" -> version, "topic" -> topic, "partition" -> partition)) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 3635057..09630d0 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -16,8 +16,9 @@ */ package kafka.controller -import collection._ -import collection.Set +import java.util + +import scala.collection._ import com.yammer.metrics.core.Gauge import java.util.concurrent.TimeUnit import kafka.admin.AdminUtils @@ -31,7 +32,7 @@ import kafka.utils.ZkUtils._ import kafka.utils._ import kafka.utils.CoreUtils._ import org.apache.zookeeper.Watcher.Event.KeeperState -import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} +import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.locks.ReentrantLock @@ -169,6 +170,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt private val partitionReassignedListener = new PartitionsReassignedListener(this) private val preferredReplicaElectionListener = new PreferredReplicaElectionListener(this) + private val isrChangeNotificationListener = new IsrChangeNotificationListener(this) newGauge( "ActiveControllerCount", @@ -307,6 +309,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt incrementControllerEpoch(zkClient) // before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks registerReassignedPartitionsListener() + registerIsrChangeNotificationListener() registerPreferredReplicaElectionListener() partitionStateMachine.registerListeners() replicaStateMachine.registerListeners() @@ -792,8 +795,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt controllerContext.controllerChannelManager.startup() } - private def updateLeaderAndIsrCache() { - val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, controllerContext.partitionReplicaAssignment.keySet) + def updateLeaderAndIsrCache(topicAndPartitions: Set[TopicAndPartition] = controllerContext.partitionReplicaAssignment.keySet) { + val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, topicAndPartitions) for((topicPartition, leaderIsrAndControllerEpoch) <- leaderAndIsrInfo) controllerContext.partitionLeadershipInfo.put(topicPartition, leaderIsrAndControllerEpoch) } @@ -892,6 +895,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) } + private def registerIsrChangeNotificationListener() = { + debug("Registering IsrChangeNotificationListener") + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) + zkClient.subscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) + } + private def deregisterReassignedPartitionsListener() = { zkClient.unsubscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) } @@ -1281,6 +1290,56 @@ class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic: } /** + * Called when leader intimates of isr change + * @param controller + */ +class IsrChangeNotificationListener(controller: KafkaController) extends IZkChildListener with Logging { + var topicAndPartitionSet: Set[TopicAndPartition] = Set() + + override def handleChildChange(parentPath: String, currentChildren: util.List[String]): Unit = { + import scala.collection.JavaConverters._ + + inLock(controller.controllerContext.controllerLock) { + debug("[IsrChangeNotificationListener] Fired!!!") + val childrenAsScala: mutable.Buffer[String] = currentChildren.asScala + val topicAndPartitions: immutable.Set[TopicAndPartition] = childrenAsScala.map(x => getTopicAndPartition(x)).flatten.toSet + controller.updateLeaderAndIsrCache(topicAndPartitions) + processUpdateNotifications(topicAndPartitions) + + // delete processed children + childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, ZkUtils.TopicConfigChangesPath + "/" + x)) + } + } + + private def processUpdateNotifications(topicAndPartitions: immutable.Set[TopicAndPartition]) { + val liveBrokers: Seq[Int] = controller.controllerContext.liveOrShuttingDownBrokerIds.toSeq + controller.sendUpdateMetadataRequest(liveBrokers, topicAndPartitions) + debug("Sending MetadataRequest to Brokers:" + liveBrokers + " for TopicAndPartitions:" + topicAndPartitions) + } + + private def getTopicAndPartition(child: String): Option[TopicAndPartition] = { + val changeZnode: String = ZkUtils.IsrChangeNotificationPath + "/" + child + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(controller.controllerContext.zkClient, changeZnode) + if (jsonOpt.isDefined) { + val json = Json.parseFull(jsonOpt.get) + + json match { + case Some(m) => + val topicAndPartition = m.asInstanceOf[Map[String, Any]] + val topic = topicAndPartition("topic").asInstanceOf[String] + val partition = topicAndPartition("partition").asInstanceOf[Int] + Some(TopicAndPartition(topic, partition)) + case None => + error("Invalid topic and partition JSON: " + json + " in ZK: " + changeZnode) + None + } + } else { + None + } + } +} + +/** * Starts the preferred replica leader election for the list of partitions specified under * /admin/preferred_replica_election - */ diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala index 6068733..783ba10 100644 --- a/core/src/main/scala/kafka/utils/ReplicationUtils.scala +++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala @@ -18,22 +18,32 @@ package kafka.utils import kafka.api.LeaderAndIsr +import kafka.common.TopicAndPartition import kafka.controller.LeaderIsrAndControllerEpoch -import org.apache.zookeeper.data.Stat import org.I0Itec.zkclient.ZkClient +import org.apache.zookeeper.data.Stat -import scala.Some import scala.collection._ object ReplicationUtils extends Logging { + val IsrChangeNotificationPrefix = "isr_change_" + def updateLeaderAndIsr(zkClient: ZkClient, topic: String, partitionId: Int, newLeaderAndIsr: LeaderAndIsr, controllerEpoch: Int, zkVersion: Int): (Boolean,Int) = { debug("Updated ISR for partition [%s,%d] to %s".format(topic, partitionId, newLeaderAndIsr.isr.mkString(","))) val path = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partitionId) val newLeaderData = ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch) // use the epoch of the controller that made the leadership decision, instead of the current controller epoch - ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) + val updatePersistentPath: (Boolean, Int) = ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) + if (updatePersistentPath._1) { + val topicAndPartition: TopicAndPartition = TopicAndPartition(topic, partitionId) + val isrChangeNotificationPath: String = ZkUtils.createSequentialPersistentPath( + zkClient, ZkUtils.IsrChangeNotificationPath + "/" + IsrChangeNotificationPrefix, + topicAndPartition.toJson) + debug("Added " + isrChangeNotificationPath + " for " + topicAndPartition) + } + updatePersistentPath } def checkLeaderAndIsrZkData(zkClient: ZkClient, path: String, expectedLeaderAndIsrInfo: String): (Boolean,Int) = { diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 78475e3..166814c 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -47,6 +47,7 @@ object ZkUtils extends Logging { val DeleteTopicsPath = "/admin/delete_topics" val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" val BrokerSequenceIdPath = "/brokers/seqid" + val IsrChangeNotificationPath = "/isr_change_notification" def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 995b059..a95ee5e 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -17,28 +17,32 @@ package kafka.integration -import org.apache.kafka.common.protocol.SecurityProtocol -import org.scalatest.junit.JUnit3Suite -import kafka.zk.ZooKeeperTestHarness -import kafka.admin.AdminUtils import java.nio.ByteBuffer + import junit.framework.Assert._ -import kafka.cluster.{BrokerEndPoint, Broker} +import kafka.admin.AdminUtils +import kafka.api.{TopicMetadataResponse, TopicMetadataRequest} +import kafka.client.ClientUtils +import kafka.cluster.{Broker, BrokerEndPoint} +import kafka.common.ErrorMapping +import kafka.server.{NotRunning, KafkaConfig, KafkaServer} import kafka.utils.TestUtils import kafka.utils.TestUtils._ -import kafka.server.{KafkaServer, KafkaConfig} -import kafka.api.TopicMetadataRequest -import kafka.common.ErrorMapping -import kafka.client.ClientUtils +import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.protocol.SecurityProtocol +import org.scalatest.junit.JUnit3Suite class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { private var server1: KafkaServer = null var brokerEndPoints: Seq[BrokerEndPoint] = null + var adHocConfigs: Seq[KafkaConfig] = null + val numConfigs: Int = 2 override def setUp() { super.setUp() - val props = createBrokerConfigs(1, zkConnect) - val configs = props.map(KafkaConfig.fromProps) + val props = createBrokerConfigs(numConfigs, zkConnect) + val configs: Seq[KafkaConfig] = props.map(KafkaConfig.fromProps) + adHocConfigs = configs.takeRight(configs.size - 1) // Started and stopped by individual test cases server1 = TestUtils.createServer(configs.head) brokerEndPoints = Seq(new Broker(server1.config.brokerId, server1.config.hostName, server1.boundPort()).getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) } @@ -130,4 +134,62 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(1, partitionMetadata.head.replicas.size) assertTrue(partitionMetadata.head.leader.isDefined) } + + private def checkIsr(servers: Seq[KafkaServer]): Unit = { + val activeBrokers: Seq[KafkaServer] = servers.filter(x => x.brokerState.currentState != NotRunning.state) + val expectedIsr: Seq[BrokerEndPoint] = activeBrokers.map( + x => new BrokerEndPoint(x.config.brokerId, + if (x.config.hostName.nonEmpty) x.config.hostName else "localhost", + x.boundPort()) + ) + + // Assert that topic metadata at new brokers is updated correctly + activeBrokers.foreach(x => { + var metadata: TopicMetadataResponse = new TopicMetadataResponse(Seq(), Seq(), -1) + waitUntilTrue(() => { + metadata = ClientUtils.fetchTopicMetadata( + Set.empty, + Seq(new BrokerEndPoint( + x.config.brokerId, + if (x.config.hostName.nonEmpty) x.config.hostName else "localhost", + x.boundPort())), + "TopicMetadataTest-testBasicTopicMetadata", + 2000, 0) + metadata.topicsMetadata.nonEmpty && + metadata.topicsMetadata.head.partitionsMetadata.nonEmpty && + expectedIsr == metadata.topicsMetadata.head.partitionsMetadata.head.isr + }, + "Topic metadata is not correctly updated for broker " + x + ".\n" + + "Expected ISR: " + expectedIsr + "\n" + + "Actual ISR : " + (if (metadata.topicsMetadata.nonEmpty && + metadata.topicsMetadata.head.partitionsMetadata.nonEmpty) + metadata.topicsMetadata.head.partitionsMetadata.head.isr + else + "")) + }) + } + + + def testIsrAfterBrokerShutDownAndJoinsBack { + // start adHoc brokers + val adHocServers = adHocConfigs.map(p => createServer(p)) + val allServers: Seq[KafkaServer] = Seq(server1) ++ adHocServers + + // create topic + val topic: String = "test" + AdminUtils.createTopic(zkClient, topic, 1, numConfigs) + + // shutdown a broker + adHocServers.last.shutdown() + adHocServers.last.awaitShutdown() + + // startup a broker + adHocServers.last.startup() + + // check metadata is still correct and updated at all brokers + checkIsr(allServers) + + // shutdown adHoc brokers + adHocServers.map(p => p.shutdown()) + } } diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index c96c0ff..b9de8d6 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -70,6 +70,8 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) EasyMock.replay(replicaManager) + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) + val replicas = List(0,1) // regular update -- 1.7.12.4 From ad485e148d7ac104abe173687ba27dccff8e4d39 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Tue, 7 Jul 2015 10:09:11 -0700 Subject: [PATCH 48/59] KAFKA-2304 Supported enabling JMX in Kafka Vagrantfile patch by Stevo Slavic reviewed by Ewen Cheslack-Postava --- Vagrantfile | 7 +++++-- vagrant/broker.sh | 5 +++++ vagrant/zk.sh | 7 ++++++- 3 files changed, 16 insertions(+), 3 deletions(-) diff --git a/Vagrantfile b/Vagrantfile index 55c67dd..1d7cc01 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -22,6 +22,7 @@ VAGRANTFILE_API_VERSION = "2" # General config enable_dns = false +enable_jmx = false num_zookeepers = 1 num_brokers = 3 num_workers = 0 # Generic workers that get the code, but don't start any services @@ -135,7 +136,8 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| 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] + zk_jmx_port = enable_jmx ? (8000 + i).to_s : "" + zookeeper.vm.provision "shell", path: "vagrant/zk.sh", :args => [i.to_s, num_zookeepers, zk_jmx_port] end } @@ -151,7 +153,8 @@ Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| # 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] + kafka_jmx_port = enable_jmx ? (9000 + i).to_s : "" + broker.vm.provision "shell", path: "vagrant/broker.sh", :args => [i.to_s, enable_dns ? name : ip_address, zookeeper_connect, kafka_jmx_port] end } diff --git a/vagrant/broker.sh b/vagrant/broker.sh index 63f2d4f..bc040c9 100644 --- a/vagrant/broker.sh +++ b/vagrant/broker.sh @@ -22,6 +22,7 @@ set -e BROKER_ID=$1 PUBLIC_ADDRESS=$2 PUBLIC_ZOOKEEPER_ADDRESSES=$3 +JMX_PORT=$4 cd /opt/kafka @@ -35,4 +36,8 @@ echo "Killing server" bin/kafka-server-stop.sh || true sleep 5 # Because kafka-server-stop.sh doesn't actually wait echo "Starting server" +if [[ -n $JMX_PORT ]]; then + export JMX_PORT=$JMX_PORT + export KAFKA_JMX_OPTS="-Djava.rmi.server.hostname=$PUBLIC_ADDRESS -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " +fi 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 index 15517f8..6fc4b7c 100644 --- a/vagrant/zk.sh +++ b/vagrant/zk.sh @@ -21,6 +21,7 @@ set -e ZKID=$1 NUM_ZK=$2 +JMX_PORT=$3 cd /opt/kafka @@ -37,6 +38,10 @@ 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 +sleep 5 # Because zookeeper-server-stop.sh doesn't actually wait echo "Starting ZooKeeper" +if [[ -n $JMX_PORT ]]; then + export JMX_PORT=$JMX_PORT + export KAFKA_JMX_OPTS="-Djava.rmi.server.hostname=zk$ZKID -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " +fi bin/zookeeper-server-start.sh config/zookeeper-$ZKID.properties 1>> /tmp/zk.log 2>> /tmp/zk.log & -- 1.7.12.4 From a99f70feb23db9ac4274cad9e8cbbf9934d3d075 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Tue, 7 Jul 2015 12:42:49 -0700 Subject: [PATCH 49/59] KAFKA-2306: add another metric for buffer exhausted; reviewed by Guozhang Wang --- .../java/org/apache/kafka/clients/producer/KafkaProducer.java | 4 ++++ .../kafka/clients/producer/internals/RecordAccumulator.java | 10 ++++++++-- 2 files changed, 12 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 5671a3f..03b8dd2 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 @@ -403,6 +403,10 @@ public class KafkaProducer implements Producer { } catch (InterruptedException e) { this.errors.record(); throw new InterruptException(e); + } catch (BufferExhaustedException e) { + this.errors.record(); + this.metrics.sensor("buffer-exhausted-records").record(); + throw e; } catch (KafkaException e) { this.errors.record(); throw e; 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 87dbd64..a152bd7 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 @@ -21,6 +21,8 @@ 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.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -112,7 +114,6 @@ public final class RecordAccumulator { } 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); Measurable waitingThreads = new Measurable() { public double measure(MetricConfig config, long now) { @@ -120,7 +121,7 @@ public final class RecordAccumulator { } }; 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); Measurable totalBytes = new Measurable() { public double measure(MetricConfig config, long now) { @@ -128,6 +129,7 @@ public final class RecordAccumulator { } }; 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); Measurable availableBytes = new Measurable() { public double measure(MetricConfig config, long now) { @@ -135,6 +137,10 @@ public final class RecordAccumulator { } }; metrics.addMetric(metricName, availableBytes); + + Sensor bufferExhaustedRecordSensor = metrics.sensor("buffer-exhausted-records"); + metricName = new MetricName("buffer-exhausted-rate", metricGrpName, "The average per-second number of record sends that are dropped due to buffer exhaustion", metricTags); + bufferExhaustedRecordSensor.add(metricName, new Rate()); } /** -- 1.7.12.4 From 826276de1eaabfebf2a414c69ebcdff17429016c Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 7 Jul 2015 13:26:00 -0700 Subject: [PATCH 50/59] KAFKA-2317: follow-up of KAFKA1367; reviewed by Guozhang Wang --- core/src/main/scala/kafka/controller/KafkaController.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 09630d0..20f1499 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -342,6 +342,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt */ def onControllerResignation() { // de-register listeners + deregisterIsrChangeNotificationListener() deregisterReassignedPartitionsListener() deregisterPreferredReplicaElectionListener() @@ -891,16 +892,21 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt } } - private def registerReassignedPartitionsListener() = { - zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) - } - private def registerIsrChangeNotificationListener() = { debug("Registering IsrChangeNotificationListener") ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) zkClient.subscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) } + private def deregisterIsrChangeNotificationListener() = { + debug("De-registering IsrChangeNotificationListener") + zkClient.unsubscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) + } + + private def registerReassignedPartitionsListener() = { + zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) + } + private def deregisterReassignedPartitionsListener() = { zkClient.unsubscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) } -- 1.7.12.4 From f13dd8024d5bc1c11587a3b539556ea01e2c84ca Mon Sep 17 00:00:00 2001 From: Onur Karaman Date: Tue, 7 Jul 2015 13:36:55 -0700 Subject: [PATCH 51/59] KAFKA-2313: javadoc fix for KafkaConsumer deserialization; reviewed by Guozhang Wang --- .../org/apache/kafka/clients/consumer/KafkaConsumer.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 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 1f0e515..7aa0760 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 @@ -131,8 +131,8 @@ import static org.apache.kafka.common.utils.Utils.min; * 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"); + * props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + * props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props); * consumer.subscribe("foo", "bar"); * while (true) { @@ -159,8 +159,8 @@ import static org.apache.kafka.common.utils.Utils.min; * 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. + * The deserializer settings specify how to turn bytes into objects. For example, by specifying string deserializers, we + * are saying that our record's key and value will just be simple strings. * *

        Controlling When Messages Are Considered Consumed

        * @@ -183,8 +183,8 @@ import static org.apache.kafka.common.utils.Utils.min; * 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"); + * props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + * props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props); * consumer.subscribe("foo", "bar"); * int commitInterval = 200; -- 1.7.12.4 From add17e001003b04c845e13e3222f67019c227321 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 20 May 2015 18:12:10 -0700 Subject: [PATCH 52/59] KAFKA-2205: First commit --- core/src/main/scala/kafka/admin/AdminUtils.scala | 56 ++++++-- .../src/main/scala/kafka/admin/ConfigCommand.scala | 156 +++++++++++++++++++++ core/src/main/scala/kafka/admin/TopicCommand.scala | 1 + .../kafka/controller/TopicDeletionManager.scala | 5 +- .../main/scala/kafka/server/ConfigHandler.scala | 56 ++++++++ .../scala/kafka/server/DynamicConfigManager.scala | 138 ++++++++++++++++++ core/src/main/scala/kafka/server/KafkaServer.scala | 17 ++- .../scala/kafka/server/TopicConfigManager.scala | 15 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 22 ++- .../test/scala/unit/kafka/admin/AdminTest.scala | 8 +- .../scala/unit/kafka/admin/ConfigCommandTest.scala | 73 ++++++++++ .../scala/unit/kafka/admin/TopicCommandTest.scala | 2 +- .../kafka/server/DynamicConfigChangeTest.scala | 75 +++++++++- 13 files changed, 587 insertions(+), 37 deletions(-) create mode 100644 core/src/main/scala/kafka/admin/ConfigCommand.scala create mode 100644 core/src/main/scala/kafka/server/ConfigHandler.scala create mode 100644 core/src/main/scala/kafka/server/DynamicConfigManager.scala create mode 100644 core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index f06edf4..d0dac2b 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -21,6 +21,7 @@ import kafka.common._ import kafka.cluster.{BrokerEndPoint, Broker} import kafka.log.LogConfig +import kafka.server.ConfigType import kafka.utils._ import kafka.api.{TopicMetadata, PartitionMetadata} @@ -40,10 +41,8 @@ import org.I0Itec.zkclient.exception.ZkNodeExistsException object AdminUtils extends Logging { val rand = new Random - val AdminClientId = "__admin_client" - - val TopicConfigChangeZnodePrefix = "config_change_" + val EntityConfigChangeZnodePrefix = "config_change_" /** * There are 2 goals of replica assignment: @@ -249,7 +248,7 @@ object AdminUtils extends Logging { partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment)) // write out the config if there is any, this isn't transactional with the partition assignments - writeTopicConfig(zkClient, topic, config) + writeEntityConfig(zkClient, ConfigType.Topics, topic, config) // create the partition assignment writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update) @@ -273,7 +272,19 @@ object AdminUtils extends Logging { case e2: Throwable => throw new AdminOperationException(e2.toString) } } - + + /** + * Update the config for a client and create a change notification so the change will propagate to other brokers + * @param zkClient: The ZkClient handle used to write the new config to zookeeper + * @param clientId: The clientId for which configs are being changed + * @param configs: The final set of configs that will be applied to the topic. If any new configs need to be added or + * existing configs need to be deleted, it should be done prior to invoking this API + * + */ + def changeClientIdConfig(zkClient: ZkClient, clientId: String, configs: Properties) { + changeEntityConfig(zkClient, ConfigType.Clients, clientId, configs) + } + /** * Update the config for an existing topic and create a change notification so the change will propagate to other brokers * @param zkClient: The ZkClient handle used to write the new config to zookeeper @@ -285,34 +296,49 @@ object AdminUtils extends Logging { def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties) { if(!topicExists(zkClient, topic)) throw new AdminOperationException("Topic \"%s\" does not exist.".format(topic)) - // remove the topic overrides LogConfig.validate(configs) + changeEntityConfig(zkClient, ConfigType.Topics, topic, configs) + } + private def changeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, configs: Properties) { // write the new config--may not exist if there were previously no overrides - writeTopicConfig(zkClient, topic, configs) - + writeEntityConfig(zkClient, entityType, entityName, configs) + // create the change notification - zkClient.createPersistentSequential(ZkUtils.TopicConfigChangesPath + "/" + TopicConfigChangeZnodePrefix, Json.encode(topic)) + val seqNode = ZkUtils.EntityConfigChangesPath + "/" + EntityConfigChangeZnodePrefix + val content = Json.encode(getConfigChangeZnodeData(entityType, entityName)) + zkClient.createPersistentSequential(seqNode, content) } - + + def getConfigChangeZnodeData(entityType: String, entityName: String) : Map[String, Any] = { + Map("version" -> 1, "entityType" -> entityType, "entityName" -> entityName) + } + /** * Write out the topic config to zk, if there is any */ - private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties) { + private def writeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, config: Properties) { val configMap: mutable.Map[String, String] = { import JavaConversions._ config } val map = Map("version" -> 1, "config" -> configMap) - ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicConfigPath(topic), Json.encode(map)) + ZkUtils.updatePersistentPath(zkClient, ZkUtils.getEntityConfigPath(entityType, entityName), Json.encode(map)) } /** * Read the topic config (if any) from zk */ def fetchTopicConfig(zkClient: ZkClient, topic: String): Properties = { - val str: String = zkClient.readData(ZkUtils.getTopicConfigPath(topic), true) + fetchEntityConfig(zkClient, ConfigType.Topics, topic) + } + + /** + * Read the entity (topic or client) config (if any) from zk + */ + def fetchEntityConfig(zkClient: ZkClient, entityType: String, entity: String): Properties = { + val str: String = zkClient.readData(ZkUtils.getEntityConfigPath(entityType, entity), true) val props = new Properties() if(str != null) { Json.parseFull(str) match { @@ -326,9 +352,9 @@ object AdminUtils extends Logging { configTup match { case (k: String, v: String) => props.setProperty(k, v) - case _ => throw new IllegalArgumentException("Invalid topic config: " + str) + case _ => throw new IllegalArgumentException("Invalid " + entityType + " config: " + str) } - case _ => throw new IllegalArgumentException("Invalid topic config: " + str) + case _ => throw new IllegalArgumentException("Invalid " + entityType + " config: " + str) } case o => throw new IllegalArgumentException("Unexpected value in config: " + str) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala new file mode 100644 index 0000000..9d00363 --- /dev/null +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -0,0 +1,156 @@ +/** + * 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 joptsimple._ +import java.util.Properties +import kafka.server.ConfigType +import kafka.utils.{ZKStringSerializer, CommandLineUtils} +import org.I0Itec.zkclient.ZkClient +import scala.collection._ +import scala.collection.JavaConversions._ +import org.apache.kafka.common.utils.Utils + + +object ConfigCommand { + + def main(args: Array[String]): Unit = { + + val opts = new ConfigCommandOptions(args) + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(opts.parser, "Add/Remove entity (topic/client) configs") + + // should have exactly one action + val actions = Seq(opts.alterOpt, opts.describeOpt).count(opts.options.has _) + if(actions != 1) + CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: --describe, --alter") + + opts.checkArgs() + + val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) + + try { + if(opts.options.has(opts.alterOpt)) + alterConfig(zkClient, opts) + else if(opts.options.has(opts.describeOpt)) 1 + describeConfig(zkClient, opts) + } catch { + case e: Throwable => + println("Error while executing topic command " + e.getMessage) + println(Utils.stackTrace(e)) + } finally { + zkClient.close() + } + } + + def alterConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { + val configsToBeAdded = parseConfigsToBeAdded(opts) + val configsToBeDeleted = parseConfigsToBeDeleted(opts) + val entityType = opts.options.valueOf(opts.entityType) + val entityName = opts.options.valueOf(opts.entityName) + + // compile the final set of configs + val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName) + configs.putAll(configsToBeAdded) + configsToBeDeleted.foreach(config => configs.remove(config)) + + if(entityType.equals(ConfigType.Topics)) { + AdminUtils.changeTopicConfig(zkClient, entityName, configs) + println("Updated config for topic: \"%s\".".format(entityName)) + } else { + AdminUtils.changeClientIdConfig(zkClient, entityName, configs) + println("Updated config for clientId: \"%s\".".format(entityName)) + } + } + + def describeConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { + val entityType = opts.options.valueOf(opts.entityType) + val entityName = opts.options.valueOf(opts.entityName) + val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName) + println("Configs for %s:%s are %s" + .format(entityType, entityName, configs.map(kv => kv._1 + "=" + kv._2).mkString(","))) + } + + def parseConfigsToBeAdded(opts: ConfigCommandOptions): Properties = { + val configsToBeAdded = opts.options.valuesOf(opts.addedConfig).map(_.split("""\s*=\s*""")) + require(configsToBeAdded.forall(config => config.length == 2), + "Invalid entity config: all configs to be added must be in the format \"key=val\".") + val props = new Properties + configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim)) + props + } + + def parseConfigsToBeDeleted(opts: ConfigCommandOptions): Seq[String] = { + if (opts.options.has(opts.deletedConfig)) { + val configsToBeDeleted = opts.options.valuesOf(opts.deletedConfig).map(_.trim()) + val propsToBeDeleted = new Properties + configsToBeDeleted.foreach(propsToBeDeleted.setProperty(_, "")) + configsToBeDeleted + } + else + Seq.empty + } + + class ConfigCommandOptions(args: Array[String]) { + val parser = new OptionParser + 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.") + .withRequiredArg + .describedAs("urls") + .ofType(classOf[String]) + val alterOpt = parser.accepts("alter", "Alter the configuration for the entity.") + val describeOpt = parser.accepts("describe", "List configs for the given entity.") + val entityType = parser.accepts("entityType", "Type of entity (topic/client)") + .withRequiredArg + .ofType(classOf[String]) + val entityName = parser.accepts("entityName", "Name of entity (topic name/client id)") + .withRequiredArg + .ofType(classOf[String]) + val addedConfig = parser.accepts("added-config", "Key Value pairs configs to add 'k1=v1,k2=v2'") + .withRequiredArg + .ofType(classOf[String]) + .withValuesSeparatedBy(',') + val deletedConfig = parser.accepts("deleted-config", "config keys to remove 'k1,k2'") + .withRequiredArg + .ofType(classOf[String]) + .withValuesSeparatedBy(',') + val helpOpt = parser.accepts("help", "Print usage information.") + val options = parser.parse(args : _*) + + val allOpts: Set[OptionSpec[_]] = Set(alterOpt, describeOpt, entityType, entityName, addedConfig, deletedConfig, helpOpt) + + def checkArgs() { + // check required args + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt, entityType, entityName) + CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(describeOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, describeOpt, Set(alterOpt)) + if(options.has(alterOpt)) { + val isAddedPresent: Boolean = options.has(addedConfig) + val isDeletedPresent: Boolean = options.has(deletedConfig) + if(! isAddedPresent && ! isDeletedPresent) + throw new IllegalArgumentException("At least one of --added-config or --deleted-config must be specified with --alter") + } + val entityTypeVal = options.valueOf(entityType) + if(! entityTypeVal.equals(ConfigType.Topics) && ! entityTypeVal.equals(ConfigType.Clients)) { + throw new IllegalArgumentException("entityType must be 'client' or 'topic'") + } + } + } + +} diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index a2ecb96..d5d1962 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -20,6 +20,7 @@ package kafka.admin import joptsimple._ import java.util.Properties import kafka.common.{Topic, AdminCommandFailedException} +import kafka.utils.CommandLineUtils import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala index 64ecb49..bff9d2f 100755 --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -16,6 +16,9 @@ */ package kafka.controller + +import kafka.server.ConfigType + import collection.mutable import kafka.utils.{ShutdownableThread, Logging, ZkUtils} import kafka.utils.CoreUtils._ @@ -284,7 +287,7 @@ class TopicDeletionManager(controller: KafkaController, topicsToBeDeleted -= topic partitionsToBeDeleted.retain(_.topic != topic) controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicPath(topic)) - controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicConfigPath(topic)) + controllerContext.zkClient.deleteRecursive(ZkUtils.getEntityConfigPath(ConfigType.Topics, topic)) controllerContext.zkClient.delete(ZkUtils.getDeleteTopicPath(topic)) controllerContext.removeTopic(topic) } diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala new file mode 100644 index 0000000..e452515 --- /dev/null +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -0,0 +1,56 @@ +package kafka.server + + +import java.util.Properties + +import kafka.log.{LogConfig, LogManager} +import org.apache.kafka.common.metrics.Metrics + + +/** + * The ConfigHandler is used to process config change notifications received by the DynamicConfigManager + */ +trait ConfigHandler { + def processConfigChanges(entityName : String, value : Properties) +} + +/** + * The TopicConfigHandler will process topic config changes in ZK. + * The callback provides the topic name and the full properties set read from ZK + */ +class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandler{ + + def processConfigChanges(topic : String, topicConfig : Properties) { + System.out.println("Received change " + topic + ", config " + topicConfig) + + val logs = logManager.logsByTopicPartition.toBuffer + val logsByTopic = logs.groupBy(_._1.topic).mapValues(_.map(_._2)) + + if (logsByTopic.contains(topic)) + { + System.out.println("Chaning log config") + + /* combine the default properties with the overrides in zk to create the new LogConfig */ + val props = new Properties(logManager.defaultConfig.toProps) + props.putAll(topicConfig) + val logConfig = LogConfig.fromProps(props) + for (log <- logsByTopic(topic)) + log.config = logConfig + } + } +} + +/** + * The ClientIdConfigHandler will process clientId config changes in ZK. + * The callback provides the clientId and the full properties set read from ZK. + * This implementation does nothing currently. In the future, it will change quotas per client + */ +class ClientIdConfigHandler extends ConfigHandler { + @volatile var clientId: String = null + @volatile var clientConfig: Properties = null + + def processConfigChanges(clientId : String, clientConfig : Properties): Unit = { + this.clientId = clientId + this.clientConfig = clientConfig + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala new file mode 100644 index 0000000..93080a9 --- /dev/null +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -0,0 +1,138 @@ +/** + * 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 scala.collection._ +import kafka.utils._ +import kafka.admin.AdminUtils +import org.I0Itec.zkclient.{IZkChildListener, ZkClient} + +object ConfigType { + val Topics = "topics" + val Clients = "clients" +} + +class DynamicConfigManager(private val zkClient: ZkClient, + private val configHandler : Map[String, ConfigHandler], + private val changeExpirationMs: Long = 15*60*1000, + private val time: Time = SystemTime) extends Logging { + private var lastExecutedChange = -1L + + /** + * Begin watching for config changes + */ + def startup() { + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.EntityConfigChangesPath) + zkClient.subscribeChildChanges(ZkUtils.EntityConfigChangesPath, ConfigChangeListener) + processAllConfigChanges() + } + + /** + * Process all config changes + */ + private def processAllConfigChanges() { + val configChanges = zkClient.getChildren(ZkUtils.EntityConfigChangesPath) + import JavaConversions._ + processConfigChanges((configChanges: mutable.Buffer[String]).sorted) + } + + /** + * Process the given list of config changes + */ + private def processConfigChanges(notifications: Seq[String]) { + if (notifications.size > 0) { + info("Processing config change notification(s)...") + val now = time.milliseconds + for (notification <- notifications) { + val changeId = changeNumber(notification) + + if (changeId > lastExecutedChange) { + val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification + + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode) + processNotification(jsonOpt) + } + lastExecutedChange = changeId + } + purgeObsoleteNotifications(now, notifications) + } + } + + def processNotification(jsonOpt: Option[String]) = { + if(jsonOpt.isDefined) { + val json = jsonOpt.get + Json.parseFull(json) match { + case None => // there are no config overrides + case Some(mapAnon: Map[_, _]) => + val map = mapAnon collect + { case (k: String, v: Any) => k -> v } + require(map("version") == 1) + + val entityType = map.get("entityType") match { + case Some(ConfigType.Topics) => ConfigType.Topics + case Some(ConfigType.Clients) => ConfigType.Clients + case _ => throw new IllegalArgumentException("Invalid entityType config. Must be either 'client' or 'topic'") + } + + val entity = map.get("entityName") match { + case Some(value: String) => value + case _ => throw new IllegalArgumentException("Value not specified in config change notification " + json) + } + if (configHandler.contains(entityType)) + configHandler(entityType).processConfigChanges(entity, AdminUtils.fetchEntityConfig(zkClient, entityType, entity)) + else + warn("Cannot process config changes for entity " + entity) + + case o => throw new IllegalArgumentException("Unexpected value in config: " + o) + } + } + } + + private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { + for(notification <- notifications.sorted) { + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.EntityConfigChangesPath + "/" + notification) + if(jsonOpt.isDefined) { + val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification + if (now - stat.getCtime > changeExpirationMs) { + debug("Purging config change notification " + notification) + ZkUtils.deletePath(zkClient, changeZnode) + } else { + return + } + } + } + } + + /* get the change number from a change notification znode */ + private def changeNumber(name: String): Long = name.substring(AdminUtils.EntityConfigChangeZnodePrefix.length).toLong + + /** + * A listener that applies config changes to logs + */ + object ConfigChangeListener extends IZkChildListener { + override def handleChildChange(path: String, chillins: java.util.List[String]) { + try { + import JavaConversions._ + processConfigChanges(chillins: mutable.Buffer[String]) + } catch { + case e: Exception => error("Error processing config change:", e) + } + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 18917bc..4ec2314 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -31,6 +31,7 @@ import java.io.File import kafka.utils._ import org.apache.kafka.common.metrics._ import org.apache.kafka.common.network.NetworkReceive +import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import scala.collection.{JavaConversions, mutable} import org.I0Itec.zkclient.ZkClient @@ -77,7 +78,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var replicaManager: ReplicaManager = null - var topicConfigManager: TopicConfigManager = null + var dynamicConfigHandlers: Map[String, ConfigHandler] = null + var dynamicConfigManager: DynamicConfigManager = null + val metrics: Metrics = new Metrics() var consumerCoordinator: ConsumerCoordinator = null @@ -171,9 +174,11 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg Mx4jLoader.maybeLoad() - /* start topic config manager */ - topicConfigManager = new TopicConfigManager(zkClient, logManager) - topicConfigManager.startup() + /* start dynamic config manager */ + dynamicConfigHandlers = Map[String, ConfigHandler](ConfigType.Topics -> new TopicConfigHandler(logManager), + ConfigType.Clients -> new ClientIdConfigHandler) + dynamicConfigManager = new DynamicConfigManager(zkClient, dynamicConfigHandlers) + dynamicConfigManager.startup() /* tell everyone we are alive */ val listeners = config.advertisedListeners.map {case(protocol, endpoint) => @@ -203,6 +208,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg } } + private def initMetrics() : Metrics = { + new Metrics() + } + private def initZk(): ZkClient = { info("Connecting to zookeeper on " + config.zkConnect) diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala index 01b1b0a..5d157ed 100644 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ b/core/src/main/scala/kafka/server/TopicConfigManager.scala @@ -68,8 +68,8 @@ class TopicConfigManager(private val zkClient: ZkClient, * Begin watching for config changes */ def startup() { - ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.TopicConfigChangesPath) - zkClient.subscribeChildChanges(ZkUtils.TopicConfigChangesPath, ConfigChangeListener) + ZkUtils.makeSurePersistentPathExists(zkClient, null)//ZkUtils.TopicConfigChangesPath) + //zkClient.subscribeChildChanges(ZkUtils.TopicConfigChangesPath, ConfigChangeListener) processAllConfigChanges() } @@ -77,7 +77,8 @@ class TopicConfigManager(private val zkClient: ZkClient, * Process all config changes */ private def processAllConfigChanges() { - val configChanges = zkClient.getChildren(ZkUtils.TopicConfigChangesPath) + //val configChanges = zkClient.getChildren(ZkUtils.TopicConfigChangesPath) + val configChanges = zkClient.getChildren(null) import JavaConversions._ processConfigChanges((configChanges: mutable.Buffer[String]).sorted) } @@ -94,7 +95,7 @@ class TopicConfigManager(private val zkClient: ZkClient, for (notification <- notifications) { val changeId = changeNumber(notification) if (changeId > lastExecutedChange) { - val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification + val changeZnode = null //ZkUtils.ATopicConfigChangesPath //+ "/" //+ notification val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode) if(jsonOpt.isDefined) { val json = jsonOpt.get @@ -119,9 +120,9 @@ class TopicConfigManager(private val zkClient: ZkClient, private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { for(notification <- notifications.sorted) { - val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.TopicConfigChangesPath + "/" + notification) + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, null) //ZkUtils.TopicConfigChangesPath + "/" + notification) if(jsonOpt.isDefined) { - val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification + val changeZnode = null //ZkUtils.TopicConfigChangesPath + "/" + notification if (now - stat.getCtime > changeExpirationMs) { debug("Purging config change notification " + notification) ZkUtils.deletePath(zkClient, changeZnode) @@ -133,7 +134,7 @@ class TopicConfigManager(private val zkClient: ZkClient, } /* get the change number from a change notification znode */ - private def changeNumber(name: String): Long = name.substring(AdminUtils.TopicConfigChangeZnodePrefix.length).toLong + private def changeNumber(name: String): Long = 1 //name.substring(AdminUtils.TopicConfigChangeZnodePrefix.length).toLong /** * A listener that applies config changes to logs diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 166814c..c5a7559 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -19,6 +19,7 @@ package kafka.utils import kafka.cluster._ import kafka.consumer.{ConsumerThreadId, TopicCount} +import kafka.server.ConfigType import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, ZkMarshallingError, ZkBadVersionException} @@ -39,8 +40,6 @@ object ZkUtils extends Logging { val ConsumersPath = "/consumers" val BrokerIdsPath = "/brokers/ids" val BrokerTopicsPath = "/brokers/topics" - val TopicConfigPath = "/config/topics" - val TopicConfigChangesPath = "/config/changes" val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" val ReassignPartitionsPath = "/admin/reassign_partitions" @@ -48,6 +47,8 @@ object ZkUtils extends Logging { val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" val BrokerSequenceIdPath = "/brokers/seqid" val IsrChangeNotificationPath = "/isr_change_notification" + val EntityConfigPath = "/config" + val EntityConfigChangesPath = "/config/changes" def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic @@ -57,8 +58,11 @@ object ZkUtils extends Logging { getTopicPath(topic) + "/partitions" } - def getTopicConfigPath(topic: String): String = - TopicConfigPath + "/" + topic + def getEntityConfigRootPath(entityType: String): String = + EntityConfigPath + "/" + entityType + + def getEntityConfigPath(entityType: String, entity: String): String = + getEntityConfigRootPath(entityType) + "/" + entity def getDeleteTopicPath(topic: String): String = DeleteTopicsPath + "/" + topic @@ -93,8 +97,14 @@ object ZkUtils extends Logging { } def setupCommonPaths(zkClient: ZkClient) { - for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath, - DeleteTopicsPath, BrokerSequenceIdPath)) + for(path <- Seq(ConsumersPath, + BrokerIdsPath, + BrokerTopicsPath, + EntityConfigChangesPath, + ZkUtils.getEntityConfigRootPath(ConfigType.Topics), + ZkUtils.getEntityConfigRootPath(ConfigType.Clients), + DeleteTopicsPath, + BrokerSequenceIdPath)) makeSurePersistentPathExists(zkClient, path) } diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index 252ac81..499c8d0 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -25,7 +25,7 @@ import kafka.log._ import kafka.zk.ZooKeeperTestHarness import kafka.utils.{Logging, ZkUtils, TestUtils} import kafka.common.{TopicExistsException, TopicAndPartition} -import kafka.server.{KafkaServer, KafkaConfig} +import kafka.server.{ConfigType, KafkaServer, KafkaConfig} import java.io.File import TestUtils._ @@ -393,12 +393,16 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { checkConfig(maxMessageSize, retentionMs) // now double the config values for the topic and check that it is applied + val newConfig: Properties = makeConfig(2*maxMessageSize, 2 * retentionMs) AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs)) checkConfig(2*maxMessageSize, 2 * retentionMs) + + // Verify that the same config can be read from ZK + val configInZk = AdminUtils.fetchEntityConfig(server.zkClient, ConfigType.Topics, topic) + assertEquals(newConfig, configInZk) } finally { server.shutdown() server.config.logDirs.foreach(CoreUtils.rm(_)) } } - } diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala new file mode 100644 index 0000000..11cb35b --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.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.admin + +import junit.framework.Assert._ +import kafka.admin.ConfigCommand.ConfigCommandOptions +import org.junit.Test +import org.scalatest.junit.JUnit3Suite +import kafka.utils.Logging +import kafka.utils.TestUtils +import kafka.zk.ZooKeeperTestHarness +import kafka.server.{ConfigType, OffsetManager, KafkaConfig} +import kafka.admin.TopicCommand.TopicCommandOptions +import kafka.utils.ZkUtils + +class ConfigCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { + @Test + def testArgumentParse() { + // Should parse correctly + var createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entityName", "x", + "--entityType", "clients", + "--describe")) + createOpts.checkArgs() + + // For --alter and added config + createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entityName", "x", + "--entityType", "clients", + "--alter", + "--added-config", "a=b,c=d")) + createOpts.checkArgs() + + // For alter and deleted config + createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entityName", "x", + "--entityType", "clients", + "--alter", + "--deleted-config", "a,b,c")) + createOpts.checkArgs() + + // For alter and both added, deleted config + createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, + "--entityName", "x", + "--entityType", "clients", + "--alter", + "--added-config", "a=b,c=d", + "--deleted-config", "a")) + createOpts.checkArgs() + val addedProps = ConfigCommand.parseConfigsToBeAdded(createOpts) + assertEquals(2, addedProps.size()) + assertEquals("b", addedProps.getProperty("a")) + assertEquals("d", addedProps.getProperty("c")) + + val deletedProps = ConfigCommand.parseConfigsToBeDeleted(createOpts) + assertEquals(1, deletedProps.size) + assertEquals("a", deletedProps(0)) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index dcd6988..62918c3 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -48,7 +48,7 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal)) // pre-create the topic config changes path to avoid a NoNodeException - ZkUtils.createPersistentPath(zkClient, ZkUtils.TopicConfigChangesPath) + ZkUtils.createPersistentPath(zkClient, ZkUtils.EntityConfigChangesPath) // modify the topic to add new partitions val numPartitionsModified = 3 diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 8a871cf..5719e38 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -19,6 +19,8 @@ package kafka.server import java.util.Properties import junit.framework.Assert._ +import org.I0Itec.zkclient.ZkClient +import org.easymock.{Capture, EasyMock} import org.junit.Test import kafka.integration.KafkaServerTestHarness import kafka.utils._ @@ -32,6 +34,8 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testConfigChange() { + assertTrue("Should contain a ConfigHandler for topics", + this.servers(0).dynamicConfigHandlers.contains(ConfigType.Topics)) val oldVal: java.lang.Long = 100000 val newVal: java.lang.Long = 200000 val tp = TopicAndPartition("test", 0) @@ -50,6 +54,24 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { } } + // For now client config changes do not do anything. Simply verify that the call was made + @Test + def testClientConfigChange() { + assertTrue("Should contain a ConfigHandler for topics", + this.servers(0).dynamicConfigHandlers.contains(ConfigType.Clients)) + val clientId = "testClient" + val props = new Properties() + props.put("a.b", "c") + props.put("x.y", "z") + AdminUtils.changeClientIdConfig(zkClient, clientId, props) + TestUtils.retry(10000) { + val configHandler = this.servers(0).dynamicConfigHandlers(ConfigType.Clients).asInstanceOf[ClientIdConfigHandler] + assertEquals("ClientId should be set to testClient", clientId, configHandler.clientId) + assertEquals("c", configHandler.clientConfig.getProperty("a.b")) + assertEquals("z", configHandler.clientConfig.getProperty("x.y")) + } + } + @Test def testConfigChangeOnNonExistingTopic() { val topic = TestUtils.tempTopic @@ -63,4 +85,55 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { } } -} \ No newline at end of file + @Test + def testProcessNotification { + val props = new Properties() + props.put("a.b", "10") + // create nice mock since we don't particularly care about zkclient calls + val entityArgument = new Capture[String]() + val propertiesArgument = new Capture[Properties]() + val handler = EasyMock.createNiceMock(classOf[ConfigHandler]) + handler.processConfigChanges( + EasyMock.and(EasyMock.capture(entityArgument), EasyMock.isA(classOf[String])), + EasyMock.and(EasyMock.capture(propertiesArgument), EasyMock.isA(classOf[Properties]))) + EasyMock.expectLastCall() + EasyMock.replay(handler) + + val configManager = new DynamicConfigManager(zkClient, Map("Topic" -> handler)) + // Is ignored + configManager.processNotification(Some("not json")) + + // Incorrect Map. No version + try { + val jsonMap = Map("v" -> 1, "x" -> 2) + configManager.processNotification(Some(Json.encode(jsonMap))) + fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) + } + catch { + case t: Throwable => + } + // Version is provided. EntityType is incorrect + try { + val jsonMap = Map("version" -> 1, "entityType" -> "garbage", "entityName" -> "x") + configManager.processNotification(Some(Json.encode(jsonMap))) + fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) + } + catch { + case t: Throwable => + } + + // EntityName isn't provided + try { + val jsonMap = Map("version" -> 1, "entityType" -> ConfigType.Clients) + configManager.processNotification(Some(Json.encode(jsonMap))) + fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) + } + catch { + case t: Throwable => + } + + // Everything is provided + val jsonMap = Map("version" -> 1, "entityType" -> ConfigType.Clients, "entityName" -> "x") + configManager.processNotification(Some(Json.encode(jsonMap))) + } +} -- 1.7.12.4 From 78c5715bc459527fb0528d3a9f7660f096125221 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 21 May 2015 09:32:09 -0700 Subject: [PATCH 53/59] Some fixes --- core/src/main/scala/kafka/admin/AdminUtils.scala | 9 +-------- core/src/main/scala/kafka/admin/ConfigCommand.scala | 3 +++ core/src/main/scala/kafka/admin/TopicCommand.scala | 5 +++-- core/src/main/scala/kafka/cluster/Partition.scala | 5 +++++ core/src/main/scala/kafka/controller/KafkaController.scala | 4 ++-- .../main/scala/kafka/controller/PartitionLeaderSelector.scala | 6 +++--- core/src/main/scala/kafka/server/ReplicaFetcherThread.scala | 4 ++-- core/src/main/scala/kafka/server/TopicConfigManager.scala | 2 +- core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala | 7 ++++--- 9 files changed, 24 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index d0dac2b..4ddff75 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -328,13 +328,6 @@ object AdminUtils extends Logging { } /** - * Read the topic config (if any) from zk - */ - def fetchTopicConfig(zkClient: ZkClient, topic: String): Properties = { - fetchEntityConfig(zkClient, ConfigType.Topics, topic) - } - - /** * Read the entity (topic or client) config (if any) from zk */ def fetchEntityConfig(zkClient: ZkClient, entityType: String, entity: String): Properties = { @@ -364,7 +357,7 @@ object AdminUtils extends Logging { } def fetchAllTopicConfigs(zkClient: ZkClient): Map[String, Properties] = - ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchTopicConfig(zkClient, topic))).toMap + ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchEntityConfig(zkClient, ConfigType.Topics, topic))).toMap def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient): TopicMetadata = fetchTopicMetadataFromZk(topic, zkClient, new mutable.HashMap[Int, Broker]) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 9d00363..76a445d 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -27,6 +27,9 @@ import scala.collection.JavaConversions._ import org.apache.kafka.common.utils.Utils +/** + * This script can be used to change configs for topics/clients dynamically + */ object ConfigCommand { def main(args: Array[String]): Unit = { diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index d5d1962..859f13c 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -28,6 +28,7 @@ import scala.collection._ import scala.collection.JavaConversions._ import kafka.log.LogConfig import kafka.consumer.Whitelist +import kafka.server.{ConfigType, OffsetManager} import org.apache.kafka.common.utils.Utils import kafka.coordinator.ConsumerCoordinator @@ -101,7 +102,7 @@ object TopicCommand { println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt))) } topics.foreach { topic => - val configs = AdminUtils.fetchTopicConfig(zkClient, topic) + val configs = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topics, topic) if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) { val configsToBeAdded = parseTopicConfigsToBeAdded(opts) val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts) @@ -174,7 +175,7 @@ object TopicCommand { val describePartitions: Boolean = !reportOverriddenConfigs val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) if (describeConfigs) { - val configs = AdminUtils.fetchTopicConfig(zkClient, topic) + val configs = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topics, topic) if (!reportOverriddenConfigs || configs.size() != 0) { val numPartitions = topicPartitionAssignment.size val replicationFactor = topicPartitionAssignment.head._2.size diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 2649090..3f3f816 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -86,7 +86,12 @@ class Partition(val topic: String, case Some(replica) => replica case None => if (isReplicaLocal(replicaId)) { +<<<<<<< HEAD val config = LogConfig.fromProps(logManager.defaultConfig.originals, AdminUtils.fetchTopicConfig(zkClient, topic)) +======= + val config = LogConfig.fromProps(logManager.defaultConfig.toProps, + AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topics, topic)) +>>>>>>> Some fixes val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) val offsetMap = checkpoint.read diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 20f1499..a1189d7 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1028,8 +1028,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt // if the replica to be removed from the ISR is the last surviving member of the ISR and unclean leader election // is disallowed for the corresponding topic, then we must preserve the ISR membership so that the replica can // eventually be restored as the leader. - if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(zkClient, - topicAndPartition.topic)).uncleanLeaderElectionEnable) { + if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(zkClient, + ConfigType.Topics, topicAndPartition.topic)).uncleanLeaderElectionEnable) { info("Retaining last ISR %d of partition %s since unclean leader election is disabled".format(replicaId, topicAndPartition)) newIsr = leaderAndIsr.isr } diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index bb6b5c8..d7bcb18 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -21,7 +21,7 @@ import kafka.api.LeaderAndIsr import kafka.log.LogConfig import kafka.utils.Logging import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException} -import kafka.server.KafkaConfig +import kafka.server.{ConfigType, KafkaConfig} trait PartitionLeaderSelector { @@ -61,8 +61,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi case true => // Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration // for unclean leader election. - if (!LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(controllerContext.zkClient, - topicAndPartition.topic)).uncleanLeaderElectionEnable) { + if (!LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(controllerContext.zkClient, + ConfigType.Topics, topicAndPartition.topic)).uncleanLeaderElectionEnable) { throw new NoReplicaOnlineException(("No broker in ISR for partition " + "%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) + " ISR brokers are: [%s]".format(currentLeaderAndIsr.isr.mkString(","))) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index c89d00b..fe3c376 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -90,8 +90,8 @@ class ReplicaFetcherThread(name:String, // Prior to truncating the follower's log, ensure that doing so is not disallowed by the configuration for unclean leader election. // This situation could only happen if the unclean election configuration for a topic changes while a replica is down. Otherwise, // we should never encounter this situation since a non-ISR leader cannot be elected if disallowed by the broker configuration. - if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, - topicAndPartition.topic)).uncleanLeaderElectionEnable) { + if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchEntityConfig(replicaMgr.zkClient, + ConfigType.Topics, topicAndPartition.topic)).uncleanLeaderElectionEnable) { // Log a fatal error and shutdown the broker to ensure that data loss does not unexpectedly occur. fatal("Halting because log truncation is not allowed for topic %s,".format(topicAndPartition.topic) + " Current leader %d's latest offset %d is less than replica %d's latest offset %d" diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala index 5d157ed..0651398 100644 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ b/core/src/main/scala/kafka/server/TopicConfigManager.scala @@ -150,4 +150,4 @@ class TopicConfigManager(private val zkClient: ZkClient, } } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index 62918c3..20e5651 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -22,6 +22,7 @@ import org.scalatest.junit.JUnit3Suite import kafka.utils.Logging import kafka.utils.TestUtils import kafka.zk.ZooKeeperTestHarness +import kafka.server.{ConfigType, OffsetManager, KafkaConfig} import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils import kafka.coordinator.ConsumerCoordinator @@ -43,7 +44,7 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin "--config", cleanupKey + "=" + cleanupVal, "--topic", topic)) TopicCommand.createTopic(zkClient, createOpts) - val props = AdminUtils.fetchTopicConfig(zkClient, topic) + val props = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topics, topic) assertTrue("Properties after creation don't contain " + cleanupKey, props.containsKey(cleanupKey)) assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal)) @@ -56,7 +57,7 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin "--config", cleanupKey + "=" + cleanupVal, "--topic", topic)) TopicCommand.alterTopic(zkClient, alterOpts) - val newProps = AdminUtils.fetchTopicConfig(zkClient, topic) + val newProps = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topics, topic) assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey)) assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal)) } @@ -99,4 +100,4 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin } assertFalse("Delete path for topic shouldn't exist after deletion.", zkClient.exists(deleteOffsetTopicPath)) } -} \ No newline at end of file +} -- 1.7.12.4 From 5df6c2f259c624047400c690702f81d370422b78 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 21 May 2015 10:37:55 -0700 Subject: [PATCH 54/59] KAFKA-2205 --- .../main/scala/kafka/server/ConfigHandler.scala | 2 - .../scala/kafka/server/DynamicConfigManager.scala | 44 +++++- .../scala/kafka/server/TopicConfigManager.scala | 153 --------------------- 3 files changed, 43 insertions(+), 156 deletions(-) delete mode 100644 core/src/main/scala/kafka/server/TopicConfigManager.scala diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index e452515..aa00487 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -4,8 +4,6 @@ package kafka.server import java.util.Properties import kafka.log.{LogConfig, LogManager} -import org.apache.kafka.common.metrics.Metrics - /** * The ConfigHandler is used to process config change notifications received by the DynamicConfigManager diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index 93080a9..c6fc0ec 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -17,17 +17,59 @@ package kafka.server +import kafka.utils.Json +import kafka.utils.Logging +import kafka.utils.SystemTime +import kafka.utils.Time +import kafka.utils.ZkUtils import scala.collection._ -import kafka.utils._ import kafka.admin.AdminUtils import org.I0Itec.zkclient.{IZkChildListener, ZkClient} + +/** + * Represents all the entities that can be configured via ZK + */ object ConfigType { val Topics = "topics" val Clients = "clients" } +/** + * This class initiates and carries out config changes for all entities defined in ConfigType. + * + * It works as follows. + * + * Config is stored under the path: /config/entityType/entityName + * E.g. /config/topics/ and /config/clients/ + * This znode stores the overrides for this entity (but no defaults) in properties format. + * + * To avoid watching all topics for changes instead we have a notification path + * /config/changes + * The DynamicConfigManager has a child watch on this path. + * + * To update a config we first update the config properties. Then we create a new sequential + * znode under the change path which contains the name of the entityType and entityName that was updated, say + * /config/changes/config_change_13321 + * The sequential znode contains data in this format: {"version" : 1, "entityType":"topic/client", "entityName" : "topic_name/client_id"} + * This is just a notification--the actual config change is stored only once under the /config/entityType/entityName path. + * + * This will fire a watcher on all brokers. This watcher works as follows. It reads all the config change notifications. + * It keeps track of the highest config change suffix number it has applied previously. For any previously applied change it finds + * it checks if this notification is larger than a static expiration time (say 10mins) and if so it deletes this notification. + * For any new changes it reads the new configuration, combines it with the defaults, and updates the existing config. + * + * Note that config is always read from the config path in zk, the notification is just a trigger to do so. So if a broker is + * down and misses a change that is fine--when it restarts it will be loading the full config anyway. Note also that + * if there are two consecutive config changes it is possible that only the last one will be applied (since by the time the + * broker reads the config the both changes may have been made). In this case the broker would needlessly refresh the config twice, + * but that is harmless. + * + * On restart the config manager re-processes all notifications. This will usually be wasted work, but avoids any race conditions + * on startup where a change might be missed between the initial config load and registering for change notifications. + * + */ class DynamicConfigManager(private val zkClient: ZkClient, private val configHandler : Map[String, ConfigHandler], private val changeExpirationMs: Long = 15*60*1000, diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala deleted file mode 100644 index 0651398..0000000 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ /dev/null @@ -1,153 +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 java.util.Properties -import scala.collection._ -import kafka.log._ -import kafka.utils._ -import kafka.admin.AdminUtils -import org.I0Itec.zkclient.{IZkChildListener, ZkClient} - -/** - * This class initiates and carries out topic config changes. - * - * It works as follows. - * - * Config is stored under the path - * /config/topics/ - * This znode stores the topic-overrides for this topic (but no defaults) in properties format. - * - * To avoid watching all topics for changes instead we have a notification path - * /config/changes - * The TopicConfigManager has a child watch on this path. - * - * To update a topic config we first update the topic config properties. Then we create a new sequential - * znode under the change path which contains the name of the topic that was updated, say - * /config/changes/config_change_13321 - * This is just a notification--the actual config change is stored only once under the /config/topics/ path. - * - * This will fire a watcher on all brokers. This watcher works as follows. It reads all the config change notifications. - * It keeps track of the highest config change suffix number it has applied previously. For any previously applied change it finds - * it checks if this notification is larger than a static expiration time (say 10mins) and if so it deletes this notification. - * For any new changes it reads the new configuration, combines it with the defaults, and updates the log config - * for all logs for that topic (if any) that it has. - * - * Note that config is always read from the config path in zk, the notification is just a trigger to do so. So if a broker is - * down and misses a change that is fine--when it restarts it will be loading the full config anyway. Note also that - * if there are two consecutive config changes it is possible that only the last one will be applied (since by the time the - * broker reads the config the both changes may have been made). In this case the broker would needlessly refresh the config twice, - * but that is harmless. - * - * On restart the config manager re-processes all notifications. This will usually be wasted work, but avoids any race conditions - * on startup where a change might be missed between the initial config load and registering for change notifications. - * - */ -class TopicConfigManager(private val zkClient: ZkClient, - private val logManager: LogManager, - private val changeExpirationMs: Long = 15*60*1000, - private val time: Time = SystemTime) extends Logging { - private var lastExecutedChange = -1L - - /** - * Begin watching for config changes - */ - def startup() { - ZkUtils.makeSurePersistentPathExists(zkClient, null)//ZkUtils.TopicConfigChangesPath) - //zkClient.subscribeChildChanges(ZkUtils.TopicConfigChangesPath, ConfigChangeListener) - processAllConfigChanges() - } - - /** - * Process all config changes - */ - private def processAllConfigChanges() { - //val configChanges = zkClient.getChildren(ZkUtils.TopicConfigChangesPath) - val configChanges = zkClient.getChildren(null) - import JavaConversions._ - processConfigChanges((configChanges: mutable.Buffer[String]).sorted) - } - - /** - * Process the given list of config changes - */ - private def processConfigChanges(notifications: Seq[String]) { - if (notifications.size > 0) { - info("Processing config change notification(s)...") - val now = time.milliseconds - val logs = logManager.logsByTopicPartition.toBuffer - val logsByTopic = logs.groupBy(_._1.topic).mapValues(_.map(_._2)) - for (notification <- notifications) { - val changeId = changeNumber(notification) - if (changeId > lastExecutedChange) { - val changeZnode = null //ZkUtils.ATopicConfigChangesPath //+ "/" //+ notification - val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode) - if(jsonOpt.isDefined) { - val json = jsonOpt.get - val topic = json.substring(1, json.length - 1) // hacky way to dequote - if (logsByTopic.contains(topic)) { - /* combine the default properties with the overrides in zk to create the new LogConfig */ - val props = new Properties() - props.putAll(logManager.defaultConfig.originals) - props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) - val logConfig = LogConfig(props) - for (log <- logsByTopic(topic)) - log.config = logConfig - info("Processed topic config change %d for topic %s, setting new config to %s.".format(changeId, topic, props)) - purgeObsoleteNotifications(now, notifications) - } - } - lastExecutedChange = changeId - } - } - } - } - - private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { - for(notification <- notifications.sorted) { - val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, null) //ZkUtils.TopicConfigChangesPath + "/" + notification) - if(jsonOpt.isDefined) { - val changeZnode = null //ZkUtils.TopicConfigChangesPath + "/" + notification - if (now - stat.getCtime > changeExpirationMs) { - debug("Purging config change notification " + notification) - ZkUtils.deletePath(zkClient, changeZnode) - } else { - return - } - } - } - } - - /* get the change number from a change notification znode */ - private def changeNumber(name: String): Long = 1 //name.substring(AdminUtils.TopicConfigChangeZnodePrefix.length).toLong - - /** - * A listener that applies config changes to logs - */ - object ConfigChangeListener extends IZkChildListener { - override def handleChildChange(path: String, chillins: java.util.List[String]) { - try { - import JavaConversions._ - processConfigChanges(chillins: mutable.Buffer[String]) - } catch { - case e: Exception => error("Error processing config change:", e) - } - } - } - -} -- 1.7.12.4 From df1952c405b15ff8c8b9c9e2660cf763e8cc7ba0 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 21 May 2015 10:49:06 -0700 Subject: [PATCH 55/59] KAFKA-2205 --- core/src/main/scala/kafka/server/ConfigHandler.scala | 4 ---- core/src/main/scala/kafka/server/KafkaServer.scala | 4 ---- 2 files changed, 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index aa00487..1ae056e 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -19,15 +19,11 @@ trait ConfigHandler { class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandler{ def processConfigChanges(topic : String, topicConfig : Properties) { - System.out.println("Received change " + topic + ", config " + topicConfig) - val logs = logManager.logsByTopicPartition.toBuffer val logsByTopic = logs.groupBy(_._1.topic).mapValues(_.map(_._2)) if (logsByTopic.contains(topic)) { - System.out.println("Chaning log config") - /* combine the default properties with the overrides in zk to create the new LogConfig */ val props = new Properties(logManager.defaultConfig.toProps) props.putAll(topicConfig) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 4ec2314..cf7c46e 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -208,10 +208,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg } } - private def initMetrics() : Metrics = { - new Metrics() - } - private def initZk(): ZkClient = { info("Connecting to zookeeper on " + config.zkConnect) -- 1.7.12.4 From 0dfc2e0917c50db2506777528fc9b493dc9707d6 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 1 Jul 2015 18:37:19 -0700 Subject: [PATCH 56/59] Addressing Jun's comments --- core/src/main/scala/kafka/admin/AdminUtils.scala | 30 ++++++++++++---------- .../src/main/scala/kafka/admin/ConfigCommand.scala | 16 ++++++------ core/src/main/scala/kafka/admin/TopicCommand.scala | 16 +++--------- .../main/scala/kafka/server/ConfigHandler.scala | 22 +++++++++------- .../scala/kafka/server/DynamicConfigManager.scala | 7 ++--- .../scala/unit/kafka/admin/ConfigCommandTest.scala | 16 ++++++------ .../scala/unit/kafka/admin/TopicCommandTest.scala | 4 +-- .../kafka/server/DynamicConfigChangeTest.scala | 18 ++++++------- 8 files changed, 63 insertions(+), 66 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 4ddff75..c58b7e6 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -102,14 +102,12 @@ object AdminUtils extends Logging { * @param numPartitions Number of partitions to be set * @param replicaAssignmentStr Manual replica assignment * @param checkBrokerAvailable Ignore checking if assigned replica broker is available. Only used for testing - * @param config Pre-existing properties that should be preserved */ def addPartitions(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "", - checkBrokerAvailable: Boolean = true, - config: Properties = new Properties) { + checkBrokerAvailable: Boolean = true) { val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) if (existingPartitionsReplicaList.size == 0) throw new AdminOperationException("The topic %s does not exist".format(topic)) @@ -136,7 +134,7 @@ object AdminUtils extends Logging { val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2) // add the new list partitionReplicaList ++= newPartitionReplicaList - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true) } def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = { @@ -231,7 +229,7 @@ object AdminUtils extends Logging { val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig) } - + def createOrUpdateTopicPartitionAssignmentPathInZK(zkClient: ZkClient, topic: String, partitionReplicaAssignment: Map[Int, Seq[Int]], @@ -239,21 +237,24 @@ object AdminUtils extends Logging { update: Boolean = false) { // validate arguments Topic.validate(topic) - LogConfig.validate(config) require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.") val topicPath = ZkUtils.getTopicPath(topic) - if(!update && zkClient.exists(topicPath)) + if (!update && zkClient.exists(topicPath)) throw new TopicExistsException("Topic \"%s\" already exists.".format(topic)) partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment)) - - // write out the config if there is any, this isn't transactional with the partition assignments - writeEntityConfig(zkClient, ConfigType.Topics, topic, config) - + + // Configs only matter if a topic is being created. Changing configs via AlterTopic is not supported + if (!update) { + // write out the config if there is any, this isn't transactional with the partition assignments + LogConfig.validate(config) + writeEntityConfig(zkClient, ConfigType.Topics, topic, config) + } + // create the partition assignment writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update) } - + private def writeTopicPartitionAssignment(zkClient: ZkClient, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) { try { val zkPath = ZkUtils.getTopicPath(topic) @@ -312,7 +313,7 @@ object AdminUtils extends Logging { } def getConfigChangeZnodeData(entityType: String, entityName: String) : Map[String, Any] = { - Map("version" -> 1, "entityType" -> entityType, "entityName" -> entityName) + Map("version" -> 1, "entity_type" -> entityType, "entity_name" -> entityName) } /** @@ -350,7 +351,8 @@ object AdminUtils extends Logging { case _ => throw new IllegalArgumentException("Invalid " + entityType + " config: " + str) } - case o => throw new IllegalArgumentException("Unexpected value in config: " + str) + case o => throw new IllegalArgumentException("Unexpected value in config:(%s), entity_type: (%s), entity: (%s)" + .format(str, entityType, entity)) } } props diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 76a445d..ccac1b8 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -39,11 +39,6 @@ object ConfigCommand { if(args.length == 0) CommandLineUtils.printUsageAndDie(opts.parser, "Add/Remove entity (topic/client) configs") - // should have exactly one action - val actions = Seq(opts.alterOpt, opts.describeOpt).count(opts.options.has _) - if(actions != 1) - CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: --describe, --alter") - opts.checkArgs() val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) @@ -119,10 +114,10 @@ object ConfigCommand { .ofType(classOf[String]) val alterOpt = parser.accepts("alter", "Alter the configuration for the entity.") val describeOpt = parser.accepts("describe", "List configs for the given entity.") - val entityType = parser.accepts("entityType", "Type of entity (topic/client)") + val entityType = parser.accepts("entity-type", "Type of entity (topic/client)") .withRequiredArg .ofType(classOf[String]) - val entityName = parser.accepts("entityName", "Name of entity (topic name/client id)") + val entityName = parser.accepts("entity-name", "Name of entity (topic name/client id)") .withRequiredArg .ofType(classOf[String]) val addedConfig = parser.accepts("added-config", "Key Value pairs configs to add 'k1=v1,k2=v2'") @@ -139,10 +134,15 @@ object ConfigCommand { val allOpts: Set[OptionSpec[_]] = Set(alterOpt, describeOpt, entityType, entityName, addedConfig, deletedConfig, helpOpt) def checkArgs() { + // should have exactly one action + val actions = Seq(alterOpt, describeOpt).count(options.has _) + if(actions != 1) + CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --describe, --alter") + // check required args CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt, entityType, entityName) CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(describeOpt)) - CommandLineUtils.checkInvalidArgs(parser, options, describeOpt, Set(alterOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, describeOpt, Set(alterOpt, addedConfig, deletedConfig)) if(options.has(alterOpt)) { val isAddedPresent: Boolean = options.has(addedConfig) val isDeletedPresent: Boolean = options.has(deletedConfig) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 859f13c..84bc2b0 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -102,16 +102,6 @@ object TopicCommand { println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt))) } topics.foreach { topic => - val configs = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topics, topic) - if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) { - val configsToBeAdded = parseTopicConfigsToBeAdded(opts) - val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts) - // compile the final set of configs - configs.putAll(configsToBeAdded) - configsToBeDeleted.foreach(config => configs.remove(config)) - AdminUtils.changeTopicConfig(zkClient, topic, configs) - println("Updated config for topic \"%s\".".format(topic)) - } if(opts.options.has(opts.partitionsOpt)) { if (topic == ConsumerCoordinator.OffsetsTopicName) { throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.") @@ -120,7 +110,7 @@ object TopicCommand { "logic or ordering of the messages will be affected") val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt) - AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs) + AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr) println("Adding partitions succeeded!") } } @@ -260,7 +250,7 @@ object TopicCommand { .describedAs("topic") .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." + + 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 + "See the Kafka documentation for full details on the topic configs.") .withRequiredArg @@ -307,6 +297,8 @@ object TopicCommand { CommandLineUtils.checkInvalidArgs(parser, options, partitionsOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicationFactorOpt, allTopicLevelOpts -- Set(createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, allTopicLevelOpts -- Set(createOpt,alterOpt)) + // Topic configs cannot be changed with alterTopic + CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(configOpt, deleteConfigOpt)) if(options.has(createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, Set(partitionsOpt, replicationFactorOpt)) CommandLineUtils.checkInvalidArgs(parser, options, reportUnderReplicatedPartitionsOpt, diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 1ae056e..ee3c672 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -2,8 +2,13 @@ package kafka.server import java.util.Properties +import java.util.concurrent.ConcurrentHashMap -import kafka.log.{LogConfig, LogManager} +import kafka.common.TopicAndPartition +import kafka.log.{Log, LogConfig, LogManager} + +import scala.collection.{concurrent, mutable} +import scala.collection.JavaConversions._ /** * The ConfigHandler is used to process config change notifications received by the DynamicConfigManager @@ -19,11 +24,12 @@ trait ConfigHandler { class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandler{ def processConfigChanges(topic : String, topicConfig : Properties) { - val logs = logManager.logsByTopicPartition.toBuffer - val logsByTopic = logs.groupBy(_._1.topic).mapValues(_.map(_._2)) + val logs: mutable.Buffer[(TopicAndPartition, Log)] = logManager.logsByTopicPartition.toBuffer + val logsByTopic: Map[String, mutable.Buffer[Log]] = logs.groupBy{ case k: (TopicAndPartition, Log) => k._1.topic } + .mapValues{ case v: mutable.Buffer[(TopicAndPartition, Log)] => v.map(_._2) } - if (logsByTopic.contains(topic)) - { + System.out.println(logsByTopic) + if (logsByTopic.contains(topic)) { /* combine the default properties with the overrides in zk to create the new LogConfig */ val props = new Properties(logManager.defaultConfig.toProps) props.putAll(topicConfig) @@ -40,11 +46,9 @@ class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandl * This implementation does nothing currently. In the future, it will change quotas per client */ class ClientIdConfigHandler extends ConfigHandler { - @volatile var clientId: String = null - @volatile var clientConfig: Properties = null + val configMap: concurrent.Map[String, Properties] = new ConcurrentHashMap[String, Properties]() def processConfigChanges(clientId : String, clientConfig : Properties): Unit = { - this.clientId = clientId - this.clientConfig = clientConfig + configMap.put(clientId, clientConfig) } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index c6fc0ec..3295871 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -120,19 +120,20 @@ class DynamicConfigManager(private val zkClient: ZkClient, if(jsonOpt.isDefined) { val json = jsonOpt.get Json.parseFull(json) match { - case None => // there are no config overrides + case None => // There are no config overrides. + // Ignore non-json notifications because they can be from the deprecated TopicConfigManager case Some(mapAnon: Map[_, _]) => val map = mapAnon collect { case (k: String, v: Any) => k -> v } require(map("version") == 1) - val entityType = map.get("entityType") match { + val entityType = map.get("entity_type") match { case Some(ConfigType.Topics) => ConfigType.Topics case Some(ConfigType.Clients) => ConfigType.Clients case _ => throw new IllegalArgumentException("Invalid entityType config. Must be either 'client' or 'topic'") } - val entity = map.get("entityName") match { + val entity = map.get("entity_name") match { case Some(value: String) => value case _ => throw new IllegalArgumentException("Value not specified in config change notification " + json) } diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala index 11cb35b..90b6452 100644 --- a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala @@ -32,31 +32,31 @@ class ConfigCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggi def testArgumentParse() { // Should parse correctly var createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, - "--entityName", "x", - "--entityType", "clients", + "--entity-name", "x", + "--entity-type", "clients", "--describe")) createOpts.checkArgs() // For --alter and added config createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, - "--entityName", "x", - "--entityType", "clients", + "--entity-name", "x", + "--entity-type", "clients", "--alter", "--added-config", "a=b,c=d")) createOpts.checkArgs() // For alter and deleted config createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, - "--entityName", "x", - "--entityType", "clients", + "--entity-name", "x", + "--entity-type", "clients", "--alter", "--deleted-config", "a,b,c")) createOpts.checkArgs() // For alter and both added, deleted config createOpts = new ConfigCommandOptions(Array("--zookeeper", zkConnect, - "--entityName", "x", - "--entityType", "clients", + "--entity-name", "x", + "--entity-type", "clients", "--alter", "--added-config", "a=b,c=d", "--deleted-config", "a")) diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index 20e5651..3bbfc2c 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -53,9 +53,7 @@ class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Loggin // modify the topic to add new partitions val numPartitionsModified = 3 - val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, - "--config", cleanupKey + "=" + cleanupVal, - "--topic", topic)) + val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, "--topic", topic)) TopicCommand.alterTopic(zkClient, alterOpts) val newProps = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topics, topic) assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey)) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 5719e38..ccfb298 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -19,7 +19,6 @@ package kafka.server import java.util.Properties import junit.framework.Assert._ -import org.I0Itec.zkclient.ZkClient import org.easymock.{Capture, EasyMock} import org.junit.Test import kafka.integration.KafkaServerTestHarness @@ -66,9 +65,10 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { AdminUtils.changeClientIdConfig(zkClient, clientId, props) TestUtils.retry(10000) { val configHandler = this.servers(0).dynamicConfigHandlers(ConfigType.Clients).asInstanceOf[ClientIdConfigHandler] - assertEquals("ClientId should be set to testClient", clientId, configHandler.clientId) - assertEquals("c", configHandler.clientConfig.getProperty("a.b")) - assertEquals("z", configHandler.clientConfig.getProperty("x.y")) + assertTrue("ClientId testClient must exist", configHandler.configMap.contains(clientId)) + assertEquals("ClientId testClient must be the only override", 1, configHandler.configMap.size) + assertEquals("c", configHandler.configMap(clientId).getProperty("a.b")) + assertEquals("z", configHandler.configMap(clientId).getProperty("x.y")) } } @@ -99,8 +99,8 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { EasyMock.expectLastCall() EasyMock.replay(handler) - val configManager = new DynamicConfigManager(zkClient, Map("Topic" -> handler)) - // Is ignored + val configManager = new DynamicConfigManager(zkClient, Map(ConfigType.Topics -> handler)) + // Notifications created using the old TopicConfigManager are ignored. configManager.processNotification(Some("not json")) // Incorrect Map. No version @@ -114,7 +114,7 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { } // Version is provided. EntityType is incorrect try { - val jsonMap = Map("version" -> 1, "entityType" -> "garbage", "entityName" -> "x") + val jsonMap = Map("version" -> 1, "entity_type" -> "garbage", "entity_name" -> "x") configManager.processNotification(Some(Json.encode(jsonMap))) fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) } @@ -124,7 +124,7 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { // EntityName isn't provided try { - val jsonMap = Map("version" -> 1, "entityType" -> ConfigType.Clients) + val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Clients) configManager.processNotification(Some(Json.encode(jsonMap))) fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) } @@ -133,7 +133,7 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { } // Everything is provided - val jsonMap = Map("version" -> 1, "entityType" -> ConfigType.Clients, "entityName" -> "x") + val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Clients, "entity_name" -> "x") configManager.processNotification(Some(Json.encode(jsonMap))) } } -- 1.7.12.4 From a440f5fa90b1a273b7c6abae9c7029cce9ba96d1 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 7 Jul 2015 18:39:44 -0700 Subject: [PATCH 57/59] Addressing Jun's comments --- .../src/main/scala/kafka/admin/ConfigCommand.scala | 10 +++++----- core/src/main/scala/kafka/admin/TopicCommand.scala | 23 +++------------------- .../main/scala/kafka/server/ConfigHandler.scala | 3 +-- .../scala/kafka/server/DynamicConfigManager.scala | 16 ++++++++------- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- .../kafka/server/DynamicConfigChangeTest.scala | 12 +++++++---- 6 files changed, 27 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index ccac1b8..7642469 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -46,7 +46,7 @@ object ConfigCommand { try { if(opts.options.has(opts.alterOpt)) alterConfig(zkClient, opts) - else if(opts.options.has(opts.describeOpt)) 1 + else if(opts.options.has(opts.describeOpt)) describeConfig(zkClient, opts) } catch { case e: Throwable => @@ -57,7 +57,7 @@ object ConfigCommand { } } - def alterConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { + private def alterConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { val configsToBeAdded = parseConfigsToBeAdded(opts) val configsToBeDeleted = parseConfigsToBeDeleted(opts) val entityType = opts.options.valueOf(opts.entityType) @@ -77,7 +77,7 @@ object ConfigCommand { } } - def describeConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { + private def describeConfig(zkClient: ZkClient, opts: ConfigCommandOptions) { val entityType = opts.options.valueOf(opts.entityType) val entityName = opts.options.valueOf(opts.entityName) val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName) @@ -85,7 +85,7 @@ object ConfigCommand { .format(entityType, entityName, configs.map(kv => kv._1 + "=" + kv._2).mkString(","))) } - def parseConfigsToBeAdded(opts: ConfigCommandOptions): Properties = { + private[admin] def parseConfigsToBeAdded(opts: ConfigCommandOptions): Properties = { val configsToBeAdded = opts.options.valuesOf(opts.addedConfig).map(_.split("""\s*=\s*""")) require(configsToBeAdded.forall(config => config.length == 2), "Invalid entity config: all configs to be added must be in the format \"key=val\".") @@ -94,7 +94,7 @@ object ConfigCommand { props } - def parseConfigsToBeDeleted(opts: ConfigCommandOptions): Seq[String] = { + private[admin] def parseConfigsToBeDeleted(opts: ConfigCommandOptions): Seq[String] = { if (opts.options.has(opts.deletedConfig)) { val configsToBeDeleted = opts.options.valuesOf(opts.deletedConfig).map(_.trim()) val propsToBeDeleted = new Properties diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 84bc2b0..2bedff0 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -204,18 +204,6 @@ object TopicCommand { props } - def parseTopicConfigsToBeDeleted(opts: TopicCommandOptions): Seq[String] = { - if (opts.options.has(opts.deleteConfigOpt)) { - val configsToBeDeleted = opts.options.valuesOf(opts.deleteConfigOpt).map(_.trim()) - val propsToBeDeleted = new Properties - configsToBeDeleted.foreach(propsToBeDeleted.setProperty(_, "")) - LogConfig.validateNames(propsToBeDeleted) - configsToBeDeleted - } - else - Seq.empty - } - def parseReplicaAssignment(replicaAssignmentList: String): Map[Int, List[Int]] = { val partitionList = replicaAssignmentList.split(",") val ret = new mutable.HashMap[Int, List[Int]]() @@ -241,7 +229,7 @@ object TopicCommand { val listOpt = parser.accepts("list", "List all available topics.") val createOpt = parser.accepts("create", "Create a new topic.") val deleteOpt = parser.accepts("delete", "Delete a topic") - val alterOpt = parser.accepts("alter", "Alter the configuration for the topic.") + val alterOpt = parser.accepts("alter", "Alter the number of partitions and/or replica assignment for a topic") val describeOpt = parser.accepts("describe", "List details for the given topics.") val helpOpt = parser.accepts("help", "Print usage information.") val topicOpt = parser.accepts("topic", "The topic to be create, alter or describe. Can also accept a regular " + @@ -250,16 +238,12 @@ object TopicCommand { .describedAs("topic") .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." + + val configOpt = parser.accepts("config", "A configuration override for the topic being created." + "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") .ofType(classOf[String]) - val deleteConfigOpt = parser.accepts("delete-config", "A topic configuration override to be removed for an existing topic (see the list of configurations under the --config option).") - .withRequiredArg - .describedAs("name") - .ofType(classOf[String]) val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " + "altered (WARNING: If partitions are increased for a topic that has a key, the partition logic or ordering of the messages will be affected") .withRequiredArg @@ -293,12 +277,11 @@ object TopicCommand { // check invalid args CommandLineUtils.checkInvalidArgs(parser, options, configOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) - 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(createOpt,alterOpt)) // Topic configs cannot be changed with alterTopic - CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(configOpt, deleteConfigOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(configOpt)) if(options.has(createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, Set(partitionsOpt, replicationFactorOpt)) CommandLineUtils.checkInvalidArgs(parser, options, reportUnderReplicatedPartitionsOpt, diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index ee3c672..6f7a0ae 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -25,10 +25,9 @@ class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandl def processConfigChanges(topic : String, topicConfig : Properties) { val logs: mutable.Buffer[(TopicAndPartition, Log)] = logManager.logsByTopicPartition.toBuffer - val logsByTopic: Map[String, mutable.Buffer[Log]] = logs.groupBy{ case k: (TopicAndPartition, Log) => k._1.topic } + val logsByTopic: Map[String, mutable.Buffer[Log]] = logs.groupBy{ case (topicAndPartition, log) => topicAndPartition.topic } .mapValues{ case v: mutable.Buffer[(TopicAndPartition, Log)] => v.map(_._2) } - System.out.println(logsByTopic) if (logsByTopic.contains(topic)) { /* combine the default properties with the overrides in zk to create the new LogConfig */ val props = new Properties(logManager.defaultConfig.toProps) diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index 3295871..20a58b5 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -130,19 +130,21 @@ class DynamicConfigManager(private val zkClient: ZkClient, val entityType = map.get("entity_type") match { case Some(ConfigType.Topics) => ConfigType.Topics case Some(ConfigType.Clients) => ConfigType.Clients - case _ => throw new IllegalArgumentException("Invalid entityType config. Must be either 'client' or 'topic'") + case _ => throw new IllegalArgumentException("Config change notification must have 'entity_type' set to either 'client' or 'topic'." + + " Received: " + json) } val entity = map.get("entity_name") match { case Some(value: String) => value - case _ => throw new IllegalArgumentException("Value not specified in config change notification " + json) + case _ => throw new IllegalArgumentException("Config change notification does not specify 'entity_name'. Received: " + json) } - if (configHandler.contains(entityType)) - configHandler(entityType).processConfigChanges(entity, AdminUtils.fetchEntityConfig(zkClient, entityType, entity)) - else - warn("Cannot process config changes for entity " + entity) + configHandler(entityType).processConfigChanges(entity, AdminUtils.fetchEntityConfig(zkClient, entityType, entity)) - case o => throw new IllegalArgumentException("Unexpected value in config: " + o) + case o => throw new IllegalArgumentException("Config change notification has an unexpected value. The format is:" + + "{\"version\" : 1," + + " \"entity_type\":\"topic/client\"," + + " \"entity_name\" : \"topic_name/client_id\"}." + + " Received: " + json) } } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index cf7c46e..cd3ed20 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.metrics._ import org.apache.kafka.common.network.NetworkReceive import org.apache.kafka.common.metrics.{JmxReporter, Metrics} -import scala.collection.{JavaConversions, mutable} +import scala.collection.mutable import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} import kafka.cluster.{EndPoint, Broker} diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index ccfb298..b375bfb 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -89,14 +89,15 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { def testProcessNotification { val props = new Properties() props.put("a.b", "10") - // create nice mock since we don't particularly care about zkclient calls + + // Create a mock ConfigHandler to record config changes it is asked to process val entityArgument = new Capture[String]() val propertiesArgument = new Capture[Properties]() val handler = EasyMock.createNiceMock(classOf[ConfigHandler]) handler.processConfigChanges( EasyMock.and(EasyMock.capture(entityArgument), EasyMock.isA(classOf[String])), EasyMock.and(EasyMock.capture(propertiesArgument), EasyMock.isA(classOf[Properties]))) - EasyMock.expectLastCall() + EasyMock.expectLastCall().once() EasyMock.replay(handler) val configManager = new DynamicConfigManager(zkClient, Map(ConfigType.Topics -> handler)) @@ -124,7 +125,7 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { // EntityName isn't provided try { - val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Clients) + val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Topics) configManager.processNotification(Some(Json.encode(jsonMap))) fail("Should have thrown an Exception while parsing incorrect notification " + jsonMap) } @@ -133,7 +134,10 @@ class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { } // Everything is provided - val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Clients, "entity_name" -> "x") + val jsonMap = Map("version" -> 1, "entity_type" -> ConfigType.Topics, "entity_name" -> "x") configManager.processNotification(Some(Json.encode(jsonMap))) + + // Verify that processConfigChanges was only called once + EasyMock.verify(handler) } } -- 1.7.12.4 From 1d2dd051097b970cd5e95e229b12266d81e677c9 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 7 Jul 2015 19:05:40 -0700 Subject: [PATCH 58/59] Addressing Jun's comments --- .../src/main/scala/kafka/admin/ConfigCommand.scala | 4 ++-- core/src/main/scala/kafka/cluster/Partition.scala | 8 ++------ .../scala/kafka/controller/KafkaController.scala | 3 ++- .../main/scala/kafka/server/ConfigHandler.scala | 23 +++++++++++++++++++--- 4 files changed, 26 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 7642469..dd31170 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -20,7 +20,7 @@ package kafka.admin import joptsimple._ import java.util.Properties import kafka.server.ConfigType -import kafka.utils.{ZKStringSerializer, CommandLineUtils} +import kafka.utils.{ZkUtils, ZKStringSerializer, CommandLineUtils} import org.I0Itec.zkclient.ZkClient import scala.collection._ import scala.collection.JavaConversions._ @@ -41,7 +41,7 @@ object ConfigCommand { opts.checkArgs() - val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000) try { if(opts.options.has(opts.alterOpt)) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 3f3f816..f244f9a 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -22,7 +22,7 @@ import kafka.utils.CoreUtils.{inReadLock,inWriteLock} import kafka.admin.AdminUtils import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig -import kafka.server.{TopicPartitionOperationKey, LogOffsetMetadata, LogReadResult, ReplicaManager} +import kafka.server._ import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.message.ByteBufferMessageSet @@ -86,12 +86,8 @@ class Partition(val topic: String, case Some(replica) => replica case None => if (isReplicaLocal(replicaId)) { -<<<<<<< HEAD - val config = LogConfig.fromProps(logManager.defaultConfig.originals, AdminUtils.fetchTopicConfig(zkClient, topic)) -======= - val config = LogConfig.fromProps(logManager.defaultConfig.toProps, + val config = LogConfig.fromProps(logManager.defaultConfig.originals, AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topics, topic)) ->>>>>>> Some fixes val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) val offsetMap = checkpoint.read diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index a1189d7..6322109 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1313,7 +1313,8 @@ class IsrChangeNotificationListener(controller: KafkaController) extends IZkChil processUpdateNotifications(topicAndPartitions) // delete processed children - childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, ZkUtils.TopicConfigChangesPath + "/" + x)) + childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, + ZkUtils.getEntityConfigPath(ConfigType.Topics, x))) } } diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 6f7a0ae..a90d160 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -1,5 +1,21 @@ -package kafka.server +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server import java.util.Properties import java.util.concurrent.ConcurrentHashMap @@ -30,9 +46,10 @@ class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandl if (logsByTopic.contains(topic)) { /* combine the default properties with the overrides in zk to create the new LogConfig */ - val props = new Properties(logManager.defaultConfig.toProps) + val props = new Properties() + props.putAll(logManager.defaultConfig.originals) props.putAll(topicConfig) - val logConfig = LogConfig.fromProps(props) + val logConfig = LogConfig(props) for (log <- logsByTopic(topic)) log.config = logConfig } -- 1.7.12.4 From 0129417f8274b82a5cda6c70411df80d1ed9c7a1 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 14 Jul 2015 10:24:03 -0700 Subject: [PATCH 59/59] Addressing Jun's comments --- core/src/main/scala/kafka/admin/ConfigCommand.scala | 9 +++++++-- core/src/main/scala/kafka/server/ConfigHandler.scala | 2 +- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index dd31170..ed7b84e 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -19,8 +19,9 @@ package kafka.admin import joptsimple._ import java.util.Properties +import kafka.log.LogConfig import kafka.server.ConfigType -import kafka.utils.{ZkUtils, ZKStringSerializer, CommandLineUtils} +import kafka.utils.{ZkUtils, CommandLineUtils} import org.I0Itec.zkclient.ZkClient import scala.collection._ import scala.collection.JavaConversions._ @@ -120,7 +121,11 @@ object ConfigCommand { val entityName = parser.accepts("entity-name", "Name of entity (topic name/client id)") .withRequiredArg .ofType(classOf[String]) - val addedConfig = parser.accepts("added-config", "Key Value pairs configs to add 'k1=v1,k2=v2'") + + val nl = System.getProperty("line.separator") + val addedConfig = parser.accepts("added-config", "Key Value pairs configs to add 'k1=v1,k2=v2'. The following is a list of valid configurations: " + + "For entity_type 'Topics': " + LogConfig.configNames.map("\t" + _).mkString(nl) + nl + + "For entity_type 'Clients' currently no configs are processed by the brokers") .withRequiredArg .ofType(classOf[String]) .withValuesSeparatedBy(',') diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index a90d160..ecfc5ae 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -24,7 +24,6 @@ import kafka.common.TopicAndPartition import kafka.log.{Log, LogConfig, LogManager} import scala.collection.{concurrent, mutable} -import scala.collection.JavaConversions._ /** * The ConfigHandler is used to process config change notifications received by the DynamicConfigManager @@ -62,6 +61,7 @@ class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandl * This implementation does nothing currently. In the future, it will change quotas per client */ class ClientIdConfigHandler extends ConfigHandler { + import collection.JavaConversions._ val configMap: concurrent.Map[String, Properties] = new ConcurrentHashMap[String, Properties]() def processConfigChanges(clientId : String, clientConfig : Properties): Unit = { -- 1.7.12.4