From 54e53b7312274a3599229a36caa35f0cd931e341 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Thu, 22 Jan 2015 22:29:33 -0500 Subject: [PATCH 01/41] KAFKA-1891 MirrorMaker hides consumer exception - making troubleshooting challenging patch by Gwen Shapira reviewed by Joe Stein --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 5cbc810..a11dbc8 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -273,7 +273,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { streams = connector.createMessageStreamsByFilter(filterSpec, numStreams, new DefaultDecoder(), new DefaultDecoder()) } catch { case t: Throwable => - fatal("Unable to create stream - shutting down mirror maker.") + fatal("Unable to create stream - shutting down mirror maker.", t) connector.shutdown() } consumerThreads = streams.zipWithIndex.map(streamAndIndex => new ConsumerThread(streamAndIndex._1, mirrorDataChannel, streamAndIndex._2)) -- 1.9.3 (Apple Git-50) From 8cff9119f8d125e55898e2095512b90826522336 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 23 Jan 2015 09:07:12 -0800 Subject: [PATCH 02/41] KAFKA-1890 Fix bug preventing Mirror Maker from successful rebalance; reviewed by Gwen Shapira and Neha Narkhede --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index a11dbc8..3ee84ed 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -213,11 +213,11 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val customRebalanceListenerClass = options.valueOf(consumerRebalanceListenerOpt) val customRebalanceListener = { if (customRebalanceListenerClass != null) - Utils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass) + Some(Utils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass)) else - null + None } - consumerRebalanceListener = new InternalRebalanceListener(mirrorDataChannel, Some(customRebalanceListener)) + consumerRebalanceListener = new InternalRebalanceListener(mirrorDataChannel, customRebalanceListener) connector.setConsumerRebalanceListener(consumerRebalanceListener) // create producer threads -- 1.9.3 (Apple Git-50) From ff5cbcacc6fc3217ea41c624fb293adeae7bb672 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 23 Jan 2015 18:23:32 -0800 Subject: [PATCH 03/41] KAFKA-1896; Record size function should check if value is null; reviewed by Guozhang Wang --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 3ee84ed..81ae205 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -637,7 +637,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val sourceOffset: Long, val key: Array[Byte], val value: Array[Byte]) { - def size = value.length + {if (key == null) 0 else key.length} + def size = {if (value == null) 0 else value.length} + {if (key == null) 0 else key.length} } private class UnackedOffset(offset: Long) extends DoublyLinkedListNode[Long](offset) { -- 1.9.3 (Apple Git-50) From 995d0d369363ef3ad7cf43980746c52768b4df2e Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Sun, 25 Jan 2015 18:43:17 -0800 Subject: [PATCH 04/41] KAFKA-1109 Need to fix GC log configuration code, not able to override KAFKA_GC_LOG_OPTS; reviewed by Neha Narkhede --- bin/kafka-run-class.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 22a9865..881f578 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -125,7 +125,7 @@ while [ $# -gt 0 ]; do shift 2 ;; -loggc) - if [ -z "$KAFKA_GC_LOG_OPTS"] ; then + if [ -z "$KAFKA_GC_LOG_OPTS" ]; then GC_LOG_ENABLED="true" fi shift -- 1.9.3 (Apple Git-50) From 4aa3dab3de088096461941353ba27cb37f1bd9d1 Mon Sep 17 00:00:00 2001 From: jaikiran pai Date: Sun, 25 Jan 2015 18:54:51 -0800 Subject: [PATCH 05/41] KAFKA-1883 Fix NullPointerException in RequestSendThread; reviewed by Neha Narkhede --- .../controller/ControllerChannelManager.scala | 32 ++++++++++++---------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index eb492f0..fbef34c 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -125,7 +125,7 @@ class RequestSendThread(val controllerId: Int, try { lock synchronized { var isSendSuccessful = false - while(isRunning.get() && !isSendSuccessful) { + while (isRunning.get() && !isSendSuccessful) { // if a broker goes down for a long time, then at some point the controller's zookeeper listener will trigger a // removeBroker which will invoke shutdown() on this thread. At that point, we will stop retrying. try { @@ -136,7 +136,7 @@ class RequestSendThread(val controllerId: Int, case e: Throwable => // if the send was not successful, reconnect to broker and resend the message warn(("Controller %d epoch %d fails to send request %s to broker %s. " + "Reconnecting to broker.").format(controllerId, controllerContext.epoch, - request.toString, toBroker.toString()), e) + request.toString, toBroker.toString()), e) channel.disconnect() connectToBroker(toBroker, channel) isSendSuccessful = false @@ -144,20 +144,22 @@ class RequestSendThread(val controllerId: Int, Utils.swallow(Thread.sleep(300)) } } - var response: RequestOrResponse = null - request.requestId.get match { - case RequestKeys.LeaderAndIsrKey => - response = LeaderAndIsrResponse.readFrom(receive.buffer) - case RequestKeys.StopReplicaKey => - response = StopReplicaResponse.readFrom(receive.buffer) - case RequestKeys.UpdateMetadataKey => - response = UpdateMetadataResponse.readFrom(receive.buffer) - } - stateChangeLogger.trace("Controller %d epoch %d received response %s for a request sent to broker %s" - .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString)) + if (receive != null) { + var response: RequestOrResponse = null + request.requestId.get match { + case RequestKeys.LeaderAndIsrKey => + response = LeaderAndIsrResponse.readFrom(receive.buffer) + case RequestKeys.StopReplicaKey => + response = StopReplicaResponse.readFrom(receive.buffer) + case RequestKeys.UpdateMetadataKey => + response = UpdateMetadataResponse.readFrom(receive.buffer) + } + stateChangeLogger.trace("Controller %d epoch %d received response %s for a request sent to broker %s" + .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString)) - if(callback != null) { - callback(response) + if (callback != null) { + callback(response) + } } } } catch { -- 1.9.3 (Apple Git-50) From 0b171dee1fca16595f635bc9991056712e9e3b1a Mon Sep 17 00:00:00 2001 From: jaikiran pai Date: Sun, 25 Jan 2015 19:15:51 -0800 Subject: [PATCH 06/41] KAFKA-1885 Upgrade junit dependency in core to 4.6 version to allow running individual test methods via gradle command line; reviewed by Neha Narkhede --- README.md | 5 +++++ build.gradle | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 35e06b1..9bdcf70 100644 --- a/README.md +++ b/README.md @@ -34,6 +34,11 @@ Follow instuctions in http://kafka.apache.org/documentation.html#quickstart ### Running a particular unit test ### ./gradlew -Dtest.single=RequestResponseSerializationTest core:test +### Running a particular test method within a unit test ### + ./gradlew core:test --tests kafka.api.test.ProducerFailureHandlingTest.testCannotSendToInternalTopic + ./gradlew clients:test --tests org.apache.kafka.clients.producer.MetadataTest.testMetadataUpdateWaitTime + + ### Running a particular unit test with log4j output ### change the log4j setting in either clients/src/test/resources/log4j.properties or core/src/test/resources/log4j.properties ./gradlew -i -Dtest.single=RequestResponseSerializationTest core:test diff --git a/build.gradle b/build.gradle index 1cbab29..3a509c3 100644 --- a/build.gradle +++ b/build.gradle @@ -209,7 +209,7 @@ project(':core') { compile 'com.yammer.metrics:metrics-core:2.2.0' compile 'net.sf.jopt-simple:jopt-simple:3.2' - testCompile 'junit:junit:4.1' + testCompile 'junit:junit:4.6' testCompile 'org.easymock:easymock:3.0' testCompile 'org.objenesis:objenesis:1.2' testCompile project(':clients') -- 1.9.3 (Apple Git-50) From 0f6cc0a058bbd2736f1e638448fc64b58a9f6b41 Mon Sep 17 00:00:00 2001 From: Eric Olander Date: Sun, 25 Jan 2015 19:27:54 -0800 Subject: [PATCH 07/41] KAFKA-1818 KAFKA-1818 clean up code to more idiomatic scala usage; reviewed by Neha Narkhede and Gwen Shapira --- .../main/scala/kafka/utils/ReplicationUtils.scala | 32 ++++++++-------------- .../unit/kafka/utils/ReplicationUtilsTest.scala | 10 +++++++ 2 files changed, 22 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala index 7157673..6068733 100644 --- a/core/src/main/scala/kafka/utils/ReplicationUtils.scala +++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala @@ -61,30 +61,22 @@ object ReplicationUtils extends Logging { def getLeaderIsrAndEpochForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = { val leaderAndIsrPath = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition) - val leaderAndIsrInfo = ZkUtils.readDataMaybeNull(zkClient, leaderAndIsrPath) - val leaderAndIsrOpt = leaderAndIsrInfo._1 - val stat = leaderAndIsrInfo._2 - leaderAndIsrOpt match { - case Some(leaderAndIsrStr) => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat) - case None => None - } + val (leaderAndIsrOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, leaderAndIsrPath) + leaderAndIsrOpt.flatMap(leaderAndIsrStr => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat)) } private def parseLeaderAndIsr(leaderAndIsrStr: String, path: String, stat: Stat) : Option[LeaderIsrAndControllerEpoch] = { - Json.parseFull(leaderAndIsrStr) match { - case Some(m) => - val leaderIsrAndEpochInfo = m.asInstanceOf[Map[String, Any]] - val leader = leaderIsrAndEpochInfo.get("leader").get.asInstanceOf[Int] - val epoch = leaderIsrAndEpochInfo.get("leader_epoch").get.asInstanceOf[Int] - val isr = leaderIsrAndEpochInfo.get("isr").get.asInstanceOf[List[Int]] - val controllerEpoch = leaderIsrAndEpochInfo.get("controller_epoch").get.asInstanceOf[Int] - val zkPathVersion = stat.getVersion - debug("Leader %d, Epoch %d, Isr %s, Zk path version %d for leaderAndIsrPath %s".format(leader, epoch, - isr.toString(), zkPathVersion, path)) - Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch)) - case None => None - } + Json.parseFull(leaderAndIsrStr).flatMap {m => + val leaderIsrAndEpochInfo = m.asInstanceOf[Map[String, Any]] + val leader = leaderIsrAndEpochInfo.get("leader").get.asInstanceOf[Int] + val epoch = leaderIsrAndEpochInfo.get("leader_epoch").get.asInstanceOf[Int] + val isr = leaderIsrAndEpochInfo.get("isr").get.asInstanceOf[List[Int]] + val controllerEpoch = leaderIsrAndEpochInfo.get("controller_epoch").get.asInstanceOf[Int] + val zkPathVersion = stat.getVersion + debug("Leader %d, Epoch %d, Isr %s, Zk path version %d for leaderAndIsrPath %s".format(leader, epoch, + isr.toString(), zkPathVersion, path)) + Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch))} } } diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index 84e0855..305498a 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -17,6 +17,7 @@ package kafka.utils +import kafka.controller.LeaderIsrAndControllerEpoch import kafka.server.{ReplicaFetcherManager, KafkaConfig} import kafka.api.LeaderAndIsr import kafka.zk.ZooKeeperTestHarness @@ -42,6 +43,8 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { val topicDataMismatch = Json.encode(Map("controller_epoch" -> 1, "leader" -> 1, "versions" -> 2, "leader_epoch" -> 2,"isr" -> List(1,2))) + val topicDataLeaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(1,leaderEpoch,List(1,2),0), controllerEpoch) + override def setUp() { super.setUp() @@ -92,4 +95,11 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(newZkVersion3,-1) } + @Test + def testGetLeaderIsrAndEpochForPartition() { + val leaderIsrAndControllerEpoch = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partitionId) + assertEquals(topicDataLeaderIsrAndControllerEpoch, leaderIsrAndControllerEpoch.get) + assertEquals(None, ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partitionId + 1)) + } + } -- 1.9.3 (Apple Git-50) From f689e9b1e361cdf47e9966567631c06f614e446d Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 28 Jan 2015 19:07:39 -0600 Subject: [PATCH 08/41] KAFKA-1902; fix MetricName so that Yammer reporter can work correctly; patched by Jun Rao; reviewed by Manikumar Reddy, Manikumar Reddy and Joel Koshy --- .../scala/kafka/metrics/KafkaMetricsGroup.scala | 27 ++++++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala index e9e4918..9e31184 100644 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala @@ -61,9 +61,15 @@ trait KafkaMetricsGroup extends Logging { nameBuilder.append(name) } - KafkaMetricsGroup.toMBeanName(tags).map(mbeanName => nameBuilder.append(",").append(mbeanName)) + val scope: String = KafkaMetricsGroup.toScope(tags).getOrElse(null) + val tagsName = KafkaMetricsGroup.toMBeanName(tags) + tagsName match { + case Some(tn) => + nameBuilder.append(",").append(tn) + case None => + } - new MetricName(group, typeName, name, null, nameBuilder.toString()) + new MetricName(group, typeName, name, scope, nameBuilder.toString()) } def newGauge[T](name: String, metric: Gauge[T], tags: scala.collection.Map[String, String] = Map.empty) = @@ -160,6 +166,23 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { } } + private def toScope(tags: collection.Map[String, String]): Option[String] = { + val filteredTags = tags + .filter { case (tagKey, tagValue) => tagValue != ""} + if (filteredTags.nonEmpty) { + // convert dot to _ since reporters like Graphite typically use dot to represent hierarchy + val tagsString = filteredTags + .toList.sortWith((t1, t2) => t1._1 < t2._1) + .map { case (key, value) => "%s.%s".format(key, value.replaceAll("\\.", "_"))} + .mkString(".") + + Some(tagsString) + } + else { + None + } + } + def removeAllConsumerMetrics(clientId: String) { FetchRequestAndResponseStatsRegistry.removeConsumerFetchRequestAndResponseStats(clientId) ConsumerTopicStatsRegistry.removeConsumerTopicStat(clientId) -- 1.9.3 (Apple Git-50) From 11ec9bf5a5f150e5c172f88bb3e60b197ee9df5a Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Wed, 28 Jan 2015 19:33:15 -0600 Subject: [PATCH 09/41] KAFKA-1861; Publishing kafka-client:test in order to utilize the helper utils in TestUtils; patched by Manikumar Reddy; reviewed by Jun Rao --- build.gradle | 3 +++ 1 file changed, 3 insertions(+) diff --git a/build.gradle b/build.gradle index 3a509c3..a980f61 100644 --- a/build.gradle +++ b/build.gradle @@ -375,4 +375,7 @@ project(':clients') { include "**/org/apache/kafka/common/serialization/*" } + artifacts { + archives testJar + } } -- 1.9.3 (Apple Git-50) From 0699ff2ce60abb466cab5315977a224f1a70a4da Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Sun, 11 Jan 2015 11:29:48 -0800 Subject: [PATCH 10/41] KAFKA-1760: New consumer. --- build.gradle | 1 + .../org/apache/kafka/clients/ClientRequest.java | 19 +- .../kafka/clients/ClusterConnectionStates.java | 30 +- .../apache/kafka/clients/CommonClientConfigs.java | 58 + .../org/apache/kafka/clients/ConnectionState.java | 2 +- .../java/org/apache/kafka/clients/KafkaClient.java | 44 +- .../org/apache/kafka/clients/NetworkClient.java | 138 +- .../apache/kafka/clients/NodeConnectionState.java | 31 - .../kafka/clients/RequestCompletionHandler.java | 23 + .../apache/kafka/clients/consumer/CommitType.java | 5 + .../apache/kafka/clients/consumer/Consumer.java | 109 +- .../kafka/clients/consumer/ConsumerConfig.java | 296 ++-- .../consumer/ConsumerRebalanceCallback.java | 89 +- .../kafka/clients/consumer/ConsumerRecord.java | 89 +- .../kafka/clients/consumer/ConsumerRecords.java | 107 +- .../kafka/clients/consumer/KafkaConsumer.java | 1865 +++++++++++++++----- .../kafka/clients/consumer/MockConsumer.java | 233 ++- .../consumer/NoOffsetForPartitionException.java | 29 + .../kafka/clients/consumer/OffsetMetadata.java | 59 - .../clients/consumer/internals/Heartbeat.java | 47 + .../internals/NoOpConsumerRebalanceCallback.java | 30 + .../consumer/internals/SubscriptionState.java | 166 ++ .../kafka/clients/producer/KafkaProducer.java | 3 +- .../kafka/clients/producer/MockProducer.java | 2 +- .../kafka/clients/producer/ProducerConfig.java | 64 +- .../kafka/clients/producer/internals/Metadata.java | 22 +- .../clients/producer/internals/Partitioner.java | 27 +- .../kafka/clients/producer/internals/Sender.java | 155 +- .../main/java/org/apache/kafka/common/Cluster.java | 14 + .../org/apache/kafka/common/PartitionInfo.java | 4 +- .../org/apache/kafka/common/config/ConfigDef.java | 26 +- .../apache/kafka/common/errors/ApiException.java | 2 +- .../apache/kafka/common/network/Selectable.java | 58 +- .../org/apache/kafka/common/network/Selector.java | 99 +- .../org/apache/kafka/common/protocol/Errors.java | 59 +- .../apache/kafka/common/protocol/types/Struct.java | 46 +- .../org/apache/kafka/common/record/LogEntry.java | 4 + .../apache/kafka/common/record/MemoryRecords.java | 48 +- .../common/requests/ConsumerMetadataRequest.java | 9 +- .../common/requests/ConsumerMetadataResponse.java | 17 +- .../apache/kafka/common/requests/FetchRequest.java | 74 +- .../kafka/common/requests/FetchResponse.java | 21 +- .../kafka/common/requests/HeartbeatRequest.java | 13 +- .../kafka/common/requests/HeartbeatResponse.java | 9 +- .../kafka/common/requests/JoinGroupRequest.java | 17 +- .../kafka/common/requests/JoinGroupResponse.java | 25 +- .../kafka/common/requests/ListOffsetRequest.java | 25 +- .../kafka/common/requests/ListOffsetResponse.java | 19 +- .../kafka/common/requests/MetadataRequest.java | 9 +- .../kafka/common/requests/MetadataResponse.java | 33 +- .../kafka/common/requests/OffsetCommitRequest.java | 45 +- .../common/requests/OffsetCommitResponse.java | 17 +- .../kafka/common/requests/OffsetFetchRequest.java | 17 +- .../kafka/common/requests/OffsetFetchResponse.java | 46 +- .../kafka/common/requests/ProduceRequest.java | 21 +- .../kafka/common/requests/ProduceResponse.java | 19 +- .../kafka/common/serialization/Deserializer.java | 2 +- .../java/org/apache/kafka/common/utils/Utils.java | 42 + .../java/org/apache/kafka/clients/MockClient.java | 31 +- .../apache/kafka/clients/NetworkClientTest.java | 35 +- .../clients/consumer/ConsumerExampleTest.java | 297 ---- .../kafka/clients/consumer/MockConsumerTest.java | 32 + .../consumer/internals/SubscriptionStateTest.java | 61 + .../kafka/clients/producer/BufferPoolTest.java | 4 +- .../kafka/clients/producer/MockProducerTest.java | 2 +- .../kafka/clients/producer/PartitionerTest.java | 49 +- .../apache/kafka/clients/producer/SenderTest.java | 6 +- .../apache/kafka/common/config/ConfigDefTest.java | 6 +- .../apache/kafka/common/network/SelectorTest.java | 53 +- .../org/apache/kafka/common/utils/UtilsTest.java | 10 + .../java/org/apache/kafka/test/MockSelector.java | 52 +- .../scala/kafka/api/ConsumerMetadataRequest.scala | 2 +- .../scala/kafka/api/ConsumerMetadataResponse.scala | 2 +- core/src/main/scala/kafka/cluster/Partition.scala | 2 +- .../controller/ControllerChannelManager.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 4 +- core/src/main/scala/kafka/server/KafkaApis.scala | 29 + .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../main/scala/kafka/tools/ConsoleConsumer.scala | 2 - .../scala/kafka/tools/ConsumerPerformance.scala | 169 +- .../kafka/tools/SimpleConsumerPerformance.scala | 2 +- .../main/scala/kafka/utils/KafkaScheduler.scala | 56 +- .../scala/integration/kafka/api/ConsumerTest.scala | 286 +++ .../kafka/api/IntegrationTestHarness.scala | 73 + .../api/RequestResponseSerializationTest.scala | 4 +- .../kafka/integration/KafkaServerTestHarness.scala | 39 +- .../unit/kafka/integration/PrimitiveApiTest.scala | 19 +- .../scala/unit/kafka/utils/MockScheduler.scala | 2 + .../scala/unit/kafka/utils/SchedulerTest.scala | 17 + .../test/scala/unit/kafka/utils/TestUtils.scala | 34 +- 90 files changed, 4002 insertions(+), 1965 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java delete mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java create mode 100644 core/src/test/scala/integration/kafka/api/ConsumerTest.scala create mode 100644 core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala diff --git a/build.gradle b/build.gradle index a980f61..6844372 100644 --- a/build.gradle +++ b/build.gradle @@ -369,6 +369,7 @@ project(':clients') { } javadoc { + include "**/org/apache/kafka/clients/consumer/*" include "**/org/apache/kafka/clients/producer/*" include "**/org/apache/kafka/common/*" include "**/org/apache/kafka/common/errors/*" diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java index d32c319..ed4c0d9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -22,24 +22,25 @@ public final class ClientRequest { private final long createdMs; private final boolean expectResponse; private final RequestSend request; - private final Object attachment; + private final RequestCompletionHandler callback; /** * @param createdMs The unix timestamp in milliseconds for the time at which this request was created. * @param expectResponse Should we expect a response message or is this request complete once it is sent? * @param request The request - * @param attachment Associated data with the request + * @param callback A callback to execute when the response has been received (or null if no callback is necessary) */ - public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, Object attachment) { + public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, RequestCompletionHandler callback) { this.createdMs = createdMs; - this.attachment = attachment; + this.callback = callback; this.request = request; this.expectResponse = expectResponse; } @Override public String toString() { - return "ClientRequest(expectResponse=" + expectResponse + ", payload=" + attachment + ", request=" + request + ")"; + return "ClientRequest(expectResponse=" + expectResponse + ", callback=" + callback + ", request=" + request + + ")"; } public boolean expectResponse() { @@ -50,8 +51,12 @@ public final class ClientRequest { return request; } - public Object attachment() { - return attachment; + public boolean hasCallback() { + return callback != null; + } + + public RequestCompletionHandler callback() { + return callback; } public long createdTime() { diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java index 8aece7e..574287d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java @@ -119,16 +119,42 @@ final class ClusterConnectionStates { public void disconnected(int node) { nodeState(node).state = ConnectionState.DISCONNECTED; } - + /** - * Get the state of our connection to the given state + * Get the state of our connection to the given node * @param node The id of the node * @return The state of our connection */ + public ConnectionState connectionState(int node) { + return nodeState(node).state; + } + + /** + * Get the state of a given node + * @param node The node to fetch the state for + */ private NodeConnectionState nodeState(int node) { NodeConnectionState state = this.nodeState.get(node); if (state == null) throw new IllegalStateException("No entry found for node " + node); return state; } + + /** + * The state of our connection to a node + */ + private static class NodeConnectionState { + + ConnectionState state; + long lastConnectAttemptMs; + + public NodeConnectionState(ConnectionState state, long lastConnectAttempt) { + this.state = state; + this.lastConnectAttemptMs = lastConnectAttempt; + } + + public String toString() { + return "NodeState(" + state + ", " + lastConnectAttemptMs + ")"; + } + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java new file mode 100644 index 0000000..06fcfe6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.clients; + +/** + * Some configurations shared by both producer and consumer + */ +public class CommonClientConfigs { + + /* + * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE. + */ + + public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + public static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping—this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form " + + "host1:port1,host2:port2,.... Since these servers are just used for the initial connection to " + + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of " + + "servers (you may want more than one, though, in case a server is down)."; + + public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; + public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions."; + + public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes"; + public static final String SEND_BUFFER_DOC = "The size of the TCP send buffer (SO_SNDBUF) to use when sending data."; + + public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes"; + public static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer (SO_RCVBUF) to use when reading data."; + + public static final String CLIENT_ID_CONFIG = "client.id"; + public static final String CLIENT_ID_DOC = "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging."; + + public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; + public static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all requests sent by the consumer to the broker."; + + public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms"; + public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop."; + + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; + public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The number of samples maintained to compute metrics."; + + public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples"; + public static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics."; + + public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; + public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the MetricReporter interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java index ab7e322..3867f8e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java @@ -15,6 +15,6 @@ package org.apache.kafka.clients; /** * The states of a node connection */ -enum ConnectionState { +public enum ConnectionState { DISCONNECTED, CONNECTING, CONNECTED } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 3976955..8a3e55a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -26,6 +26,7 @@ public interface KafkaClient { /** * Check if we are currently ready to send another request to the given node but don't attempt to connect if we * aren't. + * * @param node The node to check * @param now The current timestamp */ @@ -34,6 +35,7 @@ public interface KafkaClient { /** * Initiate a connection to the given node (if necessary), and return true if already connected. The readiness of a * node will change only when poll is invoked. + * * @param node The node to connect to. * @param now The current time * @return true iff we are ready to immediately initiate the sending of another request to the given node. @@ -44,6 +46,7 @@ public interface KafkaClient { * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled * connections. + * * @param node The node to check * @param now The current timestamp * @return The number of milliseconds to wait. @@ -51,19 +54,44 @@ public interface KafkaClient { public long connectionDelay(Node node, long now); /** - * Initiate the sending of the given requests and return any completed responses. Requests can only be sent on ready - * connections. - * @param requests The requests to send + * Queue up the given request for sending. Requests can only be sent on ready connections. + * + * @param request The request + * @param now The current time + */ + public void send(ClientRequest request); + + /** + * Do actual reads and writes from sockets. + * * @param timeout The maximum amount of time to wait for responses in ms * @param now The current time in ms * @throws IllegalStateException If a request is sent to an unready node */ - public List poll(List requests, long timeout, long now); + public List poll(long timeout, long now); + + /** + * Complete all in-flight requests for a given node + * + * @param node The node to complete requests for + * @param now The current time in ms + * @return All requests that complete during this time period. + */ + public List completeAll(int node, long now); + + /** + * Complete all in-flight requests + * + * @param now The current time in ms + * @return All requests that complete during this time period. + */ + public List completeAll(long now); /** * Choose the node with the fewest outstanding requests. This method will prefer a node with an existing connection, * but will potentially choose a node for which we don't yet have a connection if all existing connections are in * use. + * * @param now The current time in ms * @return The node with the fewest in-flight requests. */ @@ -75,7 +103,15 @@ public interface KafkaClient { public int inFlightRequestCount(); /** + * Get the total in-flight requests for a particular node + * + * @param nodeId The id of the node + */ + public int inFlightRequestCount(int nodeId); + + /** * Generate a request header for the next request + * * @param key The API key of the request */ public RequestHeader nextRequestHeader(ApiKeys key); diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 6746275..5950191 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -102,6 +102,7 @@ public class NetworkClient implements KafkaClient { /** * Begin connecting to the given node, return true if we are already connected and ready to send to that node. + * * @param node The node to check * @param now The current timestamp * @return True if we are ready to send to the given node @@ -122,6 +123,7 @@ public class NetworkClient implements KafkaClient { * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled * connections. + * * @param node The node to check * @param now The current timestamp * @return The number of milliseconds to wait. @@ -133,7 +135,8 @@ public class NetworkClient implements KafkaClient { /** * Check if the node with the given id is ready to send more requests. - * @param node The given node id + * + * @param node The node * @param now The current time in ms * @return true if the node is ready */ @@ -141,7 +144,8 @@ public class NetworkClient implements KafkaClient { public boolean isReady(Node node, long now) { int nodeId = node.id(); if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0) - // if we need to update our metadata now declare all requests unready to make metadata requests first priority + // if we need to update our metadata now declare all requests unready to make metadata requests first + // priority return false; else // otherwise we are ready if we are connected and can send more requests @@ -150,6 +154,7 @@ public class NetworkClient implements KafkaClient { /** * Are we connected and ready and able to send more requests to the given node? + * * @param node The node */ private boolean isSendable(int node) { @@ -157,49 +162,106 @@ public class NetworkClient implements KafkaClient { } /** - * Initiate the given requests and check for any new responses, waiting up to the specified time. Requests can only - * be sent for ready nodes. - * @param requests The requests to initiate + * Return the state of the connection to the given node + * + * @param node The node to check + * @return The connection state + */ + public ConnectionState connectionState(int node) { + return connectionStates.connectionState(node); + } + + /** + * Queue up the given request for sending. Requests can only be sent out to ready nodes. + * + * @param request The request + * @param now The current time + */ + @Override + public void send(ClientRequest request) { + int nodeId = request.request().destination(); + if (!isSendable(nodeId)) + throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); + + this.inFlightRequests.add(request); + selector.send(request.request()); + } + + /** + * Do actual reads and writes to sockets. + * * @param timeout The maximum amount of time to wait (in ms) for responses if there are none immediately * @param now The current time in milliseconds * @return The list of responses received */ @Override - public List poll(List requests, long timeout, long now) { - List sends = new ArrayList(); - - for (int i = 0; i < requests.size(); i++) { - ClientRequest request = requests.get(i); - int nodeId = request.request().destination(); - if (!isSendable(nodeId)) - throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); - - this.inFlightRequests.add(request); - sends.add(request.request()); - } - + public List poll(long timeout, long now) { // should we update our metadata? long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now); long timeToNextReconnectAttempt = Math.max(this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now, 0); long waitForMetadataFetch = (this.metadataFetchInProgress ? Integer.MAX_VALUE : 0); // if there is no node available to connect, back off refreshing metadata - long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt), waitForMetadataFetch); + long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt), + waitForMetadataFetch); if (!this.metadataFetchInProgress && metadataTimeout == 0) - maybeUpdateMetadata(sends, now); - + maybeUpdateMetadata(now); // do the I/O try { - this.selector.poll(Math.min(timeout, metadataTimeout), sends); + this.selector.poll(Math.min(timeout, metadataTimeout)); } catch (IOException e) { log.error("Unexpected error during I/O in producer network thread", e); } + // process completed actions List responses = new ArrayList(); handleCompletedSends(responses, now); handleCompletedReceives(responses, now); handleDisconnections(responses, now); handleConnections(); + // invoke callbacks + for (ClientResponse response : responses) { + if (response.request().hasCallback()) { + try { + response.request().callback().onComplete(response); + } catch (Exception e) { + log.error("Uncaught error in request completion:", e); + } + } + } + + return responses; + } + + /** + * Await all the outstanding responses for requests on the given connection + * + * @param node The node to block on + * @param now The current time in ms + * @return All the collected responses + */ + @Override + public List completeAll(int node, long now) { + try { + this.selector.muteAll(); + this.selector.unmute(node); + List responses = new ArrayList(); + while (inFlightRequestCount(node) > 0) + responses.addAll(poll(Integer.MAX_VALUE, now)); + return responses; + } finally { + this.selector.unmuteAll(); + } + } + + /** + * Wait for all outstanding requests to complete. + */ + @Override + public List completeAll(long now) { + List responses = new ArrayList(); + while (inFlightRequestCount() > 0) + responses.addAll(poll(Integer.MAX_VALUE, now)); return responses; } @@ -212,7 +274,16 @@ 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); + } + + /** * Generate a request header for the given API key + * * @param key The api key * @return A request header with the appropriate client id and correlation id */ @@ -242,6 +313,7 @@ public class NetworkClient implements KafkaClient { * prefer a node with an existing connection, but will potentially choose a node for which we don't yet have a * connection if all existing connections are in use. This method will never choose a node for which there is no * existing connection and from which we have disconnected within the reconnect backoff period. + * * @return The node with the fewest in-flight requests. */ public Node leastLoadedNode(long now) { @@ -261,12 +333,12 @@ public class NetworkClient implements KafkaClient { found = node; } } - return found; } /** * Handle any completed request send. In particular if no response is expected consider the request complete. + * * @param responses The list of responses to update * @param now The current time */ @@ -283,6 +355,7 @@ public class NetworkClient implements KafkaClient { /** * Handle any completed receives and update the response list with the responses received. + * * @param responses The list of responses to update * @param now The current time */ @@ -317,6 +390,7 @@ public class NetworkClient implements KafkaClient { /** * Handle any disconnected connections + * * @param responses The list of responses that completed with the disconnection * @param now The current time */ @@ -353,10 +427,8 @@ public class NetworkClient implements KafkaClient { */ private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) { if (requestHeader.correlationId() != responseHeader.correlationId()) - throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() + - ") does not match request (" + - requestHeader.correlationId() + - ")"); + throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() + + ") does not match request (" + requestHeader.correlationId() + ")"); } /** @@ -371,7 +443,7 @@ public class NetworkClient implements KafkaClient { /** * Add a metadata request to the list of sends if we can make one */ - private void maybeUpdateMetadata(List sends, long now) { + private void maybeUpdateMetadata(long now) { // Beware that the behavior of this method and the computation of timeouts for poll() are // highly dependent on the behavior of leastLoadedNode. Node node = this.leastLoadedNode(now); @@ -382,17 +454,16 @@ public class NetworkClient implements KafkaClient { return; } - log.debug("Trying to send metadata request to node {}", node.id()); if (connectionStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) { Set topics = metadata.topics(); this.metadataFetchInProgress = true; ClientRequest metadataRequest = metadataRequest(now, node.id(), topics); log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); - sends.add(metadataRequest.request()); + this.selector.send(metadataRequest.request()); this.inFlightRequests.add(metadataRequest); } else if (connectionStates.canConnect(node.id(), now)) { // we don't have a connection to this node right now, make one - log.debug("Init connection to node {} for sending metadata request in the next iteration", node.id()); + log.debug("Initialize connection to node {} for sending metadata request", node.id()); initiateConnect(node, now); // If initiateConnect failed immediately, this node will be put into blackout and we // should allow immediately retrying in case there is another candidate node. If it @@ -412,7 +483,10 @@ public class NetworkClient implements KafkaClient { try { log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port()); this.connectionStates.connecting(node.id(), now); - selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), this.socketSendBuffer, this.socketReceiveBuffer); + selector.connect(node.id(), + new InetSocketAddress(node.host(), node.port()), + this.socketSendBuffer, + this.socketReceiveBuffer); } catch (IOException e) { /* attempt failed, we'll try again after the backoff */ connectionStates.disconnected(node.id()); diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java deleted file mode 100644 index 752a979..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java +++ /dev/null @@ -1,31 +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; - -/** - * The state of our connection to a node - */ -final class NodeConnectionState { - - ConnectionState state; - long lastConnectAttemptMs; - - public NodeConnectionState(ConnectionState state, long lastConnectAttempt) { - this.state = state; - this.lastConnectAttemptMs = lastConnectAttempt; - } - - public String toString() { - return "NodeState(" + state + ", " + lastConnectAttemptMs + ")"; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java new file mode 100644 index 0000000..6fee4e4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients; + +/** + * A callback interface for attaching an action to be executed when a request is complete and the corresponding response + * has been received. This handler will also be invoked if there is a disconnection while handling the request. + */ +public interface RequestCompletionHandler { + + public void onComplete(ClientResponse response); + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java new file mode 100644 index 0000000..072cc2e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java @@ -0,0 +1,5 @@ +package org.apache.kafka.clients.consumer; + +public enum CommitType { + SYNC, ASYNC +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index c0c636b..8f587bc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -9,14 +9,16 @@ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. -*/ + */ package org.apache.kafka.clients.consumer; import java.io.Closeable; -import java.util.Collection; +import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.MetricName; @@ -24,102 +26,85 @@ import org.apache.kafka.common.MetricName; * @see KafkaConsumer * @see MockConsumer */ -public interface Consumer extends Closeable { +public interface Consumer extends Closeable { + + /** + * @see KafkaConsumer#subscriptions() + */ + public Set subscriptions(); /** - * Incrementally subscribe to the given list of topics. This API is mutually exclusive to - * {@link #subscribe(TopicPartition...) subscribe(partitions)} - * @param topics A variable list of topics that the consumer subscribes to - */ - public void subscribe(String...topics); + * @see KafkaConsumer#subscribe(String...) + */ + public void subscribe(String... topics); /** - * Incrementally subscribes to a specific topic and partition. This API is mutually exclusive to - * {@link #subscribe(String...) subscribe(topics)} - * @param partitions Partitions to subscribe to - */ + * @see KafkaConsumer#subscribe(TopicPartition...) + */ public void subscribe(TopicPartition... partitions); /** - * Unsubscribe from the specific topics. Messages for this topic will not be returned from the next {@link #poll(long) poll()} - * onwards. This should be used in conjunction with {@link #subscribe(String...) subscribe(topics)}. It is an error to - * unsubscribe from a topic that was never subscribed to using {@link #subscribe(String...) subscribe(topics)} - * @param topics Topics to unsubscribe from + * @see KafkaConsumer#unsubscribe(String...) */ public void unsubscribe(String... topics); /** - * Unsubscribe from the specific topic partitions. Messages for these partitions will not be returned from the next - * {@link #poll(long) poll()} onwards. This should be used in conjunction with - * {@link #subscribe(TopicPartition...) subscribe(topic, partitions)}. It is an error to - * unsubscribe from a partition that was never subscribed to using {@link #subscribe(TopicPartition...) subscribe(partitions)} - * @param partitions Partitions to unsubscribe from + * @see KafkaConsumer#unsubscribe(TopicPartition...) */ public void unsubscribe(TopicPartition... partitions); - + /** - * Fetches data for the subscribed list of topics and partitions - * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits indefinitely. Must not be negative - * @return Map of topic to records for the subscribed topics and partitions as soon as data is available for a topic partition. Availability - * of data is controlled by {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG} and {@link ConsumerConfig#FETCH_MAX_WAIT_MS_CONFIG}. - * If no data is available for timeout ms, returns an empty list + * @see KafkaConsumer#poll(long) */ - public Map> poll(long timeout); + public ConsumerRecords poll(long timeout); /** - * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions. - * @param sync If true, the commit should block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false + * @see KafkaConsumer#commit(CommitType) */ - public OffsetMetadata commit(boolean sync); + public void commit(CommitType commitType); /** - * Commits the specified offsets for the specified list of topics and partitions to Kafka. - * @param offsets The map of offsets to commit for the given topic partitions - * @param sync If true, commit will block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false. + * @see KafkaConsumer#commit(Map, CommitType) */ - public OffsetMetadata commit(Map offsets, boolean sync); - + public void commit(Map offsets, CommitType commitType); + /** - * Overrides the fetch positions that the consumer will use on the next fetch request. If the consumer subscribes to a list of topics - * using {@link #subscribe(String...) subscribe(topics)}, an exception will be thrown if the specified topic partition is not owned by - * the consumer. - * @param offsets The map of fetch positions per topic and partition + * @see KafkaConsumer#seek(TopicPartition, long) */ - public void seek(Map offsets); + public void seek(TopicPartition partition, long offset); /** - * Returns the fetch position of the next message for the specified topic partition to be used on the next {@link #poll(long) poll()} - * @param partitions Partitions for which the fetch position will be returned - * @return The position from which data will be fetched for the specified partition on the next {@link #poll(long) poll()} + * @see KafkaConsumer#seekToBeginning(TopicPartition...) */ - public Map position(Collection partitions); - + public void seekToBeginning(TopicPartition... partitions); + /** - * Fetches the last committed offsets for the input list of partitions - * @param partitions The list of partitions to return the last committed offset for - * @return The list of offsets for the specified list of partitions + * @see KafkaConsumer#seekToEnd(TopicPartition...) */ - public Map committed(Collection partitions); - + public void seekToEnd(TopicPartition... partitions); + + /** + * @see KafkaConsumer#position(TopicPartition) + */ + public long position(TopicPartition partition); + /** - * Fetches offsets before a certain timestamp - * @param timestamp The unix timestamp. Value -1 indicates earliest available timestamp. Value -2 indicates latest available timestamp. - * @param partitions The list of partitions for which the offsets are returned - * @return The offsets for messages that were written to the server before the specified timestamp. + * @see KafkaConsumer#committed(TopicPartition) */ - public Map offsetsBeforeTime(long timestamp, Collection partitions); + public long committed(TopicPartition partition); /** - * Return a map of metrics maintained by the consumer + * @see KafkaConsumer#metrics() */ public Map metrics(); /** - * Close this consumer + * @see KafkaConsumer#partitionsFor(String) + */ + public List partitionsFor(String topic); + + /** + * @see KafkaConsumer#close() */ public void close(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 57c1807..6d4ff7c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -9,13 +9,16 @@ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. -*/ + */ package org.apache.kafka.clients.consumer; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; +import static org.apache.kafka.common.config.ConfigDef.ValidString.in; import java.util.Map; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceCallback; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -27,130 +30,121 @@ import org.apache.kafka.common.config.ConfigDef.Type; public class ConsumerConfig extends AbstractConfig { private static final ConfigDef config; - /** - * The identifier of the group this consumer belongs to. This is required if the consumer uses either the - * group management functionality by using {@link Consumer#subscribe(String...) subscribe(topics)}. This is also required - * if the consumer uses the default Kafka based offset management strategy. + /* + * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS + * THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE. */ - public static final String GROUP_ID_CONFIG = "group.id"; - + /** - * The timeout after which, if the {@link Consumer#poll(long) poll(timeout)} is not invoked, the consumer is - * marked dead and a rebalance operation is triggered for the group identified by {@link #GROUP_ID_CONFIG}. Relevant - * if the consumer uses the group management functionality by invoking {@link Consumer#subscribe(String...) subscribe(topics)} + * group.id */ - public static final String SESSION_TIMEOUT_MS = "session.timeout.ms"; + public static final String GROUP_ID_CONFIG = "group.id"; + private static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using subscribe(topic) or the Kafka-based offset management strategy."; /** - * The number of times a consumer sends a heartbeat to the co-ordinator broker within a {@link #SESSION_TIMEOUT_MS} time window. - * This frequency affects the latency of a rebalance operation since the co-ordinator broker notifies a consumer of a rebalance - * in the heartbeat response. Relevant if the consumer uses the group management functionality by invoking - * {@link Consumer#subscribe(String...) subscribe(topics)} + * session.timeout.ms */ - public static final String HEARTBEAT_FREQUENCY = "heartbeat.frequency"; + public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms"; + private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities."; /** - * A list of URLs to use for establishing the initial connection to the cluster. This list should be in the form - * host1:port1,host2:port2,.... These urls are just used for the initial connection to discover the - * full cluster membership (which may change dynamically) so this list need not contain the full set of servers (you - * may want more than one, though, in case a server is down). + * bootstrap.servers */ - public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; /** - * If true, periodically commit to Kafka the offsets of messages already returned by the consumer. This committed - * offset will be used when the process fails as the position from which the consumption will begin. + * enable.auto.commit */ public static final String ENABLE_AUTO_COMMIT_CONFIG = "enable.auto.commit"; - + private static final String ENABLE_AUTO_COMMIT_DOC = "If true the consumer's offset will be periodically committed in the background."; + /** - * The friendly name of the partition assignment strategy that the server will use to distribute partition ownership - * amongst consumer instances when group management is used + * auto.commit.interval.ms */ - public static final String PARTITION_ASSIGNMENT_STRATEGY = "partition.assignment.strategy"; - + public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms"; + private static final String AUTO_COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if enable.auto.commit is set to true."; + /** - * The frequency in milliseconds that the consumer offsets are committed to Kafka. Relevant if {@link #ENABLE_AUTO_COMMIT_CONFIG} - * is turned on. + * partition.assignment.strategy */ - public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms"; - + public static final String PARTITION_ASSIGNMENT_STRATEGY_CONFIG = "partition.assignment.strategy"; + private static final String PARTITION_ASSIGNMENT_STRATEGY_DOC = "The friendly name of the partition assignment strategy that the server will use to distribute partition ownership amongst consumer instances when group management is used"; + /** - * What to do when there is no initial offset in Kafka or if an offset is out of range: - *
    - *
  • smallest: automatically reset the offset to the smallest offset - *
  • largest: automatically reset the offset to the largest offset - *
  • disable: throw exception to the consumer if no previous offset is found for the consumer's group - *
  • anything else: throw exception to the consumer. - *
+ * auto.offset.reset */ public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset"; - + private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted):
  • smallest: automatically reset the offset to the smallest offset
  • largest: automatically reset the offset to the largest offset
  • none: throw exception to the consumer if no previous offset is found for the consumer's group
  • anything else: throw exception to the consumer.
"; + /** - * The minimum amount of data the server should return for a fetch request. If insufficient data is available the - * request will wait for that much data to accumulate before answering the request. + * fetch.min.bytes */ public static final String FETCH_MIN_BYTES_CONFIG = "fetch.min.bytes"; - + private static final String FETCH_MIN_BYTES_DOC = "The minimum amount of data the server should return for a fetch request. If insufficient data is available the request will wait for that much data to accumulate before answering the request. The default setting of 1 byte means that fetch requests are answered as soon as a single byte of data is available or the fetch request times out waiting for data to arrive. Setting this to something greater than 1 will cause the server to wait for larger amounts of data to accumulate which can improve server throughput a bit at the cost of some additional latency."; + /** - * The maximum amount of time the server will block before answering the fetch request if there isn't sufficient - * data to immediately satisfy {@link #FETCH_MIN_BYTES_CONFIG}. This should be less than or equal to the timeout used in - * {@link KafkaConsumer#poll(long) poll(timeout)} + * fetch.max.wait.ms */ public static final String FETCH_MAX_WAIT_MS_CONFIG = "fetch.max.wait.ms"; - + private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch.min.bytes."; + + /** metadata.max.age.ms */ + public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; + /** - * The maximum amount of time to block waiting to fetch metadata about a topic the first time a record is received - * from that topic. The consumer will throw a TimeoutException if it could not successfully fetch metadata within - * this timeout. + * max.partition.fetch.bytes */ - public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms"; + public static final String MAX_PARTITION_FETCH_BYTES_CONFIG = "max.partition.fetch.bytes"; + private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server will return. The maximum total memory used for a request will be #partitions * max.partition.fetch.bytes. This size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large message on a certain partition."; + + /** send.buffer.bytes */ + public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; + + /** receive.buffer.bytes */ + public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; /** - * The total memory used by the consumer to buffer records received from the server. This config is meant to control - * the consumer's memory usage, so it is the size of the global fetch buffer that will be shared across all partitions. + * client.id */ - public static final String TOTAL_BUFFER_MEMORY_CONFIG = "total.memory.bytes"; + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; /** - * The minimum amount of memory that should be used to fetch at least one message for a partition. This puts a lower - * bound on the consumer's memory utilization when there is at least one message for a partition available on the server. - * This size must be at least as large as the maximum message size the server allows or else it is possible for the producer - * to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large - * message on a certain partition. + * reconnect.backoff.ms */ - public static final String FETCH_BUFFER_CONFIG = "fetch.buffer.bytes"; - + public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; + /** - * The id string to pass to the server when making requests. The purpose of this is to be able to track the source - * of requests beyond just ip/port by allowing a logical application name to be included. + * retry.backoff.ms */ - public static final String CLIENT_ID_CONFIG = "client.id"; + public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; /** - * The size of the TCP send buffer to use when fetching data + * metrics.sample.window.ms */ - public static final String SOCKET_RECEIVE_BUFFER_CONFIG = "socket.receive.buffer.bytes"; + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; /** - * The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a - * host in a tight loop. This backoff applies to all requests sent by the consumer to the broker. + * metrics.num.samples */ - public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; + public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; - /** metrics.sample.window.ms */ - public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; - private static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The metrics system maintains a configurable number of samples over a fixed window size. This configuration " + "controls the size of the window. For example we might maintain two samples each measured over a 30 second period. " - + "When a window expires we erase and overwrite the oldest window."; - - /** metrics.num.samples */ - public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples"; - private static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics."; + /** + * metric.reporters + */ + public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG; - /** metric.reporters */ - public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; - private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the MetricReporter interface allows " + "plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; + /** + * rebalance.callback.class + */ + public static final String CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG = "rebalance.callback.class"; + private static final String CONSUMER_REBALANCE_CALLBACK_CLASS_DOC = "A user-provided callback to execute when partition assignments change."; + /** + * check.crcs + */ + public static final String CHECK_CRCS_CONFIG = "check.crcs"; + private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance."; + /** key.deserializer */ public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; @@ -160,38 +154,134 @@ public class ConsumerConfig extends AbstractConfig { private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; static { - /* TODO: add config docs */ - config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, "blah blah") - .define(GROUP_ID_CONFIG, Type.STRING, Importance.HIGH, "blah blah") - .define(SESSION_TIMEOUT_MS, Type.LONG, 1000, Importance.HIGH, "blah blah") - .define(HEARTBEAT_FREQUENCY, Type.INT, 3, Importance.MEDIUM, "blah blah") - .define(PARTITION_ASSIGNMENT_STRATEGY, Type.STRING, Importance.MEDIUM, "blah blah") - .define(METADATA_FETCH_TIMEOUT_CONFIG, Type.LONG, 60 * 1000, atLeast(0), Importance.MEDIUM, "blah blah") - .define(ENABLE_AUTO_COMMIT_CONFIG, Type.BOOLEAN, true, Importance.MEDIUM, "blah blah") - .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, Type.LONG, 5000, atLeast(0), Importance.LOW, "blah blah") - .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.LOW, "blah blah") - .define(TOTAL_BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.LOW, "blah blah") - .define(FETCH_BUFFER_CONFIG, Type.INT, 1 * 1024 * 1024, atLeast(0), Importance.HIGH, "blah blah") - .define(SOCKET_RECEIVE_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.LOW, "blah blah") - .define(FETCH_MIN_BYTES_CONFIG, Type.LONG, 1024, atLeast(0), Importance.HIGH, "blah blah") - .define(FETCH_MAX_WAIT_MS_CONFIG, Type.LONG, 500, atLeast(0), Importance.LOW, "blah blah") - .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), Importance.LOW, "blah blah") - .define(AUTO_OFFSET_RESET_CONFIG, Type.STRING, "largest", Importance.MEDIUM, "blah blah") + config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, + Type.LIST, + Importance.HIGH, + CommonClientConfigs.BOOSTRAP_SERVERS_DOC) + .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC) + .define(SESSION_TIMEOUT_MS_CONFIG, + Type.LONG, + 30000, + Importance.HIGH, + SESSION_TIMEOUT_MS_DOC) + .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, + Type.STRING, + "blah", + Importance.MEDIUM, + PARTITION_ASSIGNMENT_STRATEGY_DOC) + .define(METADATA_MAX_AGE_CONFIG, + Type.LONG, + 5 * 60 * 1000, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METADATA_MAX_AGE_DOC) + .define(ENABLE_AUTO_COMMIT_CONFIG, + Type.BOOLEAN, + true, + Importance.MEDIUM, + ENABLE_AUTO_COMMIT_DOC) + .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, + Type.LONG, + 5000, + atLeast(0), + Importance.LOW, + AUTO_COMMIT_INTERVAL_MS_DOC) + .define(CLIENT_ID_CONFIG, + Type.STRING, + "", + Importance.LOW, + CommonClientConfigs.CLIENT_ID_DOC) + .define(MAX_PARTITION_FETCH_BYTES_CONFIG, + Type.INT, + 1 * 1024 * 1024, + atLeast(0), + Importance.HIGH, + MAX_PARTITION_FETCH_BYTES_DOC) + .define(SEND_BUFFER_CONFIG, + Type.INT, + 128 * 1024, + atLeast(0), + Importance.MEDIUM, + CommonClientConfigs.SEND_BUFFER_DOC) + .define(RECEIVE_BUFFER_CONFIG, + Type.INT, + 32 * 1024, + atLeast(0), + Importance.MEDIUM, + CommonClientConfigs.RECEIVE_BUFFER_DOC) + .define(FETCH_MIN_BYTES_CONFIG, + Type.INT, + 1024, + atLeast(0), + Importance.HIGH, + FETCH_MIN_BYTES_DOC) + .define(FETCH_MAX_WAIT_MS_CONFIG, + Type.INT, + 500, + atLeast(0), + Importance.LOW, + FETCH_MAX_WAIT_MS_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, + Type.LONG, + 50L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, + Type.LONG, + 100L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RETRY_BACKOFF_MS_DOC) + .define(AUTO_OFFSET_RESET_CONFIG, + Type.STRING, + "latest", + in("latest", "earliest", "none"), + Importance.MEDIUM, + AUTO_OFFSET_RESET_DOC) + .define(CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG, + Type.CLASS, + NoOpConsumerRebalanceCallback.class, + Importance.LOW, + CONSUMER_REBALANCE_CALLBACK_CLASS_DOC) + .define(CHECK_CRCS_CONFIG, + Type.BOOLEAN, + true, + Importance.LOW, + CHECK_CRCS_DOC) .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, Type.LONG, 30000, atLeast(0), Importance.LOW, - METRICS_SAMPLE_WINDOW_MS_DOC) - .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC) - .define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_DESERIALIZER_CLASS_DOC) - .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC); - + CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) + .define(METRICS_NUM_SAMPLES_CONFIG, + Type.INT, + 2, + atLeast(1), + Importance.LOW, + CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, + "", + Importance.LOW, + CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) + .define(KEY_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + KEY_DESERIALIZER_CLASS_DOC) + .define(VALUE_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + VALUE_DESERIALIZER_CLASS_DOC); } ConsumerConfig(Map props) { super(config, props); } + public static void main(String[] args) { + System.out.println(config.toHtmlTable()); + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java index e4cf7d1..74dfdba 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java @@ -9,7 +9,7 @@ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. -*/ + */ package org.apache.kafka.clients.consumer; import java.util.Collection; @@ -17,34 +17,77 @@ import java.util.Collection; import org.apache.kafka.common.TopicPartition; /** - * A callback interface that the user can implement to manage customized offsets on the start and end of - * every rebalance operation. This callback will execute in the user thread as part of the - * {@link Consumer#poll(long) poll(long)} API on every rebalance attempt. - * Default implementation of the callback will {@link Consumer#seek(java.util.Map) seek(offsets)} to the last committed offsets in the - * {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned()} callback. And will commit offsets synchronously - * for the specified list of partitions to Kafka in the {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked()} - * callback. + * A callback interface that the user can implement to trigger custom actions when the set of partitions assigned to the + * consumer changes. + *

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Offsets and Consumer Position

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

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

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

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

Consumer Groups

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

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

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

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

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

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

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

Usage Examples

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

Simple Processing

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

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

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

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

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

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

Controlling When Messages Are Considered Consumed

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

Subscribing To Specific Partitions

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

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

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

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

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

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

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

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

Managing Your Own Offsets

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

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

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

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

Controlling The Consumer's Position

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

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

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

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

Multithreaded Processing

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

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

1. One Consumer Per Thread

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

2. Decouple Consumption and Processing

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

    - * The provided network sends will be started. * * When this call is completed the user can check for completed sends, receives, connections or disconnects using * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These @@ -190,29 +209,13 @@ public class Selector implements Selectable { * completed I/O. * * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely. - * @param sends The list of new sends to begin - * * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is * already an in-progress send */ @Override - public void poll(long timeout, List sends) throws IOException { + public void poll(long timeout) throws IOException { clear(); - /* register for write interest on any new sends */ - for (NetworkSend send : sends) { - SelectionKey key = keyForId(send.destination()); - Transmissions transmissions = transmissions(key); - if (transmissions.hasSend()) - throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress."); - transmissions.send = send; - try { - key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); - } catch (CancelledKeyException e) { - close(key); - } - } - /* check ready keys */ long startSelect = time.nanoseconds(); int readyKeys = select(timeout); @@ -266,21 +269,34 @@ public class Selector implements Selectable { } /* cancel any defunct sockets */ - if (!key.isValid()) + if (!key.isValid()) { close(key); + this.disconnected.add(transmissions.id); + } } catch (IOException e) { - InetAddress remoteAddress = null; - Socket socket = channel.socket(); - if (socket != null) - remoteAddress = socket.getInetAddress(); - log.warn("Error in I/O with {}", remoteAddress , e); + String desc = socketDescription(channel); + if(e instanceof EOFException) + log.info("Connection {} disconnected", desc); + else + log.warn("Error in I/O with connection to {}", desc, e); close(key); + this.disconnected.add(transmissions.id); } } } long endIo = time.nanoseconds(); this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); } + + private String socketDescription(SocketChannel channel) { + Socket socket = channel.socket(); + if(socket == null) + return "[unconnected socket]"; + else if(socket.getInetAddress() != null) + return socket.getInetAddress().toString(); + else + return socket.getLocalAddress().toString(); + } @Override public List completedSends() { @@ -302,6 +318,36 @@ public class Selector implements Selectable { return this.connected; } + @Override + public void mute(int id) { + mute(this.keyForId(id)); + } + + private void mute(SelectionKey key) { + key.interestOps(key.interestOps() & ~SelectionKey.OP_READ); + } + + @Override + public void unmute(int id) { + unmute(this.keyForId(id)); + } + + private void unmute(SelectionKey key) { + key.interestOps(key.interestOps() | SelectionKey.OP_READ); + } + + @Override + public void muteAll() { + for (SelectionKey key : this.keys.values()) + mute(key); + } + + @Override + public void unmuteAll() { + for (SelectionKey key : this.keys.values()) + unmute(key); + } + /** * Clear the results from the prior poll */ @@ -310,6 +356,8 @@ public class Selector implements Selectable { this.completedReceives.clear(); this.connected.clear(); this.disconnected.clear(); + this.disconnected.addAll(this.failedSends); + this.failedSends.clear(); } /** @@ -335,7 +383,6 @@ public class Selector implements Selectable { SocketChannel channel = channel(key); Transmissions trans = transmissions(key); if (trans != null) { - this.disconnected.add(trans.id); this.keys.remove(trans.id); trans.clearReceive(); trans.clearSend(); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 3316b6a..a8deac4 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -19,36 +19,62 @@ package org.apache.kafka.common.protocol; import java.util.HashMap; import java.util.Map; -import org.apache.kafka.common.errors.*; - +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.errors.CorruptRecordException; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.LeaderNotAvailableException; +import org.apache.kafka.common.errors.NetworkException; +import org.apache.kafka.common.errors.NotEnoughReplicasAfterAppendException; +import org.apache.kafka.common.errors.NotEnoughReplicasException; +import org.apache.kafka.common.errors.NotLeaderForPartitionException; +import org.apache.kafka.common.errors.OffsetMetadataTooLarge; +import org.apache.kafka.common.errors.OffsetOutOfRangeException; +import org.apache.kafka.common.errors.RecordBatchTooLargeException; +import org.apache.kafka.common.errors.RecordTooLargeException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; /** * This class contains all the client-server errors--those errors that must be sent from the server to the client. These * are thus part of the protocol. The names can be changed but the error code cannot. - * + * * Do not add exceptions that occur only on the client or only on the server here. */ public enum Errors { UNKNOWN(-1, new UnknownServerException("The server experienced an unexpected error when processing the request")), NONE(0, null), - OFFSET_OUT_OF_RANGE(1, new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")), - CORRUPT_MESSAGE(2, new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")), - UNKNOWN_TOPIC_OR_PARTITION(3, new UnknownTopicOrPartitionException("This server does not host this topic-partition.")), + OFFSET_OUT_OF_RANGE(1, + new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")), + CORRUPT_MESSAGE(2, + new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")), + UNKNOWN_TOPIC_OR_PARTITION(3, + new UnknownTopicOrPartitionException("This server does not host this topic-partition.")), // TODO: errorCode 4 for InvalidFetchSize - LEADER_NOT_AVAILABLE(5, new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")), - NOT_LEADER_FOR_PARTITION(6, new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")), + LEADER_NOT_AVAILABLE(5, + new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")), + NOT_LEADER_FOR_PARTITION(6, + new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")), REQUEST_TIMED_OUT(7, new TimeoutException("The request timed out.")), - // TODO: errorCode 8, 9, 11 - MESSAGE_TOO_LARGE(10, new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")), + MESSAGE_TOO_LARGE(10, + new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")), OFFSET_METADATA_TOO_LARGE(12, new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")), NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")), - // TODO: errorCode 14, 15, 16 - INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), - RECORD_LIST_TOO_LARGE(18, new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")), - NOT_ENOUGH_REPLICAS(19, new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")), - NOT_ENOUGH_REPLICAS_AFTER_APPEND(20, new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")); + OFFSET_LOAD_IN_PROGRESS(14, new ApiException("The coordinator is loading offsets and can't process requests.")), + CONSUMER_COORDINATOR_NOT_AVAILABLE(15, new ApiException("The coordinator is not available.")), + NOT_COORDINATOR_FOR_CONSUMER(16, new ApiException("This is not the correct co-ordinator for this consumer.")), + INVALID_TOPIC_EXCEPTION(17, + new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), + RECORD_LIST_TOO_LARGE(18, + new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")), + NOT_ENOUGH_REPLICAS(19, + new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")), + NOT_ENOUGH_REPLICAS_AFTER_APPEND(20, + new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")); + private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); + static { for (Errors error : Errors.values()) { codeToError.put(error.code(), error); @@ -84,8 +110,9 @@ public enum Errors { * Throw the exception corresponding to this error if there is one */ public void maybeThrow() { - if (exception != null) + if (exception != null) { throw this.exception; + } } /** diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index 121e880..ee1f78f 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -1,18 +1,14 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.protocol.types; @@ -124,14 +120,6 @@ public class Struct { return (Long) get(name); } - public ByteBuffer getBytes(Field field) { - return (ByteBuffer) get(field); - } - - public ByteBuffer getBytes(String name) { - return (ByteBuffer) get(name); - } - public Object[] getArray(Field field) { return (Object[]) get(field); } @@ -148,6 +136,14 @@ public class Struct { return (String) get(name); } + public ByteBuffer getBytes(Field field) { + return (ByteBuffer) get(field); + } + + public ByteBuffer getBytes(String name) { + return (ByteBuffer) get(name); + } + /** * Set the given field to the specified value * @@ -175,9 +171,9 @@ public class Struct { } /** - * Create a struct for the schema of a container type (struct or array). - * Note that for array type, this method assumes that the type is an array of schema and creates a struct - * of that schema. Arrays of other types can't be instantiated with this method. + * Create a struct for the schema of a container type (struct or array). Note that for array type, this method + * assumes that the type is an array of schema and creates a struct of that schema. Arrays of other types can't be + * instantiated with this method. * * @param field The field to create an instance of * @return The struct diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java index e4d688c..2e54b56 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java +++ b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java @@ -41,4 +41,8 @@ public final class LogEntry { public String toString() { return "LogEntry(" + offset + ", " + record + ")"; } + + public int size() { + return record.size() + Records.LOG_OVERHEAD; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 040e5b9..cc4084f 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -55,7 +55,7 @@ public class MemoryRecords implements Records { return emptyRecords(buffer, type, buffer.capacity()); } - public static MemoryRecords iterableRecords(ByteBuffer buffer) { + public static MemoryRecords readableRecords(ByteBuffer buffer) { return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity()); } @@ -94,22 +94,21 @@ public class MemoryRecords implements Records { * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be * accurate if compression is really used. When this happens, the following append may cause dynamic buffer * re-allocation in the underlying byte buffer stream. - * + * * Also note that besides the records' capacity, there is also a size limit for the batch. This size limit may be * smaller than the capacity (e.g. when appending a single message whose size is larger than the batch size, the - * capacity will be the message size, but the size limit will still be the batch size), and when the records' size has - * exceed this limit we also mark this record as full. + * capacity will be the message size, but the size limit will still be the batch size), and when the records' size + * has exceed this limit we also mark this record as full. */ public boolean hasRoomFor(byte[] key, byte[] value) { - return this.writable && - this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value) && - this.sizeLimit >= this.compressor.estimatedBytesWritten(); + return this.writable && this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + + Record.recordSize(key, value) && + this.sizeLimit >= this.compressor.estimatedBytesWritten(); } public boolean isFull() { - return !this.writable || - this.capacity <= this.compressor.estimatedBytesWritten() || - this.sizeLimit <= this.compressor.estimatedBytesWritten(); + return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten() || + this.sizeLimit <= this.compressor.estimatedBytesWritten(); } /** @@ -132,7 +131,7 @@ public class MemoryRecords implements Records { public int sizeInBytes() { return compressor.buffer().position(); } - + /** * The compression rate of this record set */ @@ -162,6 +161,25 @@ public class MemoryRecords implements Records { ByteBuffer copy = (ByteBuffer) this.buffer.duplicate().flip(); return new RecordsIterator(copy, CompressionType.NONE, false); } + + @Override + public String toString() { + Iterator iter = iterator(); + StringBuilder builder = new StringBuilder(); + builder.append('['); + while(iter.hasNext()) { + LogEntry entry = iter.next(); + builder.append('('); + builder.append("offset="); + builder.append(entry.offset()); + builder.append(","); + builder.append("record="); + builder.append(entry.record()); + builder.append(")"); + } + builder.append(']'); + return builder.toString(); + } public static class RecordsIterator extends AbstractIterator { private final ByteBuffer buffer; @@ -174,7 +192,7 @@ public class MemoryRecords implements Records { this.type = type; this.buffer = buffer; this.shallow = shallow; - stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); + this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); } /* @@ -199,7 +217,10 @@ public class MemoryRecords implements Records { ByteBuffer rec; if (type == CompressionType.NONE) { rec = buffer.slice(); - buffer.position(buffer.position() + size); + int newPos = buffer.position() + size; + if(newPos > buffer.limit()) + return allDone(); + buffer.position(newPos); rec.limit(size); } else { byte[] recordBuffer = new byte[size]; @@ -207,7 +228,6 @@ public class MemoryRecords implements Records { rec = ByteBuffer.wrap(recordBuffer); } LogEntry entry = new LogEntry(offset, new Record(rec)); - entry.record().ensureValid(); // decide whether to go shallow or deep iteration if it is compressed CompressionType compression = entry.record().compressionType(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java index 99b52c2..4c99d4a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java @@ -20,13 +20,14 @@ import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class ConsumerMetadataRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id); - private static String GROUP_ID_KEY_NAME = "group_id"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; private final String groupId; public ConsumerMetadataRequest(String groupId) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(GROUP_ID_KEY_NAME, groupId); this.groupId = groupId; @@ -42,6 +43,6 @@ public class ConsumerMetadataRequest extends AbstractRequestResponse { } public static ConsumerMetadataRequest parse(ByteBuffer buffer) { - return new ConsumerMetadataRequest(((Struct) curSchema.read(buffer))); + return new ConsumerMetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java index 8b8f591..173333b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java @@ -21,20 +21,21 @@ import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class ConsumerMetadataResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String COORDINATOR_KEY_NAME = "coordinator"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id); + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String COORDINATOR_KEY_NAME = "coordinator"; // coordinator level field names - private static String NODE_ID_KEY_NAME = "node_id"; - private static String HOST_KEY_NAME = "host"; - private static String PORT_KEY_NAME = "port"; + private static final String NODE_ID_KEY_NAME = "node_id"; + private static final String HOST_KEY_NAME = "host"; + private static final String PORT_KEY_NAME = "port"; private final short errorCode; private final Node node; public ConsumerMetadataResponse(short errorCode, Node node) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(ERROR_CODE_KEY_NAME, errorCode); Struct coordinator = struct.instance(COORDINATOR_KEY_NAME); coordinator.set(NODE_ID_KEY_NAME, node.id()); @@ -64,6 +65,6 @@ public class ConsumerMetadataResponse extends AbstractRequestResponse { } public static ConsumerMetadataResponse parse(ByteBuffer buffer) { - return new ConsumerMetadataResponse(((Struct) curSchema.read(buffer))); + return new ConsumerMetadataResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 2fc471f..2529a09 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -1,21 +1,23 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.requests; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; @@ -23,27 +25,23 @@ import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - public class FetchRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id); - private static String REPLICA_ID_KEY_NAME = "replica_id"; - private static String MAX_WAIT_KEY_NAME = "max_wait_time"; - private static String MIN_BYTES_KEY_NAME = "min_bytes"; - private static String TOPICS_KEY_NAME = "topics"; + + public static final int CONSUMER_REPLICA_ID = -1; + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id); + private static final String REPLICA_ID_KEY_NAME = "replica_id"; + private static final String MAX_WAIT_KEY_NAME = "max_wait_time"; + private static final String MIN_BYTES_KEY_NAME = "min_bytes"; + private static final String TOPICS_KEY_NAME = "topics"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String FETCH_OFFSET_KEY_NAME = "fetch_offset"; - private static String MAX_BYTES_KEY_NAME = "max_bytes"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset"; + private static final String MAX_BYTES_KEY_NAME = "max_bytes"; private final int replicaId; private final int maxWait; @@ -60,15 +58,25 @@ public class FetchRequest extends AbstractRequestResponse { } } + /** + * Create a non-replica fetch request + */ + public FetchRequest(int maxWait, int minBytes, Map fetchData) { + this(CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData); + } + + /** + * Create a replica fetch request + */ public FetchRequest(int replicaId, int maxWait, int minBytes, Map fetchData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(fetchData); struct.set(REPLICA_ID_KEY_NAME, replicaId); struct.set(MAX_WAIT_KEY_NAME, maxWait); struct.set(MIN_BYTES_KEY_NAME, minBytes); List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { + for (Map.Entry> topicEntry : topicsData.entrySet()) { Struct topicData = struct.instance(TOPICS_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); List partitionArray = new ArrayList(); @@ -127,6 +135,6 @@ public class FetchRequest extends AbstractRequestResponse { } public static FetchRequest parse(ByteBuffer buffer) { - return new FetchRequest(((Struct) curSchema.read(buffer))); + return new FetchRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index f719010..c1e5f44 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -30,18 +30,19 @@ import java.util.List; import java.util.Map; public class FetchResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String HIGH_WATERMARK_KEY_NAME = "high_watermark"; - private static String RECORD_SET_KEY_NAME = "record_set"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark"; + private static final String RECORD_SET_KEY_NAME = "record_set"; private final Map responseData; @@ -58,7 +59,7 @@ public class FetchResponse extends AbstractRequestResponse { } public FetchResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); @@ -105,6 +106,6 @@ public class FetchResponse extends AbstractRequestResponse { } public static FetchResponse parse(ByteBuffer buffer) { - return new FetchResponse(((Struct) curSchema.read(buffer))); + return new FetchResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index 9512db2..cfdb5de 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -20,17 +20,18 @@ import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class HeartbeatRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; private final String groupId; private final int groupGenerationId; private final String consumerId; public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(GROUP_ID_KEY_NAME, groupId); struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId); struct.set(CONSUMER_ID_KEY_NAME, consumerId); @@ -59,6 +60,6 @@ public class HeartbeatRequest extends AbstractRequestResponse { } public static HeartbeatRequest parse(ByteBuffer buffer) { - return new HeartbeatRequest(((Struct) curSchema.read(buffer))); + return new HeartbeatRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index 8997ffc..ea964f7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -20,12 +20,13 @@ import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class HeartbeatResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); + private static final String ERROR_CODE_KEY_NAME = "error_code"; private final short errorCode; public HeartbeatResponse(short errorCode) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(ERROR_CODE_KEY_NAME, errorCode); this.errorCode = errorCode; } @@ -40,6 +41,6 @@ public class HeartbeatResponse extends AbstractRequestResponse { } public static HeartbeatResponse parse(ByteBuffer buffer) { - return new HeartbeatResponse(((Struct) curSchema.read(buffer))); + return new HeartbeatResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index d6e91f3..a1d48c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -22,12 +22,13 @@ import java.util.ArrayList; import java.util.List; public class JoinGroupRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String SESSION_TIMEOUT_KEY_NAME = "session_timeout"; - private static String TOPICS_KEY_NAME = "topics"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String STRATEGY_KEY_NAME = "partition_assignment_strategy"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String SESSION_TIMEOUT_KEY_NAME = "session_timeout"; + private static final String TOPICS_KEY_NAME = "topics"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + private static final String STRATEGY_KEY_NAME = "partition_assignment_strategy"; private final String groupId; private final int sessionTimeout; @@ -36,7 +37,7 @@ public class JoinGroupRequest extends AbstractRequestResponse { private final String strategy; public JoinGroupRequest(String groupId, int sessionTimeout, List topics, String consumerId, String strategy) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(GROUP_ID_KEY_NAME, groupId); struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); struct.set(TOPICS_KEY_NAME, topics.toArray()); @@ -82,6 +83,6 @@ public class JoinGroupRequest extends AbstractRequestResponse { } public static JoinGroupRequest parse(ByteBuffer buffer) { - return new JoinGroupRequest(((Struct) curSchema.read(buffer))); + return new JoinGroupRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index efe8979..1e9f349 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -23,16 +23,17 @@ import java.nio.ByteBuffer; import java.util.*; public class JoinGroupResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions"; - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; - - public static int UNKNOWN_GENERATION_ID = -1; - public static String UNKNOWN_CONSUMER_ID = ""; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + private static final String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + public static final int UNKNOWN_GENERATION_ID = -1; + public static final String UNKNOWN_CONSUMER_ID = ""; private final short errorCode; private final int generationId; @@ -40,7 +41,7 @@ public class JoinGroupResponse extends AbstractRequestResponse { private final List assignedPartitions; public JoinGroupResponse(short errorCode, int generationId, String consumerId, List assignedPartitions) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions); @@ -97,6 +98,6 @@ public class JoinGroupResponse extends AbstractRequestResponse { } public static JoinGroupResponse parse(ByteBuffer buffer) { - return new JoinGroupResponse(((Struct) curSchema.read(buffer))); + return new JoinGroupResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index 99364c1..05c5fed 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -30,18 +30,19 @@ import java.util.List; import java.util.Map; public class ListOffsetRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id); - private static String REPLICA_ID_KEY_NAME = "replica_id"; - private static String TOPICS_KEY_NAME = "topics"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id); + private static final String REPLICA_ID_KEY_NAME = "replica_id"; + private static final String TOPICS_KEY_NAME = "topics"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String TIMESTAMP_KEY_NAME = "timestamp"; - private static String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String TIMESTAMP_KEY_NAME = "timestamp"; + private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets"; private final int replicaId; private final Map offsetData; @@ -55,9 +56,13 @@ public class ListOffsetRequest extends AbstractRequestResponse { this.maxNumOffsets = maxNumOffsets; } } + + public ListOffsetRequest(Map offsetData) { + this(-1, offsetData); + } public ListOffsetRequest(int replicaId, Map offsetData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); struct.set(REPLICA_ID_KEY_NAME, replicaId); @@ -109,6 +114,6 @@ public class ListOffsetRequest extends AbstractRequestResponse { } public static ListOffsetRequest parse(ByteBuffer buffer) { - return new ListOffsetRequest(((Struct) curSchema.read(buffer))); + return new ListOffsetRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java index ac23971..b2e473e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -30,17 +30,18 @@ import java.util.List; import java.util.Map; public class ListOffsetResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String OFFSETS_KEY_NAME = "offsets"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String OFFSETS_KEY_NAME = "offsets"; private final Map responseData; @@ -55,7 +56,7 @@ public class ListOffsetResponse extends AbstractRequestResponse { } public ListOffsetResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); @@ -103,6 +104,6 @@ public class ListOffsetResponse extends AbstractRequestResponse { } public static ListOffsetResponse parse(ByteBuffer buffer) { - return new ListOffsetResponse(((Struct) curSchema.read(buffer))); + return new ListOffsetResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index b22ca1d..0186783 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -22,13 +22,14 @@ import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; public class MetadataRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); - private static String TOPICS_KEY_NAME = "topics"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); + private static final String TOPICS_KEY_NAME = "topics"; private final List topics; public MetadataRequest(List topics) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(TOPICS_KEY_NAME, topics.toArray()); this.topics = topics; } @@ -47,6 +48,6 @@ public class MetadataRequest extends AbstractRequestResponse { } public static MetadataRequest parse(ByteBuffer buffer) { - return new MetadataRequest(((Struct) curSchema.read(buffer))); + return new MetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index d97962d..13daf59 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -28,32 +28,33 @@ import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; public class MetadataResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); - private static String BROKERS_KEY_NAME = "brokers"; - private static String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; + + private static Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); + private static final String BROKERS_KEY_NAME = "brokers"; + private static final String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; // broker level field names - private static String NODE_ID_KEY_NAME = "node_id"; - private static String HOST_KEY_NAME = "host"; - private static String PORT_KEY_NAME = "port"; + private static final String NODE_ID_KEY_NAME = "node_id"; + private static final String HOST_KEY_NAME = "host"; + private static final String PORT_KEY_NAME = "port"; // topic level field names - private static String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code"; - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_METADATA_KEY_NAME = "partition_metadata"; + private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata"; // partition level field names - private static String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code"; - private static String PARTITION_KEY_NAME = "partition_id"; - private static String LEADER_KEY_NAME = "leader"; - private static String REPLICAS_KEY_NAME = "replicas"; - private static String ISR_KEY_NAME = "isr"; + private static final String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code"; + private static final String PARTITION_KEY_NAME = "partition_id"; + private static final String LEADER_KEY_NAME = "leader"; + private static final String REPLICAS_KEY_NAME = "replicas"; + private static final String ISR_KEY_NAME = "isr"; private final Cluster cluster; private final Map errors; public MetadataResponse(Cluster cluster) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); List brokerArray = new ArrayList(); for (Node node: cluster.nodes()) { @@ -147,6 +148,6 @@ public class MetadataResponse extends AbstractRequestResponse { } public static MetadataResponse parse(ByteBuffer buffer) { - return new MetadataResponse(((Struct) curSchema.read(buffer))); + return new MetadataResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 3ee5cba..4fb48c8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -3,15 +3,21 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.requests; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; @@ -19,31 +25,26 @@ import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * This wrapper supports both v0 and v1 of OffsetCommitRequest. */ public class OffsetCommitRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String TOPICS_KEY_NAME = "topics"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + private static final String TOPICS_KEY_NAME = "topics"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String COMMIT_OFFSET_KEY_NAME = "offset"; - private static String TIMESTAMP_KEY_NAME = "timestamp"; - private static String METADATA_KEY_NAME = "metadata"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String COMMIT_OFFSET_KEY_NAME = "offset"; + private static final String TIMESTAMP_KEY_NAME = "timestamp"; + private static final String METADATA_KEY_NAME = "metadata"; public static final int DEFAULT_GENERATION_ID = -1; public static final String DEFAULT_CONSUMER_ID = ""; @@ -88,7 +89,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { * @param offsetData */ public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); initCommonFields(groupId, offsetData); struct.set(GENERATION_ID_KEY_NAME, generationId); @@ -104,7 +105,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { struct.set(GROUP_ID_KEY_NAME, groupId); List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { + for (Map.Entry> topicEntry : topicsData.entrySet()) { Struct topicData = struct.instance(TOPICS_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); List partitionArray = new ArrayList(); @@ -175,6 +176,6 @@ public class OffsetCommitRequest extends AbstractRequestResponse { } public static OffsetCommitRequest parse(ByteBuffer buffer) { - return new OffsetCommitRequest(((Struct) curSchema.read(buffer))); + return new OffsetCommitRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 711232a..2ab1dc6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -26,21 +26,22 @@ import java.util.List; import java.util.Map; public class OffsetCommitResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level fields - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level fields - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; private final Map responseData; public OffsetCommitResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); @@ -82,6 +83,6 @@ public class OffsetCommitResponse extends AbstractRequestResponse { } public static OffsetCommitResponse parse(ByteBuffer buffer) { - return new OffsetCommitResponse(((Struct) curSchema.read(buffer))); + return new OffsetCommitResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index 90d5135..333483f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -28,16 +28,17 @@ import java.util.Map; * This wrapper supports both v0 and v1 of OffsetFetchRequest. */ public class OffsetFetchRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String TOPICS_KEY_NAME = "topics"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String TOPICS_KEY_NAME = "topics"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; + private static final String PARTITION_KEY_NAME = "partition"; public static final int DEFAULT_GENERATION_ID = -1; public static final String DEFAULT_CONSUMER_ID = ""; @@ -46,7 +47,7 @@ public class OffsetFetchRequest extends AbstractRequestResponse { private final List partitions; public OffsetFetchRequest(String groupId, List partitions) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(partitions); @@ -93,6 +94,6 @@ public class OffsetFetchRequest extends AbstractRequestResponse { } public static OffsetFetchRequest parse(ByteBuffer buffer) { - return new OffsetFetchRequest(((Struct) curSchema.read(buffer))); + return new OffsetFetchRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 6b7c269..04c88c0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -3,43 +3,45 @@ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the * License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.requests; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - public class OffsetFetchResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level fields - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level fields - private static String PARTITION_KEY_NAME = "partition"; - private static String COMMIT_OFFSET_KEY_NAME = "offset"; - private static String METADATA_KEY_NAME = "metadata"; - private static String ERROR_CODE_KEY_NAME = "error_code"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String COMMIT_OFFSET_KEY_NAME = "offset"; + private static final String METADATA_KEY_NAME = "metadata"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; - private final Map responseData; + private final Map responseData; public static final class PartitionData { public final long offset; @@ -51,15 +53,19 @@ public class OffsetFetchResponse extends AbstractRequestResponse { this.metadata = metadata; this.errorCode = errorCode; } + + public boolean hasError() { + return this.errorCode != Errors.NONE.code(); + } } public OffsetFetchResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); - for (Map.Entry> entries: topicsData.entrySet()) { + for (Map.Entry> entries : topicsData.entrySet()) { Struct topicData = struct.instance(RESPONSES_KEY_NAME); topicData.set(TOPIC_KEY_NAME, entries.getKey()); List partitionArray = new ArrayList(); @@ -102,6 +108,6 @@ public class OffsetFetchResponse extends AbstractRequestResponse { } public static OffsetFetchResponse parse(ByteBuffer buffer) { - return new OffsetFetchResponse(((Struct) curSchema.read(buffer))); + return new OffsetFetchResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 3dbba8a..03a0ab1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -27,25 +27,26 @@ import java.util.List; import java.util.Map; public class ProduceRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id); - private static String ACKS_KEY_NAME = "acks"; - private static String TIMEOUT_KEY_NAME = "timeout"; - private static String TOPIC_DATA_KEY_NAME = "topic_data"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id); + private static final String ACKS_KEY_NAME = "acks"; + private static final String TIMEOUT_KEY_NAME = "timeout"; + private static final String TOPIC_DATA_KEY_NAME = "topic_data"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_DATA_KEY_NAME = "data"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_DATA_KEY_NAME = "data"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String RECORD_SET_KEY_NAME = "record_set"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String RECORD_SET_KEY_NAME = "record_set"; private final short acks; private final int timeout; private final Map partitionRecords; public ProduceRequest(short acks, int timeout, Map partitionRecords) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); struct.set(ACKS_KEY_NAME, acks); struct.set(TIMEOUT_KEY_NAME, timeout); @@ -100,6 +101,6 @@ public class ProduceRequest extends AbstractRequestResponse { } public static ProduceRequest parse(ByteBuffer buffer) { - return new ProduceRequest(((Struct) curSchema.read(buffer))); + return new ProduceRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 5220464..e42d7db 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -26,22 +26,23 @@ import java.util.List; import java.util.Map; public class ProduceResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String BASE_OFFSET_KEY_NAME = "base_offset"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String BASE_OFFSET_KEY_NAME = "base_offset"; private final Map responses; public ProduceResponse(Map responses) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); List topicDatas = new ArrayList(responseByTopic.size()); for (Map.Entry> entry : responseByTopic.entrySet()) { @@ -107,6 +108,6 @@ public class ProduceResponse extends AbstractRequestResponse { } public static ProduceResponse parse(ByteBuffer buffer) { - return new ProduceResponse(((Struct) curSchema.read(buffer))); + return new ProduceResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java index 3c001d3..13be6a3 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java @@ -29,7 +29,7 @@ public interface Deserializer { * @param isKey whether is for key or value */ public void configure(Map configs, boolean isKey); - + /** * * @param topic topic associated with the data diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 527dd0f..8a305b0 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -17,6 +17,9 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -224,6 +227,18 @@ public class Utils { } /** + * Sleep for a bit + * @param ms The duration of the sleep + */ + public static void sleep(long ms) { + try { + Thread.sleep(ms); + } catch (InterruptedException e) { + // this is okay, we just wake up early + } + } + + /** * Instantiate the class */ public static Object newInstance(Class c) { @@ -313,4 +328,31 @@ public class Utils { ? "[" + host + "]:" + port // IPv6 : host + ":" + port; } + + /** + * Create a string representation of an array joined by the given separator + * @param strs The array of items + * @param seperator The separator + * @return The string representation. + */ + public static String join(T[] strs, String seperator) { + return join(Arrays.asList(strs), seperator); + } + + /** + * Create a string representation of a list joined by the given separator + * @param list The list of items + * @param seperator The separator + * @return The string representation. + */ + public static String join(Collection list, String seperator) { + StringBuilder sb = new StringBuilder(); + Iterator iter = list.iterator(); + while(iter.hasNext()) { + sb.append(iter.next()); + if(iter.hasNext()) + sb.append(seperator); + } + return sb.toString(); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 47b5d4a..67bee40 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -59,13 +59,33 @@ public class MockClient implements KafkaClient { } @Override - public List poll(List requests, long timeoutMs, long now) { - this.requests.addAll(requests); - List copy = new ArrayList(this.responses); + public void send(ClientRequest request) { + this.requests.add(request); + } + + @Override + public List poll(long timeoutMs, long now) { + for(ClientResponse response: this.responses) + if (response.request().hasCallback()) + response.request().callback().onComplete(response); + List copy = new ArrayList(); this.responses.clear(); return copy; } + @Override + public List completeAll(int node, long now) { + return completeAll(now); + } + + @Override + public List completeAll(long now) { + List responses = poll(0, now); + if (requests.size() > 0) + throw new IllegalStateException("Requests without responses remain."); + return responses; + } + public Queue requests() { return this.requests; } @@ -81,6 +101,11 @@ public class MockClient implements KafkaClient { } @Override + public int inFlightRequestCount(int nodeId) { + return requests.size(); + } + + @Override public RequestHeader nextRequestHeader(ApiKeys key) { return new RequestHeader(key.id, "mock", correlation++); } diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 1a55242..5debcd6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -5,7 +5,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -46,14 +45,13 @@ public class NetworkClientTest { @Test public void testReadyAndDisconnect() { - List reqs = new ArrayList(); assertFalse("Client begins unready as it has no connection.", client.ready(node, time.milliseconds())); assertEquals("The connection is established as a side-effect of the readiness check", 1, selector.connected().size()); - client.poll(reqs, 1, time.milliseconds()); + client.poll(1, time.milliseconds()); selector.clear(); assertTrue("Now the client is ready", client.ready(node, time.milliseconds())); selector.disconnect(node.id()); - client.poll(reqs, 1, time.milliseconds()); + client.poll(1, time.milliseconds()); selector.clear(); assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds())); assertTrue("Metadata should get updated.", metadata.timeToNextUpdate(time.milliseconds()) == 0); @@ -65,7 +63,8 @@ public class NetworkClientTest { client.nextRequestHeader(ApiKeys.METADATA), new MetadataRequest(Arrays.asList("test")).toStruct()); ClientRequest request = new ClientRequest(time.milliseconds(), false, send, null); - client.poll(Arrays.asList(request), 1, time.milliseconds()); + client.send(request); + client.poll(1, time.milliseconds()); } @Test @@ -73,9 +72,11 @@ public class NetworkClientTest { ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.emptyMap()); RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE); RequestSend send = new RequestSend(node.id(), reqHeader, produceRequest.toStruct()); - ClientRequest request = new ClientRequest(time.milliseconds(), true, send, null); + TestCallbackHandler handler = new TestCallbackHandler(); + ClientRequest request = new ClientRequest(time.milliseconds(), true, send, handler); awaitReady(client, node); - client.poll(Arrays.asList(request), 1, time.milliseconds()); + client.send(request); + client.poll(1, time.milliseconds()); assertEquals(1, client.inFlightRequestCount()); ResponseHeader respHeader = new ResponseHeader(reqHeader.correlationId()); Struct resp = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); @@ -86,16 +87,26 @@ public class NetworkClientTest { resp.writeTo(buffer); buffer.flip(); selector.completeReceive(new NetworkReceive(node.id(), buffer)); - List responses = client.poll(new ArrayList(), 1, time.milliseconds()); + List responses = client.poll(1, time.milliseconds()); assertEquals(1, responses.size()); - ClientResponse response = responses.get(0); - assertTrue("Should have a response body.", response.hasResponse()); - assertEquals("Should be correlated to the original request", request, response.request()); + assertTrue("The handler should have executed.", handler.executed); + assertTrue("Should have a response body.", handler.response.hasResponse()); + assertEquals("Should be correlated to the original request", request, handler.response.request()); } private void awaitReady(NetworkClient client, Node node) { while (!client.ready(node, time.milliseconds())) - client.poll(new ArrayList(), 1, time.milliseconds()); + client.poll(1, time.milliseconds()); + } + + private static class TestCallbackHandler implements RequestCompletionHandler { + public boolean executed = false; + public ClientResponse response; + + public void onComplete(ClientResponse response) { + this.executed = true; + this.response = response; + } } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java deleted file mode 100644 index 29ad25e..0000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java +++ /dev/null @@ -1,297 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. -*/ -package org.apache.kafka.clients.consumer; - -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; - -import org.apache.kafka.common.TopicPartition; -import org.junit.Test; - -/** - * TODO: Clean this after the consumer implementation is complete. Until then, it is useful to write some sample test code using the new APIs - * - */ -public class ConsumerExampleTest { - /** - * This example demonstrates how to use the consumer to leverage Kafka's group management functionality for automatic consumer load - * balancing and failure detection. This example assumes that the offsets are stored in Kafka and are automatically committed periodically, - * as controlled by the auto.commit.interval.ms config - */ -// @Test -// public void testConsumerGroupManagementWithAutoOffsetCommits() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "true"); -// props.put("auto.commit.interval.ms", "10000"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some topics -// consumer.subscribe("foo", "bar"); -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// process(records); -// } -// consumer.close(); -// } - - /** - * This example demonstrates how to use the consumer to leverage Kafka's group management functionality for automatic consumer load - * balancing and failure detection. This example assumes that the offsets are stored in Kafka and are manually committed using the - * commit() API. This example also demonstrates rewinding the consumer's offsets if processing of consumed messages fails. - */ -// @Test -// public void testConsumerGroupManagementWithManualOffsetCommit() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "false"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some topics -// consumer.subscribe("foo", "bar"); -// int commitInterval = 100; -// int numRecords = 0; -// boolean isRunning = true; -// Map consumedOffsets = new HashMap(); -// while(isRunning) { -// Map records = consumer.poll(100); -// try { -// Map lastConsumedOffsets = process(records); -// consumedOffsets.putAll(lastConsumedOffsets); -// numRecords += records.size(); -// // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance -// if(numRecords % commitInterval == 0) -// consumer.commit(true); -// } catch(Exception e) { -// // rewind consumer's offsets for failed partitions -// List failedPartitions = getFailedPartitions(); -// Map offsetsToRewindTo = new HashMap(); -// for(TopicPartition failedPartition : failedPartitions) { -// // rewind to the last consumed offset for the failed partition. Since process() failed for this partition, the consumed offset -// // should still be pointing to the last successfully processed offset and hence is the right offset to rewind consumption to. -// offsetsToRewindTo.put(failedPartition, consumedOffsets.get(failedPartition)); -// } -// // seek to new offsets only for partitions that failed the last process() -// consumer.seek(offsetsToRewindTo); -// } -// } -// consumer.close(); -// } - - private List getFailedPartitions() { return null; } - - /** - * This example demonstrates the consumer can be used to leverage Kafka's group management functionality along with custom offset storage. - * In this example, the assumption made is that the user chooses to store the consumer offsets outside Kafka. This requires the user to - * plugin logic for retrieving the offsets from a custom store and provide the offsets to the consumer in the ConsumerRebalanceCallback - * callback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance and - * before the consumption restarts post rebalance. This is the right place to supply offsets from a custom store to the consumer. - */ -// @Test -// public void testConsumerRebalanceWithCustomOffsetStore() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "true"); -// props.put("auto.commit.interval.ms", "10000"); -// KafkaConsumer consumer = new KafkaConsumer(props, -// new ConsumerRebalanceCallback() { -// public void onPartitionsAssigned(Consumer consumer, Collection partitions) { -// Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(partitions); -// consumer.seek(lastCommittedOffsets); -// } -// public void onPartitionsRevoked(Consumer consumer, Collection partitions) { -// Map offsets = getLastConsumedOffsets(partitions); // implemented by the user -// commitOffsetsToCustomStore(offsets); // implemented by the user -// } -// private Map getLastCommittedOffsetsFromCustomStore(Collection partitions) { -// return null; -// } -// private Map getLastConsumedOffsets(Collection partitions) { return null; } -// private void commitOffsetsToCustomStore(Map offsets) {} -// }); -// // subscribe to topics -// consumer.subscribe("foo", "bar"); -// int commitInterval = 100; -// int numRecords = 0; -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// numRecords += records.size(); -// // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance -// if(numRecords % commitInterval == 0) -// commitOffsetsToCustomStore(consumedOffsets); -// } -// consumer.close(); -// } - - /** - * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality along with Kafka based offset storage. - * In this example, the assumption made is that the user chooses to use Kafka based offset management. - */ -// @Test -// public void testConsumerRewindWithGroupManagementAndKafkaOffsetStorage() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "false"); -// KafkaConsumer consumer = new KafkaConsumer(props, -// new ConsumerRebalanceCallback() { -// boolean rewindOffsets = true; -// public void onPartitionsAssigned(Consumer consumer, Collection partitions) { -// if(rewindOffsets) { -// Map latestCommittedOffsets = consumer.committed(null); -// Map newOffsets = rewindOffsets(latestCommittedOffsets, 100); -// consumer.seek(newOffsets); -// } -// } -// public void onPartitionsRevoked(Consumer consumer, Collection partitions) { -// consumer.commit(true); -// } -// // this API rewinds every partition back by numberOfMessagesToRewindBackTo messages -// private Map rewindOffsets(Map currentOffsets, -// long numberOfMessagesToRewindBackTo) { -// Map newOffsets = new HashMap(); -// for(Map.Entry offset : currentOffsets.entrySet()) { -// newOffsets.put(offset.getKey(), offset.getValue() - numberOfMessagesToRewindBackTo); -// } -// return newOffsets; -// } -// }); -// // subscribe to topics -// consumer.subscribe("foo", "bar"); -// int commitInterval = 100; -// int numRecords = 0; -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// numRecords += records.size(); -// // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance -// if(numRecords % commitInterval == 0) -// commitOffsetsToCustomStore(consumedOffsets); -// } -// consumer.close(); -// } - - /** - * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use Kafka based offset storage. The user still has to specify a group.id to use Kafka - * based offset management. However, session.timeout.ms is not required since the Kafka consumer only does failure detection with group - * management. - */ -// @Test -// public void testConsumerWithKafkaBasedOffsetManagement() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("auto.commit.enable", "true"); -// props.put("auto.commit.interval.ms", "10000"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some partitions of topic foo -// TopicPartition partition0 = new TopicPartition("foo", 0); -// TopicPartition partition1 = new TopicPartition("foo", 1); -// TopicPartition[] partitions = new TopicPartition[2]; -// partitions[0] = partition0; -// partitions[1] = partition1; -// consumer.subscribe(partitions); -// // find the last committed offsets for partitions 0,1 of topic foo -// Map lastCommittedOffsets = consumer.committed(null); -// // seek to the last committed offsets to avoid duplicates -// consumer.seek(lastCommittedOffsets); -// // find the offsets of the latest available messages to know where to stop consumption -// Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, null); -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// for(TopicPartition partition : partitions) { -// if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition)) -// isRunning = false; -// else -// isRunning = true; -// } -// } -// consumer.close(); -// } - - /** - * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use custom offset storage. - */ - @Test - public void testConsumerWithCustomOffsetManagement() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some partitions of topic foo -// TopicPartition partition0 = new TopicPartition("foo", 0); -// TopicPartition partition1 = new TopicPartition("foo", 1); -// TopicPartition[] partitions = new TopicPartition[2]; -// partitions[0] = partition0; -// partitions[1] = partition1; -// consumer.subscribe(partitions); -// Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(); -// // seek to the last committed offsets to avoid duplicates -// consumer.seek(lastCommittedOffsets); -// // find the offsets of the latest available messages to know where to stop consumption -// Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, null); -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// // commit offsets for partitions 0,1 for topic foo to custom store -// commitOffsetsToCustomStore(consumedOffsets); -// for(TopicPartition partition : partitions) { -// if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition)) -// isRunning = false; -// else -// isRunning = true; -// } -// } -// consumer.close(); - } - - private Map getLastCommittedOffsetsFromCustomStore() { return null; } - private void commitOffsetsToCustomStore(Map consumedOffsets) {} - private Map process(Map records) { - Map processedOffsets = new HashMap(); - for(Entry recordMetadata : records.entrySet()) { - List recordsPerTopic = recordMetadata.getValue().records(); - for(int i = 0;i < recordsPerTopic.size();i++) { - ConsumerRecord record = recordsPerTopic.get(i); - // process record - try { - processedOffsets.put(record.topicAndPartition(), record.offset()); - } catch (Exception e) { - e.printStackTrace(); - } - } - } - return processedOffsets; - } -} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java new file mode 100644 index 0000000..e51d2df --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -0,0 +1,32 @@ +package org.apache.kafka.clients.consumer; + +import static org.junit.Assert.*; + +import java.util.Iterator; + +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +public class MockConsumerTest { + + private MockConsumer consumer = new MockConsumer(); + + @Test + public void testSimpleMock() { + consumer.subscribe("topic"); + assertEquals(0, consumer.poll(1000).count()); + ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, "key1", "value1"); + ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, "key2", "value2"); + consumer.addRecord(rec1); + consumer.addRecord(rec2); + ConsumerRecords recs = consumer.poll(1); + Iterator> iter = recs.iterator(); + assertEquals(rec1, iter.next()); + assertEquals(rec2, iter.next()); + assertFalse(iter.hasNext()); + assertEquals(1L, consumer.position(new TopicPartition("test", 0))); + consumer.commit(CommitType.SYNC); + assertEquals(1L, consumer.committed(new TopicPartition("test", 0))); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java new file mode 100644 index 0000000..864f1c7 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -0,0 +1,61 @@ +package org.apache.kafka.clients.consumer.internals; + +import static org.junit.Assert.*; +import static java.util.Arrays.asList; + +import java.util.Collections; + +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +public class SubscriptionStateTest { + + private final SubscriptionState state = new SubscriptionState(); + private final TopicPartition tp0 = new TopicPartition("test", 0); + private final TopicPartition tp1 = new TopicPartition("test", 1); + + @Test + public void partitionSubscription() { + state.subscribe(tp0); + assertEquals(Collections.singleton(tp0), state.assignedPartitions()); + state.committed(tp0, 1); + state.fetched(tp0, 1); + state.consumed(tp0, 1); + assertAllPositions(tp0, 1L); + state.unsubscribe(tp0); + assertTrue(state.assignedPartitions().isEmpty()); + assertAllPositions(tp0, null); + } + + public void topicSubscription() { + state.subscribe("test"); + assertEquals(1, state.subscribedTopics().size()); + assertTrue(state.assignedPartitions().isEmpty()); + assertTrue(state.partitionsAutoAssigned()); + state.changePartitionAssignment(asList(tp0)); + state.committed(tp0, 1); + state.fetched(tp0, 1); + state.consumed(tp0, 1); + assertAllPositions(tp0, 1L); + state.changePartitionAssignment(asList(tp1)); + assertAllPositions(tp0, null); + assertEquals(Collections.singleton(tp1), state.assignedPartitions()); + } + + @Test(expected = IllegalArgumentException.class) + public void cantChangeFetchPositionForNonAssignedPartition() { + state.fetched(tp0, 1); + } + + @Test(expected = IllegalArgumentException.class) + public void cantChangeConsumedPositionForNonAssignedPartition() { + state.consumed(tp0, 1); + } + + public void assertAllPositions(TopicPartition tp, Long offset) { + assertEquals(offset, state.committed(tp)); + assertEquals(offset, state.fetched(tp)); + assertEquals(offset, state.consumed(tp)); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java index 1236803..77b23e7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java @@ -43,7 +43,7 @@ public class BufferPoolTest { */ @Test public void testSimple() throws Exception { - int totalMemory = 64 * 1024; + long totalMemory = 64 * 1024; int size = 1024; BufferPool pool = new BufferPool(totalMemory, size, false, metrics, time, metricGroup, metricTags); ByteBuffer buffer = pool.allocate(size); @@ -100,7 +100,7 @@ public class BufferPoolTest { ByteBuffer buffer = pool.allocate(1024); CountDownLatch doDealloc = asyncDeallocate(pool, buffer); CountDownLatch allocation = asyncAllocate(pool, 5 * 1024); - assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1, allocation.getCount()); + assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1L, allocation.getCount()); doDealloc.countDown(); // return the memory allocation.await(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 3676b05..d3377ef 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -38,7 +38,7 @@ public class MockProducerTest { Future metadata = producer.send(record); assertTrue("Send should be immediately complete", metadata.isDone()); assertFalse("Send should be successful", isError(metadata)); - assertEquals("Offset should be 0", 0, metadata.get().offset()); + assertEquals("Offset should be 0", 0L, metadata.get().offset()); assertEquals(topic, metadata.get().topic()); assertEquals("We should have the record in our history", asList(record), producer.history()); producer.clear(); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java index 1d077fd..82d8083 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java @@ -1,30 +1,23 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.clients.producer; import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import java.util.List; - -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.internals.Partitioner; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; @@ -34,7 +27,6 @@ import org.junit.Test; public class PartitionerTest { private byte[] key = "key".getBytes(); - private byte[] value = "value".getBytes(); private Partitioner partitioner = new Partitioner(); private Node node0 = new Node(0, "localhost", 99); private Node node1 = new Node(1, "localhost", 100); @@ -48,33 +40,28 @@ public class PartitionerTest { @Test public void testUserSuppliedPartitioning() { - assertEquals("If the user supplies a partition we should use it.", - 0, - partitioner.partition(new ProducerRecord("test", 0, key, value), cluster)); + assertEquals("If the user supplies a partition we should use it.", 0, partitioner.partition("test", key, 0, cluster)); } @Test public void testKeyPartitionIsStable() { - int partition = partitioner.partition(new ProducerRecord("test", key, value), cluster); - assertEquals("Same key should yield same partition", - partition, - partitioner.partition(new ProducerRecord("test", key, "value2".getBytes()), cluster)); + int partition = partitioner.partition("test", key, null, cluster); + assertEquals("Same key should yield same partition", partition, partitioner.partition("test", key, null, cluster)); } @Test public void testRoundRobinIsStable() { - int startPart = partitioner.partition(new ProducerRecord("test", value), cluster); + int startPart = partitioner.partition("test", null, null, cluster); for (int i = 1; i <= 100; i++) { - int partition = partitioner.partition(new ProducerRecord("test", value), cluster); - assertEquals("Should yield a different partition each call with round-robin partitioner", - partition, (startPart + i) % 2); - } + int partition = partitioner.partition("test", null, null, cluster); + assertEquals("Should yield a different partition each call with round-robin partitioner", partition, (startPart + i) % 2); + } } @Test public void testRoundRobinWithDownNode() { for (int i = 0; i < partitions.size(); i++) { - int part = partitioner.partition(new ProducerRecord("test", value), cluster); + int part = partitioner.partition("test", null, null, cluster); assertTrue("We should never choose a leader-less node in round robin", part >= 0 && part < 2); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java index 66cbdf5..888b929 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java @@ -72,14 +72,14 @@ public class SenderTest { @Test public void testSimple() throws Exception { - int offset = 0; + long offset = 0; Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null).future; sender.run(time.milliseconds()); // connect sender.run(time.milliseconds()); // send produce request assertEquals("We should have a single produce request in flight.", 1, client.inFlightRequestCount()); client.respond(produceResponse(tp.topic(), tp.partition(), offset, Errors.NONE.code())); sender.run(time.milliseconds()); - assertEquals("All requests completed.", offset, client.inFlightRequestCount()); + assertEquals("All requests completed.", offset, (long) client.inFlightRequestCount()); sender.run(time.milliseconds()); assertTrue("Request should be completed", future.isDone()); assertEquals(offset, future.get().offset()); @@ -110,7 +110,7 @@ public class SenderTest { sender.run(time.milliseconds()); // reconnect sender.run(time.milliseconds()); // resend assertEquals(1, client.inFlightRequestCount()); - int offset = 0; + long offset = 0; client.respond(produceResponse(tp.topic(), tp.partition(), offset, Errors.NONE.code())); sender.run(time.milliseconds()); assertTrue("Request should have retried and completed", future.isDone()); diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 3c442a2..16d3fed 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -110,18 +110,18 @@ public class ConfigDefTest { @Test(expected = ConfigException.class) public void testInvalidDefaultRange() { - ConfigDef def = new ConfigDef().define("name", Type.INT, -1, Range.between(0,10), Importance.HIGH, "docs"); + new ConfigDef().define("name", Type.INT, -1, Range.between(0,10), Importance.HIGH, "docs"); } @Test(expected = ConfigException.class) public void testInvalidDefaultString() { - ConfigDef def = new ConfigDef().define("name", Type.STRING, "bad", ValidString.in(Arrays.asList("valid", "values")), Importance.HIGH, "docs"); + new ConfigDef().define("name", Type.STRING, "bad", ValidString.in("valid", "values"), Importance.HIGH, "docs"); } @Test public void testValidators() { testValidators(Type.INT, Range.between(0,10), 5, new Object[]{1, 5, 9}, new Object[]{-1, 11}); - testValidators(Type.STRING, ValidString.in(Arrays.asList("good", "values", "default")), "default", + testValidators(Type.STRING, ValidString.in("good", "values", "default"), "default", new Object[]{"good", "values", "default"}, new Object[]{"bad", "inputs"}); } diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index 74c1957..a14659a 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -12,7 +12,6 @@ */ package org.apache.kafka.common.network; -import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -74,7 +73,7 @@ public class SelectorTest { // disconnect this.server.closeConnections(); while (!selector.disconnected().contains(node)) - selector.poll(1000L, EMPTY); + selector.poll(1000L); // reconnect and do another request blockingConnect(node); @@ -89,7 +88,8 @@ public class SelectorTest { int node = 0; blockingConnect(node); selector.disconnect(node); - selector.poll(10, asList(createSend(node, "hello1"))); + selector.send(createSend(node, "hello1")); + selector.poll(10); assertEquals("Request should not have succeeded", 0, selector.completedSends().size()); assertEquals("There should be a disconnect", 1, selector.disconnected().size()); assertTrue("The disconnect should be from our node", selector.disconnected().contains(node)); @@ -104,7 +104,9 @@ public class SelectorTest { public void testCantSendWithInProgress() throws Exception { int node = 0; blockingConnect(node); - selector.poll(1000L, asList(createSend(node, "test1"), createSend(node, "test2"))); + selector.send(createSend(node, "test1")); + selector.send(createSend(node, "test2")); + selector.poll(1000L); } /** @@ -112,7 +114,8 @@ public class SelectorTest { */ @Test(expected = IllegalStateException.class) public void testCantSendWithoutConnecting() throws Exception { - selector.poll(1000L, asList(createSend(0, "test"))); + selector.send(createSend(0, "test")); + selector.poll(1000L); } /** @@ -131,7 +134,7 @@ public class SelectorTest { int node = 0; selector.connect(node, new InetSocketAddress("localhost", TestUtils.choosePort()), BUFFER_SIZE, BUFFER_SIZE); while (selector.disconnected().contains(node)) - selector.poll(1000L, EMPTY); + selector.poll(1000L); } /** @@ -152,14 +155,13 @@ public class SelectorTest { int[] requests = new int[conns]; int[] responses = new int[conns]; int responseCount = 0; - List sends = new ArrayList(); for (int i = 0; i < conns; i++) - sends.add(createSend(i, i + "-" + 0)); + selector.send(createSend(i, i + "-" + 0)); // loop until we complete all requests while (responseCount < conns * reqs) { // do the i/o - selector.poll(0L, sends); + selector.poll(0L); assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size()); @@ -175,12 +177,11 @@ public class SelectorTest { } // prepare new sends for the next round - sends.clear(); for (NetworkSend send : selector.completedSends()) { int dest = send.destination(); requests[dest]++; if (requests[dest] < reqs) - sends.add(createSend(dest, dest + "-" + requests[dest])); + selector.send(createSend(dest, dest + "-" + requests[dest])); } } } @@ -212,10 +213,34 @@ public class SelectorTest { blockingConnect(0); } + @Test + public void testMute() throws Exception { + blockingConnect(0); + blockingConnect(1); + + selector.send(createSend(0, "hello")); + selector.send(createSend(1, "hi")); + + selector.mute(1); + + while (selector.completedReceives().isEmpty()) + selector.poll(5); + assertEquals("We should have only one response", 1, selector.completedReceives().size()); + assertEquals("The response should not be from the muted node", 0, selector.completedReceives().get(0).source()); + + selector.unmute(1); + do { + selector.poll(5); + } while (selector.completedReceives().isEmpty()); + assertEquals("We should have only one response", 1, selector.completedReceives().size()); + assertEquals("The response should be from the previously muted node", 1, selector.completedReceives().get(0).source()); + } + private String blockingRequest(int node, String s) throws IOException { - selector.poll(1000L, asList(createSend(node, s))); + selector.send(createSend(node, s)); + selector.poll(1000L); while (true) { - selector.poll(1000L, EMPTY); + selector.poll(1000L); for (NetworkReceive receive : selector.completedReceives()) if (receive.source() == node) return asString(receive); @@ -226,7 +251,7 @@ public class SelectorTest { private void blockingConnect(int node) throws IOException { selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE); while (!selector.connected().contains(node)) - selector.poll(10000L, EMPTY); + selector.poll(10000L); } private NetworkSend createSend(int node, String s) { diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index a39fab5..4c2ea34 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.common.utils; +import java.util.Arrays; +import java.util.Collections; + import org.junit.Test; import static org.apache.kafka.common.utils.Utils.getHost; @@ -48,4 +51,11 @@ public class UtilsTest { assertEquals("[::1]:1234", formatAddress("::1", 1234)); assertEquals("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678", formatAddress("2001:db8:85a3:8d3:1319:8a2e:370:7348", 5678)); } + + @Test + public void testJoin() { + assertEquals("", Utils.join(Collections.emptyList(), ",")); + assertEquals("1", Utils.join(Arrays.asList("1"), ",")); + assertEquals("1,2,3", Utils.join(Arrays.asList(1, 2, 3), ",")); + } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/test/MockSelector.java b/clients/src/test/java/org/apache/kafka/test/MockSelector.java index d61de52..ea89b06 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockSelector.java +++ b/clients/src/test/java/org/apache/kafka/test/MockSelector.java @@ -1,18 +1,14 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.test; @@ -26,13 +22,13 @@ import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.utils.Time; - /** * A fake selector to use for testing */ public class MockSelector implements Selectable { private final Time time; + private final List initiatedSends = new ArrayList(); private final List completedSends = new ArrayList(); private final List completedReceives = new ArrayList(); private final List disconnected = new ArrayList(); @@ -68,8 +64,14 @@ public class MockSelector implements Selectable { } @Override - public void poll(long timeout, List sends) throws IOException { - this.completedSends.addAll(sends); + public void send(NetworkSend send) { + this.initiatedSends.add(send); + } + + @Override + public void poll(long timeout) throws IOException { + this.completedSends.addAll(this.initiatedSends); + this.initiatedSends.clear(); time.sleep(timeout); } @@ -101,4 +103,20 @@ public class MockSelector implements Selectable { return connected; } + @Override + public void mute(int id) { + } + + @Override + public void unmute(int id) { + } + + @Override + public void muteAll() { + } + + @Override + public void unmuteAll() { + } + } diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index 6d00ed0..a3b1b78 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -63,7 +63,7 @@ case class ConsumerMetadataRequest(group: String, override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { // return ConsumerCoordinatorNotAvailable for all uncaught errors - val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode) + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, correlationId) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) } diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala index 84f6017..24aaf95 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -40,7 +40,7 @@ object ConsumerMetadataResponse { } -case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int = 0) +case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int) extends RequestOrResponse() { def sizeInBytes = diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index b230e9a..e6ad8be 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -90,7 +90,7 @@ class Partition(val topic: String, val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) val offsetMap = checkpoint.read if (!offsetMap.contains(TopicAndPartition(topic, partitionId))) - warn("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId)) + info("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId)) val offset = offsetMap.getOrElse(TopicAndPartition(topic, partitionId), 0L).min(log.logEndOffset) val localReplica = new Replica(replicaId, this, time, offset, Some(log)) addReplicaIfNotExists(localReplica) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index fbef34c..14b22ab 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -141,7 +141,7 @@ class RequestSendThread(val controllerId: Int, connectToBroker(toBroker, channel) isSendSuccessful = false // backoff before retrying the connection and send - Utils.swallow(Thread.sleep(300)) + Utils.swallowTrace(Thread.sleep(300)) } } if (receive != null) { diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 4631bc7..8b67aee 100644 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -179,12 +179,12 @@ object LogConfig { .define(FileDeleteDelayMsProp, LONG, Defaults.FileDeleteDelayMs, atLeast(0), MEDIUM, FileDeleteDelayMsDoc) .define(MinCleanableDirtyRatioProp, DOUBLE, Defaults.MinCleanableDirtyRatio, between(0, 1), MEDIUM, MinCleanableRatioDoc) - .define(CleanupPolicyProp, STRING, if (Defaults.Compact) Compact else Delete, in(asList(Compact, Delete)), MEDIUM, + .define(CleanupPolicyProp, STRING, if (Defaults.Compact) Compact else Delete, in(Compact, Delete), MEDIUM, CompactDoc) .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, MEDIUM, UncleanLeaderElectionEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc) - .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(seqAsJavaList(BrokerCompressionCodec.brokerCompressionOptions)), MEDIUM, CompressionTypeDoc) + .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc) } def configNames() = { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index ec8d9f7..48bc435 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,6 +17,12 @@ package kafka.server +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.JoinGroupResponse +import org.apache.kafka.common.requests.HeartbeatResponse +import org.apache.kafka.common.requests.ResponseHeader +import org.apache.kafka.common.protocol.types.Struct + import kafka.api._ import kafka.common._ import kafka.log._ @@ -26,6 +32,9 @@ import kafka.network.RequestChannel.Response import kafka.controller.KafkaController import kafka.utils.{SystemTime, Logging} +import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic._ import scala.collection._ import org.I0Itec.zkclient.ZkClient @@ -43,6 +52,7 @@ class KafkaApis(val requestChannel: RequestChannel, this.logIdent = "[KafkaApi-%d] ".format(brokerId) val metadataCache = new MetadataCache + private var consumerGroupGenerationId = 0 /** * Top-level method that handles all requests and multiplexes to the right api @@ -62,6 +72,8 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request) case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request) case RequestKeys.ConsumerMetadataKey => handleConsumerMetadataRequest(request) + case RequestKeys.JoinGroupKey => handleJoinGroupRequest(request) + case RequestKeys.HeartbeatKey => handleHeartbeatRequest(request) case requestId => throw new KafkaException("Unknown api code " + requestId) } } catch { @@ -442,6 +454,23 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } + def handleJoinGroupRequest(request: RequestChannel.Request) { + val joinGroupReq = request.requestObj.asInstanceOf[JoinGroupRequestAndHeader] + val topics = JavaConversions.asScalaIterable(joinGroupReq.body.topics()).toSet + val partitions = this.replicaManager.logManager.allLogs.filter(log => topics.contains(log.topicAndPartition.topic)) + val partitionList = partitions.map(_.topicAndPartition).map(tp => new org.apache.kafka.common.TopicPartition(tp.topic, tp.partition)).toBuffer + this.consumerGroupGenerationId += 1 + val response = new JoinGroupResponse(ErrorMapping.NoError, this.consumerGroupGenerationId, joinGroupReq.body.consumerId, JavaConversions.asJavaList(partitionList)) + val send = new BoundedByteBufferSend(new JoinGroupResponseAndHeader(joinGroupReq.correlationId, response)) + requestChannel.sendResponse(new RequestChannel.Response(request, send)) + } + + def handleHeartbeatRequest(request: RequestChannel.Request) { + val hbReq = request.requestObj.asInstanceOf[HeartbeatRequestAndHeader] + val send = new BoundedByteBufferSend(new HeartbeatResponseAndHeader(hbReq.correlationId, new HeartbeatResponse(Errors.NONE.code))) + requestChannel.sendResponse(new RequestChannel.Response(request, send)) + } + def close() { // TODO currently closing the API is an no-op since the API no longer maintain any modules // maybe removing the closing call in the end when KafkaAPI becomes a pure stateless layer diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index e58fbb9..fb948b9 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -327,7 +327,7 @@ class ReplicaManager(val config: KafkaConfig, BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" - .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) + .format(messages.sizeInBytes, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) (topicAndPartition, LogAppendResult(info)) } catch { // NOTE: Failed produce requests metric is not incremented for known exceptions diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index e455cb9..910691e 100644 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -113,8 +113,6 @@ object ConsoleConsumer extends Logging { KafkaMetricsReporter.startReporters(verifiableProps) } - - val consumerProps = if (options.has(consumerConfigOpt)) Utils.loadProps(options.valueOf(consumerConfigOpt)) else diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala index 093c800..c39c067 100644 --- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala @@ -17,14 +17,21 @@ package kafka.tools -import java.util.concurrent.CountDownLatch +import scala.collection.JavaConversions._ import java.util.concurrent.atomic.AtomicLong import java.nio.channels.ClosedByInterruptException import org.apache.log4j.Logger +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.common.record.Record +import org.apache.kafka.common.record.Records +import org.apache.kafka.common.serialization.ByteArrayDeserializer import kafka.message.Message import kafka.utils.{ZkUtils, CommandLineUtils} import java.util.{ Random, Properties } -import kafka.consumer._ +import kafka.consumer.Consumer +import kafka.consumer.ConsumerConnector +import kafka.consumer.KafkaStream +import kafka.consumer.ConsumerTimeoutException import java.text.SimpleDateFormat /** @@ -42,50 +49,98 @@ object ConsumerPerformance { if (!config.hideHeader) { if (!config.showDetailedStats) - println("start.time, end.time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") + println("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") else - println("time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") + println("time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") } - // clean up zookeeper state for this group id for every perf run - ZkUtils.maybeDeletePath(config.consumerConfig.zkConnect, "/consumers/" + config.consumerConfig.groupId) - - val consumerConnector: ConsumerConnector = Consumer.create(config.consumerConfig) - - val topicMessageStreams = consumerConnector.createMessageStreams(Map(config.topic -> config.numThreads)) - var threadList = List[ConsumerPerfThread]() - for ((topic, streamList) <- topicMessageStreams) - for (i <- 0 until streamList.length) - threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, - totalMessagesRead, totalBytesRead) - - logger.info("Sleeping for 1 second.") - Thread.sleep(1000) - logger.info("starting threads") - val startMs = System.currentTimeMillis - for (thread <- threadList) - thread.start - - for (thread <- threadList) - thread.join - - val endMs = System.currentTimeMillis - val elapsedSecs = (endMs - startMs - config.consumerConfig.consumerTimeoutMs) / 1000.0 + var startMs, endMs = 0L + if(config.useNewConsumer) { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](config.props) + consumer.subscribe(config.topic) + startMs = System.currentTimeMillis + consume(consumer, config.numMessages, 1000, config, totalMessagesRead, totalBytesRead) + endMs = System.currentTimeMillis + } else { + import kafka.consumer.ConsumerConfig + val consumerConfig = new ConsumerConfig(config.props) + val consumerConnector: ConsumerConnector = Consumer.create(consumerConfig) + val topicMessageStreams = consumerConnector.createMessageStreams(Map(config.topic -> config.numThreads)) + var threadList = List[ConsumerPerfThread]() + for ((topic, streamList) <- topicMessageStreams) + for (i <- 0 until streamList.length) + threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, totalMessagesRead, totalBytesRead) + + logger.info("Sleeping for 1 second.") + Thread.sleep(1000) + logger.info("starting threads") + startMs = System.currentTimeMillis + for (thread <- threadList) + thread.start + for (thread <- threadList) + thread.join + endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs + } + val elapsedSecs = (endMs - startMs) / 1000.0 if (!config.showDetailedStats) { val totalMBRead = (totalBytesRead.get * 1.0) / (1024 * 1024) - println(("%s, %s, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs), - config.consumerConfig.fetchMessageMaxBytes, totalMBRead, totalMBRead / elapsedSecs, totalMessagesRead.get, - totalMessagesRead.get / elapsedSecs)) + println(("%s, %s, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs), + totalMBRead, totalMBRead / elapsedSecs, totalMessagesRead.get, totalMessagesRead.get / elapsedSecs)) } System.exit(0) } + + def consume(consumer: KafkaConsumer[Array[Byte], Array[Byte]], count: Long, timeout: Long, config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) { + var bytesRead = 0L + var messagesRead = 0L + val startMs = System.currentTimeMillis + var lastReportTime: Long = startMs + var lastBytesRead = 0L + var lastMessagesRead = 0L + var lastConsumed = System.currentTimeMillis + while(messagesRead < count && lastConsumed >= System.currentTimeMillis - timeout) { + val records = consumer.poll(100) + if(records.count() > 0) + lastConsumed = System.currentTimeMillis + for(record <- records) { + messagesRead += 1 + if(record.key != null) + bytesRead += record.key.size + if(record.value != null) + bytesRead += record.value.size + + if (messagesRead % config.reportingInterval == 0) { + if (config.showDetailedStats) + printProgressMessage(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis, config.dateFormat) + lastReportTime = System.currentTimeMillis + lastMessagesRead = messagesRead + lastBytesRead = bytesRead + } + } + } + totalMessagesRead.set(messagesRead) + totalBytesRead.set(bytesRead) + } + + def printProgressMessage(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, lastMessagesRead: Long, + startMs: Long, endMs: Long, dateFormat: SimpleDateFormat) = { + val elapsedMs: Double = endMs - startMs + val totalMBRead = (bytesRead * 1.0) / (1024 * 1024) + val mbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024) + println(("%s, %d, %.4f, %.4f, %d, %.4f").format(dateFormat.format(endMs), id, totalMBRead, + 1000.0 * (mbRead / elapsedMs), messagesRead, ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0)) + } class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) { - val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + - "Multiple URLS can be given to allow fail-over.") + val zkConnectOpt = parser.accepts("zookeeper", "The connection string for the zookeeper connection in the form host:port. " + + "Multiple URLS can be given to allow fail-over. This option is only used with the old consumer.") .withRequiredArg .describedAs("urls") .ofType(classOf[String]) + val bootstrapServersOpt = parser.accepts("broker-list", "A broker list to use for connecting if using the new consumer.") + .withRequiredArg() + .describedAs("host") + .ofType(classOf[String]) val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") .withRequiredArg .describedAs("topic") @@ -117,20 +172,35 @@ object ConsumerPerformance { .describedAs("count") .ofType(classOf[java.lang.Integer]) .defaultsTo(1) + val useNewConsumerOpt = parser.accepts("new-consumer", "Use the new consumer implementation.") val options = parser.parse(args: _*) - CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, zkConnectOpt) - + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt) + + val useNewConsumer = options.has(useNewConsumerOpt) + val props = new Properties - props.put("group.id", options.valueOf(groupIdOpt)) - props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) - props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString) - props.put("auto.offset.reset", if (options.has(resetBeginningOffsetOpt)) "largest" else "smallest") - props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) - props.put("consumer.timeout.ms", "5000") - props.put("num.consumer.fetchers", options.valueOf(numFetchersOpt).toString) - val consumerConfig = new ConsumerConfig(props) + if(useNewConsumer) { + import org.apache.kafka.clients.consumer.ConsumerConfig + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf(bootstrapServersOpt)) + props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt)) + props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString) + props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString) + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, if (options.has(resetBeginningOffsetOpt)) "latest" else "earliest") + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer]) + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer]) + props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false") + } else { + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt) + props.put("group.id", options.valueOf(groupIdOpt)) + props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) + props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString) + props.put("auto.offset.reset", if (options.has(resetBeginningOffsetOpt)) "largest" else "smallest") + props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) + props.put("consumer.timeout.ms", "1000") + props.put("num.consumer.fetchers", options.valueOf(numFetchersOpt).toString) + } val numThreads = options.valueOf(numThreadsOpt).intValue val topic = options.valueOf(topicOpt) val numMessages = options.valueOf(numMessagesOpt).longValue @@ -161,7 +231,7 @@ object ConsumerPerformance { if (messagesRead % config.reportingInterval == 0) { if (config.showDetailedStats) - printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis) + printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis, config.dateFormat) lastReportTime = System.currentTimeMillis lastMessagesRead = messagesRead lastBytesRead = bytesRead @@ -176,18 +246,9 @@ object ConsumerPerformance { totalMessagesRead.addAndGet(messagesRead) totalBytesRead.addAndGet(bytesRead) if (config.showDetailedStats) - printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis) + printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis, config.dateFormat) } - private def printMessage(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, lastMessagesRead: Long, - startMs: Long, endMs: Long) = { - val elapsedMs = endMs - startMs - val totalMBRead = (bytesRead * 1.0) / (1024 * 1024) - val mbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024) - println(("%s, %d, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(endMs), id, - config.consumerConfig.fetchMessageMaxBytes, totalMBRead, - 1000.0 * (mbRead / elapsedMs), messagesRead, ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0)) - } } } diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala index 7602b8d..900f7df 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala @@ -79,7 +79,7 @@ object SimpleConsumerPerformance { done = true else // we only did one fetch so we find the offset for the first (head) messageset - offset += messageSet.validBytes + offset = messageSet.last.nextOffset totalBytesRead += bytesRead totalMessagesRead += messagesRead diff --git a/core/src/main/scala/kafka/utils/KafkaScheduler.scala b/core/src/main/scala/kafka/utils/KafkaScheduler.scala index 9a16343..7ceadcc 100644 --- a/core/src/main/scala/kafka/utils/KafkaScheduler.scala +++ b/core/src/main/scala/kafka/utils/KafkaScheduler.scala @@ -41,6 +41,11 @@ trait Scheduler { def shutdown() /** + * Check if the scheduler has been started + */ + def isStarted: Boolean + + /** * Schedule a task * @param name The name of this task * @param delay The amount of time to wait before the first execution @@ -63,13 +68,13 @@ trait Scheduler { class KafkaScheduler(val threads: Int, val threadNamePrefix: String = "kafka-scheduler-", daemon: Boolean = true) extends Scheduler with Logging { - @volatile private var executor: ScheduledThreadPoolExecutor = null + private var executor: ScheduledThreadPoolExecutor = null private val schedulerThreadId = new AtomicInteger(0) override def startup() { debug("Initializing task scheduler.") this synchronized { - if(executor != null) + if(isStarted) throw new IllegalStateException("This scheduler has already been started!") executor = new ScheduledThreadPoolExecutor(threads) executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false) @@ -83,34 +88,45 @@ class KafkaScheduler(val threads: Int, override def shutdown() { debug("Shutting down task scheduler.") - ensureStarted - executor.shutdown() - executor.awaitTermination(1, TimeUnit.DAYS) - this.executor = null + this synchronized { + if(isStarted) { + executor.shutdown() + executor.awaitTermination(1, TimeUnit.DAYS) + this.executor = null + } + } } def schedule(name: String, fun: ()=>Unit, delay: Long, period: Long, unit: TimeUnit) = { debug("Scheduling task %s with initial delay %d ms and period %d ms." .format(name, TimeUnit.MILLISECONDS.convert(delay, unit), TimeUnit.MILLISECONDS.convert(period, unit))) - ensureStarted - val runnable = Utils.runnable { - try { - trace("Begining execution of scheduled task '%s'.".format(name)) - fun() - } catch { - case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t) - } finally { - trace("Completed execution of scheduled task '%s'.".format(name)) + this synchronized { + ensureStarted + val runnable = Utils.runnable { + try { + trace("Begining execution of scheduled task '%s'.".format(name)) + fun() + } catch { + case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t) + } finally { + trace("Completed execution of scheduled task '%s'.".format(name)) + } } + if(period >= 0) + executor.scheduleAtFixedRate(runnable, delay, period, unit) + else + executor.schedule(runnable, delay, unit) + } + } + + def isStarted: Boolean = { + this synchronized { + executor != null } - if(period >= 0) - executor.scheduleAtFixedRate(runnable, delay, period, unit) - else - executor.schedule(runnable, delay, unit) } private def ensureStarted = { - if(executor == null) + if(!isStarted) throw new IllegalStateException("Kafka scheduler has not been started") } } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala new file mode 100644 index 0000000..798f035 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -0,0 +1,286 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package kafka.api + +import org.apache.kafka.common.KafkaException +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.consumer.Consumer +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.consumer.CommitType +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException +import scala.collection.mutable.Buffer +import scala.collection.JavaConversions._ +import java.util.ArrayList +import java.util.Arrays +import org.junit.Assert._ +import kafka.utils.TestUtils +import kafka.utils.Logging +import kafka.server.OffsetManager + +/** + * Integration tests for the new consumer that cover basic usage as well as server failures + */ +class ConsumerTest extends IntegrationTestHarness with Logging { + + val producerCount = 1 + val consumerCount = 2 + val serverCount = 3 + + val topic = "topic" + val part = 0 + val tp = new TopicPartition(topic, part) + + // configure the servers and clients + this.serverConfig.setProperty("controlled.shutdown.enable", "false") // speed up shutdown + this.serverConfig.setProperty("offsets.topic.replication.factor", "3") // don't want to lose offset + this.serverConfig.setProperty("offsets.topic.num.partitions", "1") + this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + + override def setUp() { + super.setUp() + // this will trigger the creation of the consumer offsets topic + this.consumers(0).partitionsFor(OffsetManager.OffsetsTopicName) + } + + def testSimpleConsumption() { + val numRecords = 10000 + sendRecords(numRecords) + + assertEquals(0, this.consumers(0).subscriptions.size) + this.consumers(0).subscribe(tp) + assertEquals(1, this.consumers(0).subscriptions.size) + + this.consumers(0).seek(tp, 0) + consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) + } + + def testAutoOffsetReset() { + sendRecords(1) + this.consumers(0).subscribe(tp) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + def testSeek() { + val consumer = this.consumers(0) + val totalRecords = 50L + sendRecords(totalRecords.toInt) + consumer.subscribe(tp) + + consumer.seekToEnd(tp) + assertEquals(totalRecords, consumer.position(tp)) + assertFalse(consumer.poll(totalRecords).iterator().hasNext()) + + consumer.seekToBeginning(tp) + assertEquals(0, consumer.position(tp), 0) + consumeRecords(consumer, numRecords = 1, startingOffset = 0) + + val mid = totalRecords / 2 + consumer.seek(tp, mid) + assertEquals(mid, consumer.position(tp)) + consumeRecords(consumer, numRecords = 1, startingOffset = mid.toInt) + } + + def testGroupConsumption() { + // we need to do this test with only one server since we have the hack join group + // that just assigns the partition hosted on the local machine (with two we might get the wrong machine + this.servers.last.shutdown() + this.servers.head.shutdown() + sendRecords(10) + this.consumers(0).subscribe(topic) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + def testPositionAndCommit() { + sendRecords(5) + + // committed() on a partition with no committed offset throws an exception + intercept[NoOffsetForPartitionException] { + this.consumers(0).committed(new TopicPartition(topic, 15)) + } + + // position() on a partition that we aren't subscribed to throws an exception + intercept[IllegalArgumentException] { + this.consumers(0).position(new TopicPartition(topic, 15)) + } + + this.consumers(0).subscribe(tp) + + assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, this.consumers(0).position(tp)) + this.consumers(0).commit(CommitType.SYNC) + assertEquals(0L, this.consumers(0).committed(tp)) + + consumeRecords(this.consumers(0), 5, 0) + assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp)) + this.consumers(0).commit(CommitType.SYNC) + assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp)); + + sendRecords(1) + + // another consumer in the same group should get the same position + this.consumers(1).subscribe(tp) + consumeRecords(this.consumers(1), 1, 5) + } + + def testPartitionsFor() { + val numParts = 2; + TestUtils.createTopic(this.zkClient, topic, numParts, 1, this.servers) + val parts = this.consumers(0).partitionsFor(topic) + assertNotNull(parts) + assertEquals(2, parts.length) + assertNull(this.consumers(0).partitionsFor("non-existant-topic")) + } + + def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(numRecords = 1000) + + /* + * 1. Produce a bunch of messages + * 2. Then consume the messages while killing and restarting brokers at random + */ + def consumeWithBrokerFailures(numRecords: Int) { + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + sendRecords(numRecords) + this.producers.map(_.close) + var consumed = 0 + val consumer = this.consumers(0) + consumer.subscribe(topic) + while (consumed < numRecords) { + // check that we are getting the messages in order + for (record <- consumer.poll(200)) { + assertEquals(consumed.toLong, record.offset()) + consumed += 1 + } + consumer.commit(CommitType.SYNC); + + /* restart any dead brokers, and kill a broker (with probability 1/3) */ + restartDeadBrokers() + if (TestUtils.random.nextInt(3) == 0) { + info("Killing broker") + killRandomBroker() + } + } + } + + def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(20) + + def seekAndCommitWithBrokerFailures(numIters: Int) { + // create a topic and send it some data + val numRecords = 1000 + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + sendRecords(numRecords) + this.producers.map(_.close) + + val consumer = this.consumers(0) + consumer.subscribe(tp) + consumer.seek(tp, 0) + for (iter <- 0 until numIters) { + val coin = TestUtils.random.nextInt(4) + if (coin == 0) { + info("Seeking to end of log") + consumer.seekToEnd() + assertEquals(1000.toLong, consumer.position(tp)) + } else if (coin == 1) { + val pos = TestUtils.random.nextInt(numRecords).toLong + info("Seeking to " + pos) + consumer.seek(tp, pos) + assertEquals(pos, consumer.position(tp)) + } else if (coin == 2) { + info("Committing offset.") + consumer.commit(CommitType.SYNC) + assertEquals(consumer.position(tp), consumer.committed(tp)) + } else { + restartDeadBrokers() + killRandomBroker() + } + } + } + + def testPartitionReassignmentCallback() { + val callback = new TestConsumerReassignmentCallback() + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test + val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumer0.subscribe("test") + + // the initial subscription should cause a callback execution + while(callback.callsToAssigned == 0) + consumer0.poll(50) + + // get metadata for the topic + var parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + while(parts == null) + parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + assertEquals(1, parts.size) + assertNotNull(parts(0).leader()) + + // shutdown the co-ordinator + val coordinator = parts(0).leader().id() + this.servers(coordinator).shutdown() + + // this should cause another callback execution + while(callback.callsToAssigned < 2) + consumer0.poll(50) + assertEquals(2, callback.callsToAssigned) + assertEquals(2, callback.callsToRevoked) + + consumer0.close() + } + + class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback { + var callsToAssigned = 0 + var callsToRevoked = 0 + def onPartitionsAssigned(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { + info("onPartitionsAssigned called.") + callsToAssigned += 1 + } + def onPartitionsRevoked(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { + info("onPartitionsRevoked called.") + callsToRevoked += 1 + } + } + + private def sendRecords(numRecords: Int) { + val futures = (0 until numRecords).map { i => + this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) + } + futures.map(_.get) + } + + private def consumeRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int) { + val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]() + val maxIters = numRecords * 300 + var iters = 0 + while (records.size < numRecords) { + for (record <- consumer.poll(50)) + records.add(record) + if(iters > maxIters) + throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations."); + iters += 1 + } + for (i <- 0 until numRecords) { + val record = records.get(i) + val offset = startingOffset + i + assertEquals(topic, record.topic()) + assertEquals(part, record.partition()) + assertEquals(offset.toLong, record.offset()) + } + } + +} \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala new file mode 100644 index 0000000..5650b4a --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.scalatest.junit.JUnit3Suite +import collection._ +import kafka.utils.TestUtils +import java.util.Properties +import java.util.Arrays +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer.KafkaProducer +import kafka.server.KafkaConfig +import kafka.integration.KafkaServerTestHarness +import scala.collection.mutable.Buffer + +/** + * A helper class for writing integration tests that involve producers, consumers, and servers + */ +trait IntegrationTestHarness extends KafkaServerTestHarness { + + val producerCount: Int + val consumerCount: Int + val serverCount: Int + lazy val producerConfig = new Properties + lazy val consumerConfig = new Properties + lazy val serverConfig = new Properties + override lazy val configs = { + val cfgs = TestUtils.createBrokerConfigs(serverCount) + cfgs.map(_.putAll(serverConfig)) + cfgs.map(new KafkaConfig(_)) + } + + var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + var producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() + + override def setUp() { + super.setUp() + producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) + producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) + consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + for(i <- 0 until producerCount) + producers += new KafkaProducer(producerConfig) + for(i <- 0 until consumerCount) + consumers += new KafkaConsumer(consumerConfig) + } + + override def tearDown() { + producers.map(_.close()) + consumers.map(_.close()) + super.tearDown() + } + +} diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index cd16ced..a1f72f8 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -192,7 +192,7 @@ object SerializationTestUtils { } def createConsumerMetadataResponse: ConsumerMetadataResponse = { - ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError) + ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError, 0) } def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = { @@ -237,7 +237,7 @@ class RequestResponseSerializationTest extends JUnitSuite { private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse private val consumerMetadataRequest = SerializationTestUtils.createConsumerMetadataRequest private val consumerMetadataResponse = SerializationTestUtils.createConsumerMetadataResponse - private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode) + private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, 0) private val heartbeatRequest = SerializationTestUtils.createHeartbeatRequestAndHeader private val heartbeatResponse = SerializationTestUtils.createHeartbeatResponseAndHeader private val joinGroupRequest = SerializationTestUtils.createJoinGroupRequestAndHeader diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 3cf7c9b..ef4c9ae 100644 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -17,11 +17,14 @@ package kafka.integration +import java.util.Arrays +import scala.collection.mutable.Buffer import kafka.server._ import kafka.utils.{Utils, TestUtils} import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import kafka.common.KafkaException +import kafka.utils.TestUtils /** * A test harness that brings up some number of broker nodes @@ -29,15 +32,22 @@ import kafka.common.KafkaException trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { val configs: List[KafkaConfig] - var servers: List[KafkaServer] = null + var servers: Buffer[KafkaServer] = null var brokerList: String = null - + var alive: Array[Boolean] = null + + def serverForId(id: Int) = servers.find(s => s.config.brokerId == id) + + def bootstrapUrl = configs.map(c => c.hostName + ":" + c.port).mkString(",") + override def setUp() { super.setUp if(configs.size <= 0) throw new KafkaException("Must suply at least one server config.") brokerList = TestUtils.getBrokerListStrFromConfigs(configs) - servers = configs.map(TestUtils.createServer(_)) + servers = configs.map(TestUtils.createServer(_)).toBuffer + alive = new Array[Boolean](servers.length) + Arrays.fill(alive, true) } override def tearDown() { @@ -45,4 +55,27 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { servers.map(server => server.config.logDirs.map(Utils.rm(_))) super.tearDown } + + /** + * Pick a broker at random and kill it if it isn't already dead + * Return the id of the broker killed + */ + def killRandomBroker(): Int = { + val index = TestUtils.random.nextInt(servers.length) + if(alive(index)) { + servers(index).shutdown() + alive(index) = false + } + index + } + + /** + * Restart any dead brokers + */ + def restartDeadBrokers() { + for(i <- 0 until servers.length if !alive(i)) { + servers(i) = TestUtils.createServer(configs(i)) + alive(i) = true + } + } } diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index a5386a0..aeb7a19 100644 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -32,6 +32,7 @@ import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionExc import kafka.utils.{StaticPartitioner, TestUtils, Utils} import kafka.serializer.StringEncoder import java.util.Properties +import TestUtils._ /** * End to end tests of the primitive apis against a local server @@ -113,7 +114,8 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with } private def produceAndMultiFetch(producer: Producer[String, String]) { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4")) + for(topic <- List("test1", "test2", "test3", "test4")) + TestUtils.createTopic(zkClient, topic, servers = servers) // send some messages val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); @@ -181,7 +183,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with private def multiProduce(producer: Producer[String, String]) { val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) - createSimpleTopicsAndAwaitLeader(zkClient, topics.keys) + topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers)) val messages = new mutable.HashMap[String, Seq[String]] val builder = new FetchRequestBuilder() @@ -215,7 +217,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with def testPipelinedProduceRequests() { val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) - createSimpleTopicsAndAwaitLeader(zkClient, topics.keys) + topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers)) val props = new Properties() props.put("request.required.acks", "0") val pipelinedProducer: Producer[String, String] = @@ -265,15 +267,4 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) } } - - /** - * For testing purposes, just create these topics each with one partition and one replica for - * which the provided broker should the leader for. Create and wait for broker to lead. Simple. - */ - private def createSimpleTopicsAndAwaitLeader(zkClient: ZkClient, topics: Iterable[String]) { - for( topic <- topics ) { - AdminUtils.createTopic(zkClient, topic, partitions = 1, replicationFactor = 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition = 0) - } - } } diff --git a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala index d5896ed..c674078 100644 --- a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala +++ b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala @@ -36,6 +36,8 @@ class MockScheduler(val time: Time) extends Scheduler { /* a priority queue of tasks ordered by next execution time */ var tasks = new PriorityQueue[MockTask]() + + def isStarted = true def startup() {} diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index b364ac2..cfea63b 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -90,4 +90,21 @@ class SchedulerTest { assertTrue("Should count to 20", counter1.get >= 20) } } + + @Test + def testRestart() { + // schedule a task to increment a counter + mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1) + mockTime.sleep(1) + assertEquals(1, counter1.get()) + + // restart the scheduler + mockTime.scheduler.shutdown() + mockTime.scheduler.startup() + + // schedule another task to increment the counter + mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1) + mockTime.sleep(1) + assertEquals(2, counter1.get()) + } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index ac15d34..54755e8 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -45,6 +45,7 @@ import kafka.log._ import junit.framework.AssertionFailedError import junit.framework.Assert._ import org.apache.kafka.clients.producer.KafkaProducer +import collection.Iterable import scala.collection.Map @@ -709,24 +710,21 @@ object TestUtils extends Logging { /** * Create new LogManager instance with default configuration for testing */ - def createLogManager( - logDirs: Array[File] = Array.empty[File], - defaultConfig: LogConfig = LogConfig(), - cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), - time: MockTime = new MockTime()) = - { - new LogManager( - logDirs = logDirs, - topicConfigs = Map(), - defaultConfig = defaultConfig, - cleanerConfig = cleanerConfig, - ioThreads = 4, - flushCheckMs = 1000L, - flushCheckpointMs = 10000L, - retentionCheckMs = 1000L, - scheduler = time.scheduler, - time = time, - brokerState = new BrokerState()) + def createLogManager(logDirs: Array[File] = Array.empty[File], + defaultConfig: LogConfig = LogConfig(), + cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), + time: MockTime = new MockTime()): LogManager = { + new LogManager(logDirs = logDirs, + topicConfigs = Map(), + defaultConfig = defaultConfig, + cleanerConfig = cleanerConfig, + ioThreads = 4, + flushCheckMs = 1000L, + flushCheckpointMs = 10000L, + retentionCheckMs = 1000L, + scheduler = time.scheduler, + time = time, + brokerState = new BrokerState()) } def sendMessagesToPartition(configs: Seq[KafkaConfig], -- 1.9.3 (Apple Git-50) From 75a286e4308831203cc7e1d4748bdab504e8a1f3 Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Thu, 29 Jan 2015 20:09:36 -0800 Subject: [PATCH 11/41] KAFKA-1760 Follow-up: fix compilation issue with Scala 2.11 --- core/src/main/scala/kafka/server/KafkaApis.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 48bc435..f2b027b 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -455,12 +455,13 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleJoinGroupRequest(request: RequestChannel.Request) { + import JavaConversions._ val joinGroupReq = request.requestObj.asInstanceOf[JoinGroupRequestAndHeader] - val topics = JavaConversions.asScalaIterable(joinGroupReq.body.topics()).toSet + val topics = joinGroupReq.body.topics().toSet val partitions = this.replicaManager.logManager.allLogs.filter(log => topics.contains(log.topicAndPartition.topic)) val partitionList = partitions.map(_.topicAndPartition).map(tp => new org.apache.kafka.common.TopicPartition(tp.topic, tp.partition)).toBuffer this.consumerGroupGenerationId += 1 - val response = new JoinGroupResponse(ErrorMapping.NoError, this.consumerGroupGenerationId, joinGroupReq.body.consumerId, JavaConversions.asJavaList(partitionList)) + val response = new JoinGroupResponse(ErrorMapping.NoError, this.consumerGroupGenerationId, joinGroupReq.body.consumerId, partitionList) val send = new BoundedByteBufferSend(new JoinGroupResponseAndHeader(joinGroupReq.correlationId, response)) requestChannel.sendResponse(new RequestChannel.Response(request, send)) } -- 1.9.3 (Apple Git-50) From 7bc9b7ff0a31aed04f3e5630c31750d7f41dc133 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 4 Dec 2014 21:50:21 -0800 Subject: [PATCH 12/41] first commit of refactoring. --- .../org/apache/kafka/clients/NetworkClient.java | 9 +- .../kafka/clients/producer/KafkaProducer.java | 3 +- .../kafka/clients/producer/ProducerConfig.java | 13 ++- .../org/apache/kafka/common/protocol/Protocol.java | 12 ++- .../kafka/common/requests/MetadataRequest.java | 29 +++++- .../apache/kafka/clients/NetworkClientTest.java | 2 +- .../kafka/common/requests/RequestResponseTest.java | 2 +- core/src/main/scala/kafka/admin/AdminUtils.scala | 9 +- core/src/main/scala/kafka/admin/TopicCommand.scala | 4 +- .../scala/kafka/api/ConsumerMetadataRequest.scala | 16 +++- .../scala/kafka/api/ConsumerMetadataResponse.scala | 8 +- core/src/main/scala/kafka/api/TopicMetadata.scala | 26 +++-- .../scala/kafka/api/TopicMetadataRequest.scala | 29 +++++- .../scala/kafka/api/TopicMetadataResponse.scala | 14 ++- core/src/main/scala/kafka/client/ClientUtils.scala | 26 ++--- core/src/main/scala/kafka/cluster/Broker.scala | 106 +++++++++++++++++---- .../main/scala/kafka/cluster/BrokerEndPoint.scala | 59 ++++++++++++ core/src/main/scala/kafka/cluster/EndPoint.scala | 58 +++++++++++ .../main/scala/kafka/cluster/ProtocolType.scala | 26 +++++ .../BrokerEndPointNotAvailableException.scala | 22 +++++ .../main/scala/kafka/consumer/ConsumerConfig.scala | 5 + .../kafka/consumer/ConsumerFetcherManager.scala | 6 +- .../kafka/consumer/ConsumerFetcherThread.scala | 4 +- .../consumer/ZookeeperConsumerConnector.scala | 3 +- .../controller/ControllerChannelManager.scala | 3 +- .../scala/kafka/controller/KafkaController.scala | 6 +- .../kafka/javaapi/ConsumerMetadataResponse.scala | 4 +- .../main/scala/kafka/javaapi/TopicMetadata.scala | 2 +- .../scala/kafka/javaapi/TopicMetadataRequest.scala | 11 ++- .../main/scala/kafka/network/BlockingChannel.scala | 5 +- .../main/scala/kafka/network/SocketServer.scala | 30 ++++-- .../main/scala/kafka/producer/ProducerConfig.scala | 4 + .../main/scala/kafka/producer/ProducerPool.scala | 12 ++- .../kafka/server/AbstractFetcherManager.scala | 2 +- .../scala/kafka/server/AbstractFetcherThread.scala | 4 +- core/src/main/scala/kafka/server/KafkaApis.scala | 6 +- core/src/main/scala/kafka/server/KafkaConfig.scala | 27 +++--- .../main/scala/kafka/server/KafkaHealthcheck.scala | 13 +-- core/src/main/scala/kafka/server/KafkaServer.scala | 9 +- .../scala/kafka/server/ReplicaFetcherThread.scala | 2 +- .../scala/kafka/tools/ConsumerOffsetChecker.scala | 3 +- .../main/scala/kafka/tools/GetOffsetShell.scala | 4 +- .../kafka/tools/ReplicaVerificationTool.scala | 6 +- .../scala/kafka/tools/SimpleConsumerShell.scala | 14 ++- .../main/scala/kafka/tools/UpdateOffsetsInZK.scala | 5 +- core/src/main/scala/kafka/utils/Utils.scala | 8 ++ core/src/main/scala/kafka/utils/ZkUtils.scala | 17 +++- .../kafka/api/ProducerFailureHandlingTest.scala | 12 ++- .../integration/kafka/api/ProducerSendTest.scala | 8 +- .../test/scala/other/kafka/TestOffsetManager.scala | 5 +- .../scala/unit/kafka/admin/AddPartitionsTest.scala | 10 +- .../api/RequestResponseSerializationTest.scala | 12 ++- .../test/scala/unit/kafka/cluster/BrokerTest.scala | 94 ++++++++++++++++++ .../unit/kafka/consumer/ConsumerIteratorTest.scala | 4 +- .../scala/unit/kafka/integration/FetcherTest.scala | 4 +- .../unit/kafka/integration/TopicMetadataTest.scala | 14 +-- .../unit/kafka/network/SocketServerTest.scala | 6 +- .../unit/kafka/producer/SyncProducerTest.scala | 25 +++-- .../unit/kafka/server/AdvertiseBrokerTest.scala | 11 ++- .../scala/unit/kafka/server/KafkaConfigTest.scala | 21 ++-- .../unit/kafka/server/LeaderElectionTest.scala | 5 +- .../scala/unit/kafka/server/LogOffsetTest.scala | 2 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 13 ++- 63 files changed, 723 insertions(+), 211 deletions(-) create mode 100644 core/src/main/scala/kafka/cluster/BrokerEndPoint.scala create mode 100644 core/src/main/scala/kafka/cluster/EndPoint.scala create mode 100644 core/src/main/scala/kafka/cluster/ProtocolType.scala create mode 100644 core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala create mode 100644 core/src/test/scala/unit/kafka/cluster/BrokerTest.scala 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 5950191..9a2318d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -80,13 +80,17 @@ public class NetworkClient implements KafkaClient { /* the last timestamp when no broker node is available to connect */ private long lastNoNodeAvailableMs; + /* protocol used for communication to brokers */ + private String securityProtocol; + public NetworkClient(Selectable selector, Metadata metadata, String clientId, int maxInFlightRequestsPerConnection, long reconnectBackoffMs, int socketSendBuffer, - int socketReceiveBuffer) { + int socketReceiveBuffer, + String securityProtocol) { this.selector = selector; this.metadata = metadata; this.clientId = clientId; @@ -98,6 +102,7 @@ public class NetworkClient implements KafkaClient { this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE); this.metadataFetchInProgress = false; this.lastNoNodeAvailableMs = 0; + this.securityProtocol = securityProtocol; } /** @@ -435,7 +440,7 @@ public class NetworkClient implements KafkaClient { * Create a metadata request for the given topics */ private ClientRequest metadataRequest(long now, int node, Set topics) { - MetadataRequest metadata = new MetadataRequest(new ArrayList(topics)); + MetadataRequest metadata = new MetadataRequest(new ArrayList(topics),securityProtocol); RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct()); return new ClientRequest(now, true, send, null); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index ebc4c53..0986726 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 @@ -195,7 +195,8 @@ public class KafkaProducer implements Producer { config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG), config.getInt(ProducerConfig.SEND_BUFFER_CONFIG), - config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG)); + config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG), + config.getString(ProducerConfig.SECURITY_PROTOCOL)); this.sender = new Sender(client, this.metadata, this.accumulator, 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 9a43d66..bc6cd91 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 @@ -157,7 +157,6 @@ public class ProducerConfig extends AbstractConfig { private static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC = "The maximum number of unacknowledged requests the client will send on a single connection before blocking." + " Note that if this setting is set to be greater than 1 and there are failed sends, there is a risk of" + " message re-ordering due to retries (i.e., if retries are enabled)."; - /** key.serializer */ public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; private static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; @@ -166,6 +165,10 @@ 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."; + /** security.protocol */ + public static final String SECURITY_PROTOCOL = "security.protocol"; + private static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT is supported. SSL and Kerberos are planned for the near future"; + 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) @@ -214,7 +217,13 @@ 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(SECURITY_PROTOCOL, + Type.STRING, + "PLAINTEXT", + in(Arrays.asList("PLAINTEXT")), + Importance.MEDIUM, + SECURITY_PROTOCOL_DOC); } ProducerConfig(Map props) { diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 7517b87..29ce7af 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -47,11 +47,17 @@ public class Protocol { new ArrayOf(STRING), "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics.")); + public static Schema METADATA_REQUEST_V1 = new Schema(new Field("topics", + new ArrayOf(STRING), + "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics."), + new Field("security_protocol",STRING,"The response should contain broker endpoints that correspond to this protocol")); + public static Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."), new Field("host", STRING, "The hostname of the broker."), new Field("port", INT32, "The port on which the broker accepts requests.")); - public static Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code", + + public static Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code", INT16, "The error code for the partition, if any."), new Field("partition_id", INT32, "The id of the partition."), @@ -76,8 +82,8 @@ public class Protocol { "Host and port information for all brokers."), new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V0))); - public static Schema[] METADATA_REQUEST = new Schema[] { METADATA_REQUEST_V0 }; - public static Schema[] METADATA_RESPONSE = new Schema[] { METADATA_RESPONSE_V0 }; + public static Schema[] METADATA_REQUEST = new Schema[] { METADATA_REQUEST_V0, METADATA_REQUEST_V1 }; + public static Schema[] METADATA_RESPONSE = new Schema[] { METADATA_RESPONSE_V0, METADATA_RESPONSE_V0 }; /* Produce api */ 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 0186783..833797e 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 @@ -24,16 +24,31 @@ import org.apache.kafka.common.protocol.types.Struct; public class MetadataRequest extends AbstractRequestResponse { private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); - private static final String TOPICS_KEY_NAME = "topics"; + public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); + public static String DEFAULT_PROTOCOL = "PLAINTEXT"; + private static String TOPICS_KEY_NAME = "topics"; + private static String PROTOCOL_KEY_NAME = "security_protocol"; + private final List topics; + private String protocol; + /* Constructor for V0 */ public MetadataRequest(List topics) { super(new Struct(CURRENT_SCHEMA)); struct.set(TOPICS_KEY_NAME, topics.toArray()); this.topics = topics; } + /* Constructor for V1 */ + public MetadataRequest(List topics, String protocol) { + super(new Struct(ProtoUtils.requestSchema(ApiKeys.METADATA.id,1))); + struct.set(TOPICS_KEY_NAME, topics.toArray()); + struct.set(PROTOCOL_KEY_NAME, protocol); + this.topics = topics; + this.protocol = protocol; + } + public MetadataRequest(Struct struct) { super(struct); Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); @@ -41,6 +56,9 @@ public class MetadataRequest extends AbstractRequestResponse { for (Object topicObj: topicArray) { topics.add((String) topicObj); } + if (struct.hasField(PROTOCOL_KEY_NAME)) { + protocol = struct.getString(PROTOCOL_KEY_NAME); + } } public List topics() { @@ -48,6 +66,15 @@ public class MetadataRequest extends AbstractRequestResponse { } public static MetadataRequest parse(ByteBuffer buffer) { +<<<<<<< HEAD return new MetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer))); +======= + return new MetadataRequest(((Struct) curSchema.read(buffer))); + } + + public static MetadataRequest parse(ByteBuffer buffer, int versionId) { + Schema schema = ProtoUtils.requestSchema(ApiKeys.METADATA.id,versionId); + return new MetadataRequest(((Struct) schema.read(buffer))); +>>>>>>> first commit of refactoring. } } 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 5debcd6..2f9a46d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -36,7 +36,7 @@ public class NetworkClientTest { private int nodeId = 1; private Cluster cluster = TestUtils.singletonCluster("test", nodeId); private Node node = cluster.nodes().get(0); - private NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024); + private NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024,"PLAINTEXT"); @Before public void setup() { 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 df37fc6..58b5c3e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -124,7 +124,7 @@ public class RequestResponseTest { } private AbstractRequestResponse createMetadataRequest() { - return new MetadataRequest(Arrays.asList("topic1")); + return new MetadataRequest(Arrays.asList("topic1"), "PLAINTEXT"); } private AbstractRequestResponse createMetadataResponse() { diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 28b12c7..d06775c 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -18,7 +18,12 @@ package kafka.admin import kafka.common._ -import kafka.cluster.Broker +import java.util.Random +import java.util.Properties +import kafka.api.{TopicMetadata, PartitionMetadata} +import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.{BrokerEndpoint, Broker, ProtocolType} + import kafka.log.LogConfig import kafka.utils.{Logging, ZkUtils, Json} import kafka.api.{TopicMetadata, PartitionMetadata} @@ -287,6 +292,8 @@ object AdminUtils extends Logging { topics.map(topic => fetchTopicMetadataFromZk(topic, zkClient, cachedBrokerInfo)) } + + private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker]): TopicMetadata = { if(ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) { val topicPartitionAssignment = ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic).get diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 285c033..17e50bf 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -196,9 +196,7 @@ object TopicCommand { } } } - - def formatBroker(broker: Broker) = broker.id + " (" + formatAddress(broker.host, broker.port) + ")" - + def parseTopicConfigsToBeAdded(opts: TopicCommandOptions): Properties = { val configsToBeAdded = opts.options.valuesOf(opts.configOpt).map(_.split("""\s*=\s*""")) require(configsToBeAdded.forall(config => config.length == 2), diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index a3b1b78..ed950af 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -18,12 +18,14 @@ package kafka.api import java.nio.ByteBuffer +import kafka.cluster.ProtocolType +import kafka.cluster.ProtocolType.ProtocolType import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.common.ErrorMapping object ConsumerMetadataRequest { - val CurrentVersion = 0.shortValue + val CurrentVersion = 1.shortValue val DefaultClientId = "" def readFrom(buffer: ByteBuffer) = { @@ -31,10 +33,10 @@ object ConsumerMetadataRequest { val versionId = buffer.getShort val correlationId = buffer.getInt val clientId = ApiUtils.readShortString(buffer) - // request val group = ApiUtils.readShortString(buffer) - ConsumerMetadataRequest(group, versionId, correlationId, clientId) + val securityProtocol = ProtocolType.withName(ApiUtils.readShortString(buffer)) + ConsumerMetadataRequest(group, versionId, correlationId, clientId, securityProtocol) } } @@ -42,14 +44,16 @@ object ConsumerMetadataRequest { case class ConsumerMetadataRequest(group: String, versionId: Short = ConsumerMetadataRequest.CurrentVersion, correlationId: Int = 0, - clientId: String = ConsumerMetadataRequest.DefaultClientId) + clientId: String = ConsumerMetadataRequest.DefaultClientId, + securityProtocol: ProtocolType = ProtocolType.PLAINTEXT) extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey)) { def sizeInBytes = 2 + /* versionId */ 4 + /* correlationId */ ApiUtils.shortStringLength(clientId) + - ApiUtils.shortStringLength(group) + ApiUtils.shortStringLength(group) + + ApiUtils.shortStringLength(securityProtocol.toString) def writeTo(buffer: ByteBuffer) { // envelope @@ -59,6 +63,7 @@ case class ConsumerMetadataRequest(group: String, // consumer metadata request ApiUtils.writeShortString(buffer, group) + ApiUtils.writeShortString(buffer, securityProtocol.toString) } override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { @@ -74,6 +79,7 @@ case class ConsumerMetadataRequest(group: String, consumerMetadataRequest.append("; CorrelationId: " + correlationId) consumerMetadataRequest.append("; ClientId: " + clientId) consumerMetadataRequest.append("; Group: " + group) + consumerMetadataRequest.append("; SecurityProtocol" + securityProtocol.toString) consumerMetadataRequest.toString() } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala index 24aaf95..b116d62 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -18,18 +18,18 @@ package kafka.api import java.nio.ByteBuffer -import kafka.cluster.Broker +import kafka.cluster.{ProtocolType, BrokerEndpoint, Broker} import kafka.common.ErrorMapping object ConsumerMetadataResponse { val CurrentVersion = 0 - private val NoBrokerOpt = Some(Broker(id = -1, host = "", port = -1)) + private val NoBrokerOpt = Some(BrokerEndpoint(id = -1, host = "", port = -1, protocolType = ProtocolType.PLAINTEXT)) def readFrom(buffer: ByteBuffer) = { val correlationId = buffer.getInt val errorCode = buffer.getShort - val broker = Broker.readFrom(buffer) + val broker = BrokerEndpoint.readFrom(buffer) val coordinatorOpt = if (errorCode == ErrorMapping.NoError) Some(broker) else @@ -40,7 +40,7 @@ object ConsumerMetadataResponse { } -case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int) +case class ConsumerMetadataResponse (coordinatorOpt: Option[BrokerEndpoint], errorCode: Short, correlationId: Int) extends RequestOrResponse() { def sizeInBytes = diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index 0190076..b1ff35e 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -17,7 +17,7 @@ package kafka.api -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.utils.Logging @@ -110,10 +110,23 @@ object PartitionMetadata { } case class PartitionMetadata(partitionId: Int, - val leader: Option[Broker], - replicas: Seq[Broker], + val leader: Option[Broker], + replicas: Seq[Broker], isr: Seq[Broker] = Seq.empty, errorCode: Short = ErrorMapping.NoError) extends Logging { + + /*def this(partitionId: Int, + leader: Option[BrokerEndPoint], + replicas: Seq[BrokerEndPoint], + isr: Seq[BrokerEndPoint] = Seq.empty, + errorCode: Short = ErrorMapping.NoError) = { + this(partitionId, + Some(if (leader.isDefined) Broker.createBroker(leader.get) else Broker.noBroker()), + replicas.map(endpoint=>Broker.createBroker(endpoint)), + isr.map(endpoint=>Broker.createBroker(endpoint)), + errorCode) + }*/ + def sizeInBytes: Int = { 2 /* error code */ + 4 /* partition id */ + @@ -142,14 +155,13 @@ case class PartitionMetadata(partitionId: Int, override def toString(): String = { val partitionMetadataString = new StringBuilder partitionMetadataString.append("\tpartition " + partitionId) - partitionMetadataString.append("\tleader: " + (if(leader.isDefined) formatBroker(leader.get) else "none")) - partitionMetadataString.append("\treplicas: " + replicas.map(formatBroker).mkString(",")) - partitionMetadataString.append("\tisr: " + isr.map(formatBroker).mkString(",")) + partitionMetadataString.append("\tleader: " + (if(leader.isDefined) leader.get.toString else "none")) + partitionMetadataString.append("\treplicas: " + replicas.mkString(",")) + partitionMetadataString.append("\tisr: " + isr.mkString(",")) partitionMetadataString.append("\tisUnderReplicated: %s".format(if(isr.size < replicas.size) "true" else "false")) partitionMetadataString.toString() } - private def formatBroker(broker: Broker) = broker.id + " (" + formatAddress(broker.host, broker.port) + ")" } diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 7dca09c..4f1ad1a 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -19,6 +19,9 @@ package kafka.api import java.nio.ByteBuffer import kafka.api.ApiUtils._ +import kafka.cluster.ProtocolType +import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.ProtocolType.ProtocolType import collection.mutable.ListBuffer import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping @@ -26,34 +29,47 @@ import kafka.network.RequestChannel.Response import kafka.utils.Logging object TopicMetadataRequest extends Logging { - val CurrentVersion = 0.shortValue + val CurrentVersion = 1.shortValue val DefaultClientId = "" /** * TopicMetadataRequest has the following format - - * number of topics (4 bytes) list of topics (2 bytes + topic.length per topic) detailedMetadata (2 bytes) timestamp (8 bytes) count (4 bytes) + * security protocol (2 bytes + protocol.length) + * number of topics (4 bytes) + * list of topics (2 bytes + topic.length per topic) + * detailedMetadata (2 bytes) + * timestamp (8 bytes) + * count (4 bytes) */ def readFrom(buffer: ByteBuffer): TopicMetadataRequest = { val versionId = buffer.getShort + assert(versionId == 0 || versionId == 1, + "Version " + versionId + " is invalid for TopicMetadataRequest. Valid versions are 0 or 1.") val correlationId = buffer.getInt val clientId = readShortString(buffer) val numTopics = readIntInRange(buffer, "number of topics", (0, Int.MaxValue)) val topics = new ListBuffer[String]() for(i <- 0 until numTopics) topics += readShortString(buffer) - new TopicMetadataRequest(versionId, correlationId, clientId, topics.toList) + + var securityProtocol = org.apache.kafka.common.requests.MetadataRequest.DEFAULT_PROTOCOL + if (versionId == 1) { + securityProtocol = readShortString(buffer) + } + new TopicMetadataRequest(versionId, correlationId, clientId, ProtocolType.withName(securityProtocol), topics.toList) } } case class TopicMetadataRequest(val versionId: Short, val correlationId: Int, val clientId: String, + val securityProtocol: ProtocolType, val topics: Seq[String]) extends RequestOrResponse(Some(RequestKeys.MetadataKey)){ def this(topics: Seq[String], correlationId: Int) = - this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics) + this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, ProtocolType.PLAINTEXT, topics) def writeTo(buffer: ByteBuffer) { buffer.putShort(versionId) @@ -61,6 +77,7 @@ case class TopicMetadataRequest(val versionId: Short, writeShortString(buffer, clientId) buffer.putInt(topics.size) topics.foreach(topic => writeShortString(buffer, topic)) + writeShortString(buffer, securityProtocol.toString) } def sizeInBytes(): Int = { @@ -68,7 +85,8 @@ case class TopicMetadataRequest(val versionId: Short, 4 + /* correlation id */ shortStringLength(clientId) + /* client id */ 4 + /* number of topics */ - topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */ + topics.foldLeft(0)(_ + shortStringLength(_)) + /* topics */ + shortStringLength(securityProtocol.toString) /* security protocol */ } override def toString(): String = { @@ -89,6 +107,7 @@ case class TopicMetadataRequest(val versionId: Short, topicMetadataRequest.append("; Version: " + versionId) topicMetadataRequest.append("; CorrelationId: " + correlationId) topicMetadataRequest.append("; ClientId: " + clientId) + topicMetadataRequest.append("; SecurityProtocol" + securityProtocol.toString) if(details) topicMetadataRequest.append("; Topics: " + topics.mkString(",")) topicMetadataRequest.toString() diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala index 92ac4e6..db9f88e 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala @@ -17,7 +17,7 @@ package kafka.api -import kafka.cluster.Broker +import kafka.cluster.{ProtocolType, BrokerEndpoint, Broker} import java.nio.ByteBuffer object TopicMetadataResponse { @@ -25,22 +25,28 @@ object TopicMetadataResponse { def readFrom(buffer: ByteBuffer): TopicMetadataResponse = { val correlationId = buffer.getInt val brokerCount = buffer.getInt - val brokers = (0 until brokerCount).map(_ => Broker.readFrom(buffer)) - val brokerMap = brokers.map(b => (b.id, b)).toMap + val brokers = (0 until brokerCount).map(_ => BrokerEndpoint.readFrom(buffer)) + /* The broker list we are using for the TopicMetadataResponse is a collection of end-points (so clients can use them for connections) + * But partitionMetadata has multiple use-cases, some of them seem to require actual brokers + * So converting endpoints to broker objects here to preserve the partitionMetadata as is. + * TODO: check if we can use endpoints in partitionMetadata */ + val brokerMap = brokers.map(b => (b.id, Broker.createBroker(b))).toMap val topicCount = buffer.getInt val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap)) new TopicMetadataResponse(brokers, topicsMetadata, correlationId) } } -case class TopicMetadataResponse(brokers: Seq[Broker], +case class TopicMetadataResponse(brokers: Seq[BrokerEndpoint], topicsMetadata: Seq[TopicMetadata], correlationId: Int) extends RequestOrResponse() { + val sizeInBytes: Int = { 4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum } + def writeTo(buffer: ByteBuffer) { buffer.putInt(correlationId) /* brokers */ diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index ebba87f..5acc878 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -16,7 +16,9 @@ */ package kafka.client -import scala.collection._ + import kafka.cluster.ProtocolType.ProtocolType + + import scala.collection._ import kafka.cluster._ import kafka.api._ import kafka.producer._ @@ -42,10 +44,10 @@ object ClientUtils extends Logging{ * @param producerConfig The producer's config * @return topic metadata response */ - def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], producerConfig: ProducerConfig, correlationId: Int): TopicMetadataResponse = { + def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndpoint], producerConfig: ProducerConfig, correlationId: Int): TopicMetadataResponse = { var fetchMetaDataSucceeded: Boolean = false var i: Int = 0 - val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, producerConfig.clientId, topics.toSeq) + val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, producerConfig.clientId, producerConfig.securityProtocol, topics.toSeq) var topicMetadataResponse: TopicMetadataResponse = null var t: Throwable = null // shuffle the list of brokers before sending metadata requests so that most requests don't get routed to the @@ -83,7 +85,7 @@ object ClientUtils extends Logging{ * @param clientId The client's identifier * @return topic metadata response */ - def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String, timeoutMs: Int, + def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndpoint], clientId: String, timeoutMs: Int, correlationId: Int = 0): TopicMetadataResponse = { val props = new Properties() props.put("metadata.broker.list", brokers.map(_.connectionString).mkString(",")) @@ -96,22 +98,22 @@ object ClientUtils extends Logging{ /** * Parse a list of broker urls in the form host1:port1, host2:port2, ... */ - def parseBrokerList(brokerListStr: String): Seq[Broker] = { + def parseBrokerList(brokerListStr: String, protocolType: ProtocolType = ProtocolType.PLAINTEXT): Seq[BrokerEndpoint] = { val brokersStr = Utils.parseCsvList(brokerListStr) brokersStr.zipWithIndex.map { case (address, brokerId) => - new Broker(brokerId, getHost(address), getPort(address)) + BrokerEndpoint.createBrokerEndPoint(brokerId,protocolType.toString + "://" + address) } } /** * Creates a blocking channel to a random broker */ - def channelToAnyBroker(zkClient: ZkClient, socketTimeoutMs: Int = 3000) : BlockingChannel = { + def channelToAnyBroker(zkClient: ZkClient, protocolType: ProtocolType, socketTimeoutMs: Int = 3000) : BlockingChannel = { var channel: BlockingChannel = null var connected = false while (!connected) { - val allBrokers = getAllBrokersInCluster(zkClient) + val allBrokers = getAllBrokerEndPointsForChannel(zkClient,protocolType) Random.shuffle(allBrokers).find { broker => trace("Connecting to broker %s:%d.".format(broker.host, broker.port)) try { @@ -136,19 +138,19 @@ object ClientUtils extends Logging{ /** * Creates a blocking channel to the offset manager of the given group */ - def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = { - var queryChannel = channelToAnyBroker(zkClient) + def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000, protocolType: ProtocolType = ProtocolType.PLAINTEXT) = { + var queryChannel = channelToAnyBroker(zkClient, protocolType) var offsetManagerChannelOpt: Option[BlockingChannel] = None while (!offsetManagerChannelOpt.isDefined) { - var coordinatorOpt: Option[Broker] = None + var coordinatorOpt: Option[BrokerEndpoint] = None while (!coordinatorOpt.isDefined) { try { if (!queryChannel.isConnected) - queryChannel = channelToAnyBroker(zkClient) + queryChannel = channelToAnyBroker(zkClient, protocolType) debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group)) queryChannel.send(ConsumerMetadataRequest(group)) val response = queryChannel.receive() diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 0060add..684bc32 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -18,17 +18,36 @@ package kafka.cluster import kafka.utils.Utils._ -import kafka.utils.Json -import kafka.api.ApiUtils._ +import kafka.utils.{Utils, Json} import java.nio.ByteBuffer -import kafka.common.{KafkaException, BrokerNotAvailableException} -import org.apache.kafka.common.utils.Utils._ +import kafka.common.{BrokerEndPointNotAvailableException, KafkaException, BrokerNotAvailableException} +import kafka.cluster.ProtocolType._ /** * A Kafka broker + * A broker has an id, a host and a collection of end-points + * each end-point is (port,protocolType) + * currently the only channel type is PlainText + * but we will add SSL and Kerberos in the future */ object Broker { + /** + * Create a broker object from id and JSON string + * @param id + * @param brokerInfoString + * The current JSON schema for a broker is: + * {"version":1, + * "host":"localhost", + * "jmx_port":9999, + * "timestamp":2233345666, + * "endpoints": [ + * {"port":9092, + * "protocolType":"plain"}, + * {"port":9093, + * "protocolType":"ssl"}] + * @return + */ def createBroker(id: Int, brokerInfoString: String): Broker = { if(brokerInfoString == null) throw new BrokerNotAvailableException("Broker id %s does not exist".format(id)) @@ -38,7 +57,8 @@ object Broker { val brokerInfo = m.asInstanceOf[Map[String, Any]] val host = brokerInfo.get("host").get.asInstanceOf[String] val port = brokerInfo.get("port").get.asInstanceOf[Int] - new Broker(id, host, port) + val endPoints = brokerInfo.get("endpoints").get.asInstanceOf[String] + new Broker(id, Utils.listenerListToEndPoints(endPoints)) case None => throw new BrokerNotAvailableException("Broker id %d does not exist".format(id)) } @@ -47,36 +67,86 @@ object Broker { } } + def createBroker(endPoint: BrokerEndpoint) = { + Broker(endPoint.id,List(EndPoint(endPoint.host, endPoint.port,endPoint.protocolType))) + } + + def noBroker() = { + Broker(-1,List()) + } + + /** + * + * @param buffer containing serialized broker + * current serialization is: + * id (int), host (size + string), number of endpoints (int), serialized endpoints + * @return broker object + */ def readFrom(buffer: ByteBuffer): Broker = { val id = buffer.getInt - val host = readShortString(buffer) - val port = buffer.getInt - new Broker(id, host, port) + val numEndpoints = buffer.getInt + + val endpoints = List.range(0,numEndpoints).map(i => EndPoint.readFrom(buffer)) + + new Broker(id, endpoints) } } -case class Broker(id: Int, host: String, port: Int) { - - override def toString: String = "id:" + id + ",host:" + host + ",port:" + port +case class Broker(id: Int, endPoints: Seq[EndPoint]) { + + override def toString: String = id + " : " + endPoints.mkString("(",",",")") + + def this(id: Int, host: String, port: Int) = { + this(id,List(EndPoint(host,port,ProtocolType.PLAINTEXT))) + } - def connectionString: String = formatAddress(host, port) def writeTo(buffer: ByteBuffer) { buffer.putInt(id) - writeShortString(buffer, host) - buffer.putInt(port) + buffer.putInt(endPoints.size) + for(endpoint <- endPoints) { + endpoint.writeTo(buffer) + } } - def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ + def sizeInBytes: Int = + 4 + /* broker id*/ + 4 + /* number of endPoints */ + endPoints.map(_.sizeInBytes).sum /* end points */ + def supportsChannel(protocolType: ProtocolType): Unit = { + endPoints.map((endpoint)=>(endpoint.protocolType)).contains(protocolType) + } + + def getBrokerEndPoint(protocolType: ProtocolType): BrokerEndpoint = { + val endpoint = endPoints.map((endpoint)=>(endpoint.protocolType,endpoint)).toMap.get(protocolType) + endpoint match { + case Some(endpoint) => new BrokerEndpoint(id,endpoint.host,endpoint.port,endpoint.protocolType) + case None => + throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id)) + } + + + } + + + + /* TODO: Unit test! */ override def equals(obj: Any): Boolean = { obj match { case null => false - case n: Broker => id == n.id && host == n.host && port == n.port + // Yes, Scala compares lists element by element + case n: Broker => id == n.id && endPoints == n.endPoints case _ => false } } - - override def hashCode(): Int = hashcode(id, host, port) + + /* TODO: Unit test! */ + override def hashCode(): Int = hashcode(id, endPoints) } + + + + + diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala new file mode 100644 index 0000000..bf6f406 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -0,0 +1,59 @@ +package kafka.cluster + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import org.apache.kafka.common.utils.Utils._ +import kafka.cluster.ProtocolType._ + +object BrokerEndpoint { + def createBrokerEndPoint(brokerId: Int, connectionString: String): BrokerEndpoint = { + val endPoint = EndPoint.createEndPoint(connectionString) + new BrokerEndpoint(brokerId,endPoint.host,endPoint.port.toInt,endPoint.protocolType) + } + + /** + * BrokerEndpoint includes the protocol type to allow locating the right endpoint in lists + * but we don't serialize or de-serialize it + * this allows us to keep the wire protocol with the clients unchanged where the protocol is not needed + * @param buffer + * @return + */ + def readFrom(buffer: ByteBuffer): BrokerEndpoint = { + val brokerId = buffer.getInt() + val host = readShortString(buffer) + val port = buffer.getInt() + BrokerEndpoint(brokerId,host,port,null) + } +} + +// Utility class, representing a particular method of connecting to a broker +// Mostly to be used by clients +// This is not a broker and is not stored in ZooKeeper +case class BrokerEndpoint(id: Int, host: String, port: Int, protocolType: ProtocolType) { + + def connectionString(): String = formatEndpoint(host,port,protocolType) + + /** + * Formats broker endpoint as "channel://host:port" address string, + * surrounding IPv6 addresses with braces '[', ']' + * @param host hostname + * @param port port number + * @param protocolType String representing channel type (plain, SSL, KRB) + * @return address string + */ + def formatEndpoint(host: String, port: Integer, protocolType: ProtocolType): String = { + return protocolType + "://" + formatAddress(host, port) + } + + def writeTo(buffer: ByteBuffer): Unit = { + buffer.putInt(id) + writeShortString(buffer,host) + buffer.putInt(port) + } + + def sizeInBytes: Int = + 4 + /* broker Id */ + 4 + /* port */ + shortStringLength(host) +} diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala new file mode 100644 index 0000000..085b333 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.cluster + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.KafkaException +import kafka.cluster.ProtocolType._ + +object EndPoint { + + def readFrom(buffer: ByteBuffer): EndPoint = { + val port = buffer.getInt() + val host = readShortString(buffer) + val channel = readShortString(buffer) + EndPoint(host,port,ProtocolType.withName(channel)) + } + + def createEndPoint(connectionString: String): EndPoint = { + val uriParseExp = """^(.*)://([0-9a-z\-.]+):([0-9]+)""".r + connectionString match { + case uriParseExp(protocol,host,port) => new EndPoint(host,port.toInt,ProtocolType.withName(protocol)) + case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") + } + } +} + +case class EndPoint(host: String, port: Int, protocolType: ProtocolType) { + + override def toString: String = protocolType + "://" + host + ":" + port + + def writeTo(buffer: ByteBuffer): Unit = { + buffer.putInt(port) + writeShortString(buffer,host) + writeShortString(buffer,protocolType.toString) + } + + def sizeInBytes: Int = + 4 + /* port */ + shortStringLength(host) + + shortStringLength(protocolType.toString) +} diff --git a/core/src/main/scala/kafka/cluster/ProtocolType.scala b/core/src/main/scala/kafka/cluster/ProtocolType.scala new file mode 100644 index 0000000..f4e6bc3 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/ProtocolType.scala @@ -0,0 +1,26 @@ +/* + * 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.cluster + + +object ProtocolType extends Enumeration { + + type ProtocolType = Value + val PLAINTEXT = Value +} + diff --git a/core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala b/core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala new file mode 100644 index 0000000..455d8c6 --- /dev/null +++ b/core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +class BrokerEndPointNotAvailableException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 9ebbee6..fe89a30 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -19,6 +19,7 @@ package kafka.consumer import java.util.Properties import kafka.api.OffsetRequest +import kafka.cluster.ProtocolType import kafka.utils._ import kafka.common.{InvalidConfigException, Config} @@ -45,6 +46,7 @@ object ConsumerConfig extends Config { val OffsetsChannelSocketTimeoutMs = 10000 val OffsetsCommitMaxRetries = 5 val OffsetsStorage = "zookeeper" + val SecurityProtocol = "PLAINTEXT" val MirrorTopicsWhitelistProp = "mirror.topics.whitelist" val MirrorTopicsBlacklistProp = "mirror.topics.blacklist" @@ -180,6 +182,9 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin */ val partitionAssignmentStrategy = props.getString("partition.assignment.strategy", DefaultPartitionAssignmentStrategy) + + /* plaintext or SSL */ + val securityProtocol = ProtocolType.withName(props.getString("security.protocol", "PLAINTEXT")) validate(this) } diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index b9e2bea..c6700d2 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -19,7 +19,7 @@ package kafka.consumer import org.I0Itec.zkclient.ZkClient import kafka.server.{BrokerAndInitialOffset, AbstractFetcherThread, AbstractFetcherManager} -import kafka.cluster.{Cluster, Broker} +import kafka.cluster.{BrokerEndpoint, Cluster, Broker} import scala.collection.immutable import scala.collection.Map import collection.mutable.HashMap @@ -62,7 +62,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, } trace("Partitions without leader %s".format(noLeaderPartitionSet)) - val brokers = getAllBrokersInCluster(zkClient) + val brokers = getAllBrokerEndPointsForChannel(zkClient,config.securityProtocol) val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, brokers, config.clientId, @@ -117,7 +117,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { new ConsumerFetcherThread( "ConsumerFetcherThread-%s-%d-%d".format(consumerIdString, fetcherId, sourceBroker.id), - config, sourceBroker, partitionMap, this) + config, sourceBroker.getBrokerEndPoint(config.securityProtocol), partitionMap, this) } def startConnections(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster) { diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index ee6139c..7e3816d 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -17,7 +17,7 @@ package kafka.consumer -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import kafka.server.AbstractFetcherThread import kafka.message.ByteBufferMessageSet import kafka.api.{Request, OffsetRequest, FetchResponsePartitionData} @@ -26,7 +26,7 @@ import kafka.common.TopicAndPartition class ConsumerFetcherThread(name: String, val config: ConsumerConfig, - sourceBroker: Broker, + sourceBroker: BrokerEndpoint, partitionMap: Map[TopicAndPartition, PartitionTopicInfo], val consumerFetcherManager: ConsumerFetcherManager) extends AbstractFetcherThread(name = name, diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 5487259..531b25d 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -185,7 +185,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def ensureOffsetManagerConnected() { if (config.offsetsStorage == "kafka") { if (offsetsChannel == null || !offsetsChannel.isConnected) - offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient, config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs) + offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient, + config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs, config.securityProtocol) debug("Connected to offset manager %s:%d.".format(offsetsChannel.host, offsetsChannel.port)) } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 14b22ab..f9fd523 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -80,7 +80,8 @@ class ControllerChannelManager (private val controllerContext: ControllerContext private def addNewBroker(broker: Broker) { val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize) debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id)) - val channel = new BlockingChannel(broker.host, broker.port, + val brokerEndPoint = broker.getBrokerEndPoint(config.securityProtocol) + val channel = new BlockingChannel(brokerEndPoint.host, brokerEndPoint.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 66df6d2..4fbcdd3 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -212,7 +212,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt def epoch = controllerContext.epoch - def clientId = "id_%d-host_%s-port_%d".format(config.brokerId, config.hostName, config.port) + def clientId = { + val listeners = listenerListToEndPoints(config.listeners) + val controllerListener = listeners.find(endpoint => endpoint.protocolType == config.securityProtocol) + "id_%d-host_%s-port_%d".format(config.brokerId, controllerListener.get.host, controllerListener.get.port) + } /** * On clean shutdown, the controller first determines the partitions that the diff --git a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala index 1b28861..b5c4289 100644 --- a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala @@ -17,13 +17,13 @@ package kafka.javaapi -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadataResponse) { def errorCode = underlying.errorCode - def coordinator: Broker = { + def coordinator: BrokerEndpoint = { import kafka.javaapi.Implicits._ underlying.coordinatorOpt } diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala index f384e04..255a9f7 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala @@ -16,7 +16,7 @@ */ package kafka.javaapi -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import scala.collection.JavaConversions private[javaapi] object MetadataListImplicits { diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index b0b7be1..3ea12c9 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -18,6 +18,10 @@ package kafka.javaapi import kafka.api._ import java.nio.ByteBuffer +import kafka.cluster.ProtocolType +import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.ProtocolType.ProtocolType + import scala.collection.mutable import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping @@ -26,19 +30,20 @@ import kafka.network.RequestChannel.Response class TopicMetadataRequest(val versionId: Short, val correlationId: Int, val clientId: String, + val securityProtocol: ProtocolType, val topics: java.util.List[String]) extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) { val underlying: kafka.api.TopicMetadataRequest = { import scala.collection.JavaConversions._ - new kafka.api.TopicMetadataRequest(versionId, correlationId, clientId, topics: mutable.Buffer[String]) + new kafka.api.TopicMetadataRequest(versionId, correlationId, clientId, securityProtocol, topics: mutable.Buffer[String]) } def this(topics: java.util.List[String]) = - this(kafka.api.TopicMetadataRequest.CurrentVersion, 0, kafka.api.TopicMetadataRequest.DefaultClientId, topics) + this(kafka.api.TopicMetadataRequest.CurrentVersion, 0, kafka.api.TopicMetadataRequest.DefaultClientId, ProtocolType.PLAINTEXT, topics) def this(topics: java.util.List[String], correlationId: Int) = - this(kafka.api.TopicMetadataRequest.CurrentVersion, correlationId, kafka.api.TopicMetadataRequest.DefaultClientId, topics) + this(kafka.api.TopicMetadataRequest.CurrentVersion, correlationId, kafka.api.TopicMetadataRequest.DefaultClientId, ProtocolType.PLAINTEXT, topics) def writeTo(buffer: ByteBuffer) = underlying.writeTo(buffer) diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala index 6e2a38e..70be9c1 100644 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ b/core/src/main/scala/kafka/network/BlockingChannel.scala @@ -72,7 +72,10 @@ class BlockingChannel( val host: String, connectTimeoutMs)) } catch { - case e: Throwable => disconnect() + case e: Throwable => { + error(e) + disconnect() + } } } } diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 39b1651..6f21057 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -24,7 +24,11 @@ import java.net._ import java.io._ import java.nio.channels._ +import kafka.cluster.EndPoint +import kafka.cluster.ProtocolType.ProtocolType + import scala.collection._ +import scala.collection.JavaConversions._ import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup @@ -38,8 +42,7 @@ import com.yammer.metrics.core.{Gauge, Meter} * M Handler threads that handle requests and produce responses back to the processor threads for writing. */ class SocketServer(val brokerId: Int, - val host: String, - val port: Int, + val endpoints: Seq[EndPoint], val numProcessorThreads: Int, val maxQueuedRequests: Int, val sendBufferSize: Int, @@ -51,7 +54,7 @@ class SocketServer(val brokerId: Int, this.logIdent = "[Socket Server on Broker " + brokerId + "], " private val time = SystemTime private val processors = new Array[Processor](numProcessorThreads) - @volatile private var acceptor: Acceptor = null + @volatile private var acceptors: ConcurrentHashMap[EndPoint,Acceptor] = new ConcurrentHashMap[EndPoint,Acceptor]() val requestChannel = new RequestChannel(numProcessorThreads, maxQueuedRequests) /* a meter to track the average free capacity of the network processors */ @@ -72,7 +75,7 @@ class SocketServer(val brokerId: Int, requestChannel, quotas, connectionsMaxIdleMs) - Utils.newThread("kafka-network-thread-%d-%d".format(port, i), processors(i), false).start() + Utils.newThread("kafka-network-thread-%d".format(i), processors(i), false).start() } newGauge("ResponsesBeingSent", new Gauge[Int] { @@ -83,10 +86,17 @@ class SocketServer(val brokerId: Int, requestChannel.addResponseListener((id:Int) => processors(id).wakeup()) // start accepting connections - this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize, quotas) - Utils.newThread("kafka-socket-acceptor", acceptor, false).start() - acceptor.awaitStartup - info("Started") + // right now we will use the same processors for all ports, since we didn't implement different protocols + // in the future, we may implement different processors for SSL and Kerberos + + endpoints.foreach(endpoint => { + val acceptor = new Acceptor(endpoint.host, endpoint.port, processors, sendBufferSize, recvBufferSize, quotas) + acceptors.put(endpoint,acceptor) + Utils.newThread("kafka-socket-acceptor-%s-%d".format(endpoint.protocolType.toString, endpoint.port), acceptor, false).start() + acceptor.awaitStartup + }) + + info("Started " + acceptors.size() + " acceptor threads") } /** @@ -94,8 +104,8 @@ class SocketServer(val brokerId: Int, */ def shutdown() = { info("Shutting down") - if(acceptor != null) - acceptor.shutdown() + if(acceptors != null) + acceptors.values().foreach(_.shutdown()) for(processor <- processors) processor.shutdown() info("Shutdown completed") diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index 3cdf23d..bbbf651 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -19,6 +19,7 @@ package kafka.producer import async.AsyncProducerConfig import java.util.Properties +import kafka.cluster.ProtocolType import kafka.utils.{Utils, VerifiableProperties} import kafka.message.{CompressionCodec, NoCompressionCodec} import kafka.common.{InvalidConfigException, Config} @@ -113,5 +114,8 @@ class ProducerConfig private (val props: VerifiableProperties) */ val topicMetadataRefreshIntervalMs = props.getInt("topic.metadata.refresh.interval.ms", 600000) + /* plaintext or SSL */ + val securityProtocol = ProtocolType.withName(props.getString("security.protocol", "PLAINTEXT")) + validate(this) } diff --git a/core/src/main/scala/kafka/producer/ProducerPool.scala b/core/src/main/scala/kafka/producer/ProducerPool.scala index 43df70b..72686fc 100644 --- a/core/src/main/scala/kafka/producer/ProducerPool.scala +++ b/core/src/main/scala/kafka/producer/ProducerPool.scala @@ -17,7 +17,7 @@ package kafka.producer -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import java.util.Properties import collection.mutable.HashMap import java.lang.Object @@ -30,7 +30,7 @@ object ProducerPool { /** * Used in ProducerPool to initiate a SyncProducer connection with a broker. */ - def createSyncProducer(config: ProducerConfig, broker: Broker): SyncProducer = { + def createSyncProducer(config: ProducerConfig, broker: BrokerEndpoint): SyncProducer = { val props = new Properties() props.put("host", broker.host) props.put("port", broker.port.toString) @@ -44,11 +44,13 @@ class ProducerPool(val config: ProducerConfig) extends Logging { private val lock = new Object() def updateProducer(topicMetadata: Seq[TopicMetadata]) { - val newBrokers = new collection.mutable.HashSet[Broker] + val newBrokers = new collection.mutable.HashSet[BrokerEndpoint] topicMetadata.foreach(tmd => { tmd.partitionsMetadata.foreach(pmd => { - if(pmd.leader.isDefined) - newBrokers+=(pmd.leader.get) + if(pmd.leader.isDefined) { + val endpoint = pmd.leader.get.endPoints.head + newBrokers += BrokerEndpoint(pmd.leader.get.id, endpoint.host,endpoint.port,config.securityProtocol) + } }) }) lock synchronized { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 20c00cb..36f054e 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.collection.Set import scala.collection.Map import kafka.utils.{Utils, Logging} -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import kafka.metrics.KafkaMetricsGroup import kafka.common.TopicAndPartition import com.yammer.metrics.core.Gauge diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 8c281d4..d770768 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -17,7 +17,7 @@ package kafka.server -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import kafka.utils.{Pool, ShutdownableThread} import kafka.consumer.{PartitionTopicInfo, SimpleConsumer} import kafka.api.{FetchRequest, FetchResponse, FetchResponsePartitionData, FetchRequestBuilder} @@ -36,7 +36,7 @@ import com.yammer.metrics.core.Gauge /** * Abstract class for fetching data from multiple partitions from the same broker. */ -abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: Broker, socketTimeout: Int, socketBufferSize: Int, +abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: BrokerEndpoint, socketTimeout: Int, socketBufferSize: Int, fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1, isInterruptible: Boolean = true) extends ShutdownableThread(name, isInterruptible) { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index f2b027b..f61d972 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -401,7 +401,7 @@ class KafkaApis(val requestChannel: RequestChannel, val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet) 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, topicMetadata, metadataRequest.correlationId) + val response = new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(metadataRequest.securityProtocol)), topicMetadata, metadataRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } @@ -440,12 +440,14 @@ class KafkaApis(val requestChannel: RequestChannel, // get metadata (and create the topic if necessary) val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName)).head + val securityProtocol = consumerMetadataRequest.securityProtocol + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) val response = offsetsTopicMetadata.partitionsMetadata.find(_.partitionId == partition).map { partitionMetadata => partitionMetadata.leader.map { leader => - ConsumerMetadataResponse(Some(leader), ErrorMapping.NoError, consumerMetadataRequest.correlationId) + ConsumerMetadataResponse(Some(leader.getBrokerEndPoint(securityProtocol)), ErrorMapping.NoError, consumerMetadataRequest.correlationId) }.getOrElse(errorResponse) }.getOrElse(errorResponse) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 6d74983..39a67a3 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -18,6 +18,7 @@ package kafka.server import java.util.Properties +import kafka.cluster.ProtocolType import kafka.message.{MessageSet, Message} import kafka.consumer.ConsumerConfig import kafka.utils.{VerifiableProperties, ZKConfig, Utils} @@ -99,23 +100,16 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /*********** Socket Server Configuration ***********/ - /* the port to listen and accept connections on */ - val port: Int = props.getInt("port", 9092) + /* Listener List - Comma-separated list of URIs we will listen on and their protocols. + * Specify hostname as 0.0.0.0 to bind to all interfaces + * Leave hostname empty to bind to default interface */ + val listeners: String = props.getString("listeners", "PLAINTEXT://0.0.0.0:6667") - /* hostname of broker. If this is set, it will only bind to this address. If this is not set, - * it will bind to all interfaces */ - val hostName: String = props.getString("host.name", null) + /* Listeners to publish to ZooKeeper for clients to use, if different than the listeners above. + * In IaaS environments, this may need to be different from the interface to which the broker binds. + * If this is not set, it will use the value for "listeners" */ + val advertisedListeners: String = props.getString("advertised.listeners", listeners) - /* hostname to publish to ZooKeeper for clients to use. In IaaS environments, this may - * need to be different from the interface to which the broker binds. If this is not set, - * it will use the value for "host.name" if configured. Otherwise - * it will use the value returned from java.net.InetAddress.getCanonicalHostName(). */ - val advertisedHostName: String = props.getString("advertised.host.name", hostName) - - /* the port to publish to ZooKeeper for clients to use. In IaaS environments, this may - * need to be different from the port to which the broker binds. If this is not set, - * it will publish the same port that the broker binds to. */ - val advertisedPort: Int = props.getInt("advertised.port", port) /* the SO_SNDBUFF buffer of the socket sever sockets */ val socketSendBufferBytes: Int = props.getInt("socket.send.buffer.bytes", 100*1024) @@ -135,6 +129,9 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* idle connections timeout: the server socket processor threads close the connections that idle more than this */ val connectionsMaxIdleMs = props.getLong("connections.max.idle.ms", 10*60*1000L) + /* security protocol used to communicate between brokers */ + val securityProtocol = ProtocolType.withName(props.getString("security.protocol","PLAINTEXT")) + /*********** Log Configuration ***********/ /* the default number of log partitions per topic */ diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 4acdd70..3ee1256 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -17,6 +17,7 @@ package kafka.server +import kafka.cluster.EndPoint import kafka.utils._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkStateListener, ZkClient} @@ -31,9 +32,8 @@ import java.net.InetAddress * Right now our definition of health is fairly naive. If we register in zk we are healthy, otherwise * we are dead. */ -class KafkaHealthcheck(private val brokerId: Int, - private val advertisedHost: String, - private val advertisedPort: Int, +class KafkaHealthcheck(private val brokerId: Int, + private val advertisedEndpoints: Seq[EndPoint], private val zkSessionTimeoutMs: Int, private val zkClient: ZkClient) extends Logging { @@ -54,13 +54,8 @@ class KafkaHealthcheck(private val brokerId: Int, * Register this broker as "alive" in zookeeper */ def register() { - val advertisedHostName = - if(advertisedHost == null || advertisedHost.trim.isEmpty) - InetAddress.getLocalHost.getCanonicalHostName - else - advertisedHost val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt - ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedHostName, advertisedPort, zkSessionTimeoutMs, jmxPort) + ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedEndpoints, zkSessionTimeoutMs, jmxPort) } /** diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 89200da..63ac879 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -94,8 +94,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg this.logIdent = "[Kafka Server " + config.brokerId + "], " socketServer = new SocketServer(config.brokerId, - config.hostName, - config.port, + Utils.listenerListToEndPoints(config.listeners), config.numNetworkThreads, config.queuedMaxRequests, config.socketSendBufferBytes, @@ -128,7 +127,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg topicConfigManager.startup() /* tell everyone we are alive */ - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) + val endpoints = Utils.listenerListToEndPoints(config.advertisedListeners) + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId,endpoints,config.zkSessionTimeoutMs, zkClient) kafkaHealthcheck.startup() registerStats() @@ -207,7 +207,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (channel != null) { channel.disconnect() } - channel = new BlockingChannel(broker.host, broker.port, + channel = new BlockingChannel(broker.getBrokerEndPoint(config.securityProtocol).host, + broker.getBrokerEndPoint(config.securityProtocol).port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 6879e73..5cf4cfd 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -30,7 +30,7 @@ class ReplicaFetcherThread(name:String, replicaMgr: ReplicaManager) extends AbstractFetcherThread(name = name, clientId = name, - sourceBroker = sourceBroker, + sourceBroker = sourceBroker.getBrokerEndPoint(brokerConfig.securityProtocol), socketTimeout = brokerConfig.replicaSocketTimeoutMs, socketBufferSize = brokerConfig.replicaSocketReceiveBufferBytes, fetchSize = brokerConfig.replicaFetchMaxBytes, diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index d1e7c43..7198322 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -19,6 +19,7 @@ package kafka.tools import joptsimple._ +import kafka.cluster.ProtocolType import org.I0Itec.zkclient.ZkClient import kafka.utils._ import kafka.consumer.SimpleConsumer @@ -158,7 +159,7 @@ object ConsumerOffsetChecker extends Logging { topicPidMap = immutable.Map(ZkUtils.getPartitionsForTopics(zkClient, topicList).toSeq:_*) val topicPartitions = topicPidMap.flatMap { case(topic, partitionSeq) => partitionSeq.map(TopicAndPartition(topic, _)) }.toSeq - val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs) + val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs, ProtocolType.PLAINTEXT) debug("Sending offset fetch request to coordinator %s:%d.".format(channel.host, channel.port)) channel.send(OffsetFetchRequest(group, topicPartitions)) diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 3d9293e..46078b7 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -24,6 +24,7 @@ import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} import kafka.common.TopicAndPartition import kafka.client.ClientUtils import kafka.utils.{ToolsUtils, CommandLineUtils} +import kafka.cluster.ProtocolType object GetOffsetShell { @@ -93,7 +94,8 @@ object GetOffsetShell { case Some(metadata) => metadata.leader match { case Some(leader) => - val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, clientId) + val consumer = new SimpleConsumer(leader.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, + leader.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, 10000, 100000, clientId) val topicAndPartition = TopicAndPartition(topic, partitionId) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index ba6ddd7..ca06c1e 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -18,7 +18,7 @@ package kafka.tools import joptsimple.OptionParser -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import kafka.message.{MessageSet, MessageAndOffset, ByteBufferMessageSet} import java.util.concurrent.CountDownLatch import java.util.concurrent.atomic.AtomicReference @@ -197,7 +197,7 @@ private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, c private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int], leadersPerBroker: Map[Int, Seq[TopicAndPartition]], expectedNumFetchers: Int, - brokerMap: Map[Int, Broker], + brokerMap: Map[Int, BrokerEndpoint], initialOffsetTime: Long, reportInterval: Long) extends Logging { private val fetchOffsetMap = new Pool[TopicAndPartition, Long] @@ -335,7 +335,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa } } -private class ReplicaFetcher(name: String, sourceBroker: Broker, topicAndPartitions: Iterable[TopicAndPartition], +private class ReplicaFetcher(name: String, sourceBroker: BrokerEndpoint, topicAndPartitions: Iterable[TopicAndPartition], replicaBuffer: ReplicaBuffer, socketTimeout: Int, socketBufferSize: Int, fetchSize: Int, maxWait: Int, minBytes: Int, doVerification: Boolean) extends ShutdownableThread(name) { diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index b4f903b..77c4900 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -22,7 +22,7 @@ import kafka.utils._ import kafka.consumer._ import kafka.client.ClientUtils import kafka.api.{OffsetRequest, FetchRequestBuilder, Request} -import kafka.cluster.Broker +import kafka.cluster.{ProtocolType, Broker} import scala.collection.JavaConversions._ import kafka.common.TopicAndPartition @@ -167,7 +167,9 @@ object SimpleConsumerShell extends Logging { System.exit(1) } if (startingOffset < 0) { - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, ConsumerConfig.SocketTimeout, + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, + fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, + ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize, clientId) try { startingOffset = simpleConsumer.earliestOrLatestOffset(TopicAndPartition(topic, partitionId), startingOffset, @@ -188,8 +190,12 @@ object SimpleConsumerShell extends Logging { val replicaString = if(replicaId > 0) "leader" else "replica" info("Starting simple consumer shell to partition [%s, %d], %s [%d], host and port: [%s, %d], from offset [%d]" - .format(topic, partitionId, replicaString, replicaId, fetchTargetBroker.host, fetchTargetBroker.port, startingOffset)) - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, 10000, 64*1024, clientId) + .format(topic, partitionId, replicaString, replicaId, + fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, + fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, startingOffset)) + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, + fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, + 10000, 64*1024, clientId) val thread = Utils.newThread("kafka-simpleconsumer-shell", new Runnable() { def run() { var offset = startingOffset diff --git a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala index 111c9a8..38d3b48 100644 --- a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala +++ b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala @@ -17,6 +17,7 @@ package kafka.tools +import kafka.cluster.ProtocolType import org.I0Itec.zkclient.ZkClient import kafka.consumer.{SimpleConsumer, ConsumerConfig} import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} @@ -65,7 +66,9 @@ object UpdateOffsetsInZK { ZkUtils.getBrokerInfo(zkClient, broker) match { case Some(brokerInfo) => - val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024, "UpdateOffsetsInZk") + val consumer = new SimpleConsumer(brokerInfo.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, + brokerInfo.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, + 10000, 100 * 1024, "UpdateOffsetsInZk") val topicAndPartition = TopicAndPartition(topic, partition) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1))) val offset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 738c1af..b823c1e 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -24,11 +24,14 @@ import java.nio.channels._ import java.util.concurrent.locks.{ReadWriteLock, Lock} import java.lang.management._ import javax.management._ + import scala.collection._ import scala.collection.mutable import java.util.Properties import kafka.common.KafkaException import kafka.common.KafkaStorageException +import kafka.cluster.EndPoint +import kafka.cluster.ProtocolType /** @@ -607,4 +610,9 @@ object Utils extends Logging { .filter{ case (k,l) => (l > 1) } .keys } + + def listenerListToEndPoints(listeners: String): Seq[EndPoint] = { + val listenerList = parseCsvList(listeners) + listenerList.map(listener => EndPoint.createEndPoint(listener)) + } } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index c14bd45..a90fc64 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -17,7 +17,8 @@ package kafka.utils -import kafka.cluster.{Broker, Cluster} +import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster._ import kafka.consumer.{ConsumerThreadId, TopicCount} import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, @@ -83,6 +84,10 @@ object ZkUtils extends Logging { brokerIds.map(_.toInt).map(getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) } + def getAllBrokerEndPointsForChannel(zkClient: ZkClient, protocolType: ProtocolType): Seq[BrokerEndpoint] = { + getAllBrokersInCluster(zkClient).map(_.getBrokerEndPoint(protocolType)) + } + def getLeaderAndIsrForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderAndIsr] = { ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr) } @@ -168,11 +173,15 @@ object ZkUtils extends Logging { } } - def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, timeout: Int, jmxPort: Int) { + def registerBrokerInZk(zkClient: ZkClient, id: Int, advertisedEndpoints: Seq[EndPoint], timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) - val expectedBroker = new Broker(id, host, port) + val defaultEndPoint = advertisedEndpoints.find(_.protocolType == ProtocolType.PLAINTEXT).get + val host = defaultEndPoint.host + val port = defaultEndPoint.port + + val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.mkString(","), "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val expectedBroker = new Broker(id, advertisedEndpoints) try { createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 90c0b7a..6bc5791 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -19,7 +19,6 @@ package kafka.api.test import org.junit.Test import org.junit.Assert._ - import java.lang.Integer import java.util.{Properties, Random} import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} @@ -29,7 +28,9 @@ import kafka.common.Topic import kafka.consumer.SimpleConsumer import kafka.server.KafkaConfig import kafka.integration.KafkaServerTestHarness -import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils} +import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils,Utils} +import kafka.consumer.SimpleConsumer +import kafka.utils.Utils import org.apache.kafka.common.KafkaException import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasException} @@ -74,8 +75,11 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { super.setUp() // TODO: we need to migrate to new consumers when 0.9 is final - consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "") - consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "") + + val endpoint1 = Utils.listenerListToEndPoints(configs(0).listeners).head + val endpoint2 = Utils.listenerListToEndPoints(configs(1).listeners).head + consumer1 = new SimpleConsumer("localhost", endpoint1.port, 100, 1024*1024, "") + consumer2 = new SimpleConsumer("localhost", endpoint2.port, 100, 1024*1024, "") producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize) producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index b15237b..78d8da4 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -25,7 +25,7 @@ import org.junit.Test import org.junit.Assert._ import kafka.server.KafkaConfig -import kafka.utils.{TestZKUtils, TestUtils} +import kafka.utils.{Utils, TestZKUtils, TestUtils} import kafka.consumer.SimpleConsumer import kafka.api.FetchRequestBuilder import kafka.message.Message @@ -55,8 +55,10 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { super.setUp() // TODO: we need to migrate to new consumers when 0.9 is final - consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "") - consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "") + val endpoint1 = Utils.listenerListToEndPoints(configs(0).listeners).head + val endpoint2 = Utils.listenerListToEndPoints(configs(1).listeners).head + consumer1 = new SimpleConsumer("localhost", endpoint1.port, 100, 1024*1024, "") + consumer2 = new SimpleConsumer("localhost", endpoint2.port, 100, 1024*1024, "") } override def tearDown() { diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala index 41f334d..67301ed 100644 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -1,5 +1,6 @@ package other.kafka +import kafka.cluster.ProtocolType import org.I0Itec.zkclient.ZkClient import kafka.api._ import kafka.utils.{ShutdownableThread, ZKStringSerializer} @@ -110,7 +111,7 @@ object TestOffsetManager { private val fetchTimer = new KafkaTimer(timer) private val channels = mutable.Map[Int, BlockingChannel]() - private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs) + private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, ProtocolType.PLAINTEXT, SocketTimeoutMs) private val numErrors = new AtomicInteger(0) @@ -156,7 +157,7 @@ object TestOffsetManager { println("Error while querying %s:%d - shutting down query channel.".format(metadataChannel.host, metadataChannel.port)) metadataChannel.disconnect() println("Creating new query channel.") - metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs) + metadataChannel = ClientUtils.channelToAnyBroker(zkClient, ProtocolType.PLAINTEXT, SocketTimeoutMs) } finally { Thread.sleep(fetchIntervalMs) diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 1bf2667..2f338b7 100644 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -22,7 +22,7 @@ import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} -import kafka.cluster.Broker +import kafka.cluster.{ProtocolType, EndPoint, Broker} import kafka.client.ClientUtils import kafka.server.{KafkaConfig, KafkaServer} @@ -61,7 +61,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { val server4 = TestUtils.createServer(new KafkaConfig(configProps4)) servers ++= List(server1, server2, server3, server4) - brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.config.port)) + brokers = servers.map(s => new Broker(s.config.brokerId, Utils.listenerListToEndPoints(s.config.listeners))) // create topics first createTopic(zkClient, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) @@ -109,7 +109,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 2) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers, "AddPartitionsTest-testIncrementPartitions", + val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)), "AddPartitionsTest-testIncrementPartitions", 2000,0).topicsMetadata val metaDataForTopic1 = metadata.filter(p => p.topic.equals(topic1)) val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata @@ -134,7 +134,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 2) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers, "AddPartitionsTest-testManualAssignmentOfReplicas", + val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)), "AddPartitionsTest-testManualAssignmentOfReplicas", 2000,0).topicsMetadata val metaDataForTopic2 = metadata.filter(p => p.topic.equals(topic2)) val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata @@ -158,7 +158,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 5) TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 6) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers, "AddPartitionsTest-testReplicaPlacement", + val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)), "AddPartitionsTest-testReplicaPlacement", 2000,0).topicsMetadata val metaDataForTopic3 = metadata.filter(p => p.topic.equals(topic3)).head diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index a1f72f8..1287af0 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -22,7 +22,7 @@ import org.scalatest.junit.JUnitSuite import junit.framework.Assert._ import java.nio.ByteBuffer import kafka.message.{Message, ByteBufferMessageSet} -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, ProtocolType, EndPoint, Broker} import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError} import kafka.utils.SystemTime import org.apache.kafka.common.requests._ @@ -80,7 +80,9 @@ object SerializationTestUtils { TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100) ) - private val brokers = List(new Broker(0, "localhost", 1011), new Broker(1, "localhost", 1012), new Broker(2, "localhost", 1013)) + private val brokers = List(new Broker(0, List(EndPoint("localhost", 1011, ProtocolType.PLAINTEXT))), + new Broker(1, List(EndPoint("localhost", 1012, ProtocolType.PLAINTEXT))), + new Broker(2, List(EndPoint("localhost", 1013, ProtocolType.PLAINTEXT)))) private val partitionMetaData0 = new PartitionMetadata(0, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 0) private val partitionMetaData1 = new PartitionMetadata(1, Some(brokers.head), replicas = brokers, isr = brokers.tail, errorCode = 1) private val partitionMetaData2 = new PartitionMetadata(2, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 2) @@ -144,11 +146,11 @@ object SerializationTestUtils { } def createTestTopicMetadataRequest: TopicMetadataRequest = { - new TopicMetadataRequest(1, 1, "client 1", Seq(topic1, topic2)) + new TopicMetadataRequest(1, 1, "client 1", ProtocolType.PLAINTEXT, Seq(topic1, topic2)) } def createTestTopicMetadataResponse: TopicMetadataResponse = { - new TopicMetadataResponse(brokers, Seq(topicmetaData1, topicmetaData2), 1) + new TopicMetadataResponse(List(BrokerEndpoint(1,"localhost",9092,null)), Seq(topicmetaData1, topicmetaData2), 1) } def createTestOffsetCommitRequestV1: OffsetCommitRequest = { @@ -192,7 +194,7 @@ object SerializationTestUtils { } def createConsumerMetadataResponse: ConsumerMetadataResponse = { - ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError, 0) + ConsumerMetadataResponse(Some(brokers.head.getBrokerEndPoint(ProtocolType.PLAINTEXT)), ErrorMapping.NoError, 0) } def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = { diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala new file mode 100644 index 0000000..289d87b --- /dev/null +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -0,0 +1,94 @@ +/* + * 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.cluster + +import java.nio.ByteBuffer + +import kafka.utils.Logging +import org.junit.Test +import org.scalatest.junit.JUnit3Suite + +import scala.collection.mutable + +class BrokerTest extends JUnit3Suite with Logging { + + @Test + def testSerDe() = { + + val endpoint = new EndPoint("myhost",9092,ProtocolType.PLAINTEXT) + val listEndPoints = List(endpoint) + val origBroker = new Broker(1,listEndPoints) + val brokerBytes = ByteBuffer.allocate(origBroker.sizeInBytes) + + origBroker.writeTo(brokerBytes) + + val newBroker = Broker.readFrom(brokerBytes.flip().asInstanceOf[ByteBuffer]) + assert(origBroker == newBroker) + } + + @Test + def testHashAndEquals() = { + val endpoint1 = new EndPoint("myhost",9092,ProtocolType.PLAINTEXT) + val endpoint2 = new EndPoint("myhost",9092,ProtocolType.PLAINTEXT) + val endpoint3 = new EndPoint("myhost",1111,ProtocolType.PLAINTEXT) + val endpoint4 = new EndPoint("other",1111,ProtocolType.PLAINTEXT) + val broker1 = new Broker(1,List(endpoint1)) + val broker2 = new Broker(1,List(endpoint2)) + val broker3 = new Broker(2,List(endpoint3)) + val broker4 = new Broker(1,List(endpoint4)) + + assert(broker1 == broker2) + assert(broker1 != broker3) + assert(broker1 != broker4) + assert(broker1.hashCode() == broker2.hashCode()) + assert(broker1.hashCode() != broker3.hashCode()) + assert(broker1.hashCode() != broker4.hashCode()) + + val hashmap = new mutable.HashMap[Broker,Int]() + hashmap.put(broker1,1) + assert(hashmap.getOrElse(broker1,-1) == 1) + } + + @Test + def testFromJSON() = { + //val brokerInfoStr = "{\"jmx_port\":-1,\"timestamp\":\"1416974968782\",\"host\":\"kafkaf-4.ent.cloudera.com\",\"version\":1,\"port\":9092}" + val brokerInfoStr = "{\"version\":1," + + "\"host\":\"localhost\"," + + "\"port\":9092," + + "\"jmx_port\":9999," + + "\"timestamp\":\"1416974968782\"," + + "\"endpoints\":\"PLAINTEXT://localhost:9092\"}" + val broker = Broker.createBroker(1, brokerInfoStr) + assert(broker.id == 1) + assert(broker.getBrokerEndPoint(ProtocolType.PLAINTEXT).host == "localhost") + assert(broker.getBrokerEndPoint(ProtocolType.PLAINTEXT).port == 9092) + } + + @Test + def endpointFromURI() = { + val connectionString = "PLAINTEXT://localhost:9092" + val endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) + assert(endpoint.host == "localhost") + assert(endpoint.port == 9092) + assert(endpoint.protocolType == ProtocolType.PLAINTEXT) + } + + + + +} diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index c0355cc..ec31b34 100644 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -29,7 +29,7 @@ import kafka.utils.TestUtils._ import kafka.utils._ import org.junit.Test import kafka.serializer._ -import kafka.cluster.{Broker, Cluster} +import kafka.cluster.{ProtocolType, EndPoint, Broker, Cluster} import org.scalatest.junit.JUnit3Suite import kafka.integration.KafkaServerTestHarness @@ -46,7 +46,7 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { val group = "group1" val consumer0 = "consumer0" val consumedOffset = 5 - val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) + val cluster = new Cluster(configs.map(c => new Broker(c.brokerId,Utils.listenerListToEndPoints(c.listeners)))) val queue = new LinkedBlockingQueue[FetchedDataChunk] val topicInfos = configs.map(c => new PartitionTopicInfo(topic, 0, diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 25845ab..ab139ca 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -29,7 +29,7 @@ import kafka.consumer._ import kafka.serializer._ import kafka.producer.{KeyedMessage, Producer} import kafka.utils.TestUtils._ -import kafka.utils.TestUtils +import kafka.utils.{Utils, TestUtils} class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { @@ -39,7 +39,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { yield new KafkaConfig(props) val messages = new mutable.HashMap[Int, Seq[Array[Byte]]] val topic = "topic" - val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) + val cluster = new Cluster(configs.map(c => new Broker(c.brokerId,Utils.listenerListToEndPoints(c.listeners)))) val shutdown = ZookeeperConsumerConnector.shutdownCommand val queue = new LinkedBlockingQueue[FetchedDataChunk] val topicInfos = configs.map(c => new PartitionTopicInfo(topic, diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 35dc071..ceb39fa 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -22,8 +22,8 @@ import kafka.zk.ZooKeeperTestHarness import kafka.admin.AdminUtils import java.nio.ByteBuffer import junit.framework.Assert._ -import kafka.cluster.Broker -import kafka.utils.TestUtils +import kafka.cluster.{ProtocolType, EndPoint, Broker} +import kafka.utils.{Utils, TestUtils} import kafka.utils.TestUtils._ import kafka.server.{KafkaServer, KafkaConfig} import kafka.api.TopicMetadataRequest @@ -34,7 +34,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { val props = createBrokerConfigs(1) val configs = props.map(p => new KafkaConfig(p)) private var server1: KafkaServer = null - val brokers = configs.map(c => new Broker(c.brokerId,c.hostName,c.port)) + val brokerEndPoints = configs.map(c => new Broker(c.brokerId,Utils.listenerListToEndPoints(c.listeners)).getBrokerEndPoint(ProtocolType.PLAINTEXT)) override def setUp() { super.setUp() @@ -67,7 +67,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { val topic = "test" createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokerEndPoints,"TopicMetadataTest-testBasicTopicMetadata", 2000,0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) @@ -87,7 +87,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { createTopic(zkClient, topic2, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) // issue metadata request with empty list of topics - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokers, "TopicMetadataTest-testGetAllTopicMetadata", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokerEndPoints, "TopicMetadataTest-testGetAllTopicMetadata", 2000, 0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(2, topicsMetadata.size) @@ -106,7 +106,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { def testAutoCreateTopic { // auto create topic val topic = "testAutoCreateTopic" - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testAutoCreateTopic", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokerEndPoints,"TopicMetadataTest-testAutoCreateTopic", 2000,0).topicsMetadata assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode) assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) @@ -118,7 +118,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0) // retry the metadata for the auto created topic - topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", + topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokerEndPoints,"TopicMetadataTest-testBasicTopicMetadata", 2000,0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 78b431f..ab8dd77 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -19,6 +19,7 @@ package kafka.network; import java.net._ import java.io._ +import kafka.cluster.{ProtocolType, EndPoint} import org.junit._ import org.scalatest.junit.JUnitSuite import java.util.Random @@ -35,8 +36,7 @@ import scala.collection.Map class SocketServerTest extends JUnitSuite { val server: SocketServer = new SocketServer(0, - host = null, - port = kafka.utils.TestUtils.choosePort, + List(EndPoint(null,kafka.utils.TestUtils.choosePort,ProtocolType.PLAINTEXT)), numProcessorThreads = 1, maxQueuedRequests = 50, sendBufferSize = 300000, @@ -73,7 +73,7 @@ class SocketServerTest extends JUnitSuite { channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) } - def connect(s:SocketServer = server) = new Socket("localhost", s.port) + def connect(s:SocketServer = server) = new Socket("localhost", server.endpoints.find(_.protocolType == ProtocolType.PLAINTEXT).get.port) @After def cleanup() { diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index d60d8e0..708fa99 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -21,6 +21,7 @@ import java.net.SocketTimeoutException import java.util.Properties import junit.framework.Assert import kafka.admin.AdminUtils +import kafka.cluster.ProtocolType import kafka.integration.KafkaServerTestHarness import kafka.message._ import kafka.server.KafkaConfig @@ -39,7 +40,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testReachableServer() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) val firstStart = SystemTime.milliseconds @@ -74,7 +76,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testEmptyProduceRequest() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 val clientId = SyncProducerConfig.DefaultClientId @@ -91,7 +94,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLarge() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) TestUtils.createTopic(zkClient, "test", numPartitions = 1, replicationFactor = 1, servers = servers) @@ -118,8 +122,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLargeWithAckZero() { val server = servers.head + val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) props.put("request.required.acks", "0") val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -145,7 +150,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceCorrectlyReceivesResponse() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) @@ -191,7 +197,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { val timeoutMs = 500 val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) @@ -217,7 +224,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceRequestWithNoResponse() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 val clientId = SyncProducerConfig.DefaultClientId val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs @@ -232,8 +240,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { def testNotEnoughReplicas() { val topicName = "minisrtest" val server = servers.head + val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) props.put("request.required.acks", "-1") val producer = new SyncProducer(new SyncProducerConfig(props)) diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala index f0c4a56..49e3933 100644 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -17,6 +17,7 @@ package kafka.server +import kafka.cluster.ProtocolType import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import junit.framework.Assert._ @@ -31,9 +32,8 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) - props.put("advertised.host.name", advertisedHostName) - props.put("advertised.port", advertisedPort.toString) - + props.put("advertised.listeners",ProtocolType.PLAINTEXT.toString+"://"+advertisedHostName+":"+advertisedPort.toString) + server = TestUtils.createServer(new KafkaConfig(props)) } @@ -45,8 +45,9 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { def testBrokerAdvertiseToZK { val brokerInfo = ZkUtils.getBrokerInfo(zkClient, brokerId) - assertEquals(advertisedHostName, brokerInfo.get.host) - assertEquals(advertisedPort, brokerInfo.get.port) + val endpoint = brokerInfo.get.endPoints.find(_.protocolType == ProtocolType.PLAINTEXT).get + assertEquals(advertisedHostName, endpoint.host) + assertEquals(advertisedPort, endpoint.port) } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 82dce80..77bd378 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -17,12 +17,13 @@ package kafka.server +import kafka.cluster.ProtocolType import org.junit.Test import junit.framework.Assert._ import org.scalatest.junit.JUnit3Suite -import kafka.utils.TestUtils import kafka.message.GZIPCompressionCodec import kafka.message.NoCompressionCodec +import kafka.utils.{Utils, TestUtils} class KafkaConfigTest extends JUnit3Suite { @@ -93,12 +94,13 @@ class KafkaConfigTest extends JUnit3Suite { val hostName = "fake-host" val props = TestUtils.createBrokerConfig(0, port) - props.put("host.name", hostName) + props.put("listeners", "PLAINTEXT://"+hostName+":"+port) val serverConfig = new KafkaConfig(props) - - assertEquals(serverConfig.advertisedHostName, hostName) - assertEquals(serverConfig.advertisedPort, port) + val endpoints = Utils.listenerListToEndPoints(serverConfig.advertisedListeners) + val endpoint = endpoints.find(_.protocolType == ProtocolType.PLAINTEXT).get + assertEquals(endpoint.host, hostName) + assertEquals(endpoint.port, port) } @Test @@ -108,13 +110,14 @@ class KafkaConfigTest extends JUnit3Suite { val advertisedPort = 1234 val props = TestUtils.createBrokerConfig(0, port) - props.put("advertised.host.name", advertisedHostName) - props.put("advertised.port", advertisedPort.toString) + props.put("advertised.listeners", "PLAINTEXT://"+advertisedHostName+":"+advertisedPort.toString) val serverConfig = new KafkaConfig(props) + val endpoints = Utils.listenerListToEndPoints(serverConfig.advertisedListeners) + val endpoint = endpoints.find(_.protocolType == ProtocolType.PLAINTEXT).get - assertEquals(serverConfig.advertisedHostName, advertisedHostName) - assertEquals(serverConfig.advertisedPort, advertisedPort) + assertEquals(endpoint.host, advertisedHostName) + assertEquals(endpoint.port, advertisedPort) } @Test diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index c2ba07c..bc6bdd6 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -23,7 +23,7 @@ import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} -import kafka.cluster.Broker +import kafka.cluster.{ProtocolType, Broker} import kafka.common.ErrorMapping import kafka.api._ @@ -118,7 +118,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // start another controller val controllerId = 2 val controllerConfig = new KafkaConfig(TestUtils.createBrokerConfig(controllerId, TestUtils.choosePort())) - val brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.config.port)) + val endpoints = Utils.listenerListToEndPoints(controllerConfig.listeners) + val brokers = servers.map(s => new Broker(s.config.brokerId, endpoints)) val controllerContext = new ControllerContext(zkClient, 6000) controllerContext.liveBrokers = brokers.toSet val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index c06ee75..0476a59 100644 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -197,7 +197,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { private def createBrokerConfig(nodeId: Int, port: Int): Properties = { val props = new Properties props.put("broker.id", nodeId.toString) - props.put("port", port.toString) + props.put("listeners", "PLAINTEXT://localhost:" + port.toString) props.put("log.dir", getLogDir.getAbsolutePath) props.put("log.flush.interval.messages", "1") props.put("enable.zookeeper", "false") diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 54755e8..56a362b 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -24,6 +24,7 @@ import java.nio.channels._ import java.util.Random import java.util.Properties +import kafka.utils import org.apache.kafka.common.utils.Utils._ import collection.mutable.ListBuffer @@ -34,7 +35,7 @@ import kafka.server._ import kafka.producer._ import kafka.message._ import kafka.api._ -import kafka.cluster.Broker +import kafka.cluster.{ProtocolType, Broker} import kafka.consumer.{KafkaStream, ConsumerConfig} import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} import kafka.common.TopicAndPartition @@ -146,7 +147,10 @@ object TestUtils extends Logging { } def getBrokerListStrFromConfigs(configs: Seq[KafkaConfig]): String = { - configs.map(c => formatAddress(c.hostName, c.port)).mkString(",") + configs.map(c => { + val endpoint = Utils.listenerListToEndPoints(c.listeners).find(_.protocolType == ProtocolType.PLAINTEXT).get + formatAddress(endpoint.host, endpoint.port) + }).mkString(",") } /** @@ -156,8 +160,7 @@ object TestUtils extends Logging { enableControlledShutdown: Boolean = true): Properties = { val props = new Properties if (nodeId >= 0) props.put("broker.id", nodeId.toString) - props.put("host.name", "localhost") - props.put("port", port.toString) + props.put("listeners","PLAINTEXT://localhost:"+port.toString) props.put("log.dir", TestUtils.tempDir().getAbsolutePath) props.put("zookeeper.connect", TestZKUtils.zookeeperConnect) props.put("replica.socket.timeout.ms", "1500") @@ -447,7 +450,7 @@ object TestUtils extends Logging { def createBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { val brokers = ids.map(id => new Broker(id, "localhost", 6667)) - brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.host, b.port, 6000, jmxPort = -1)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.endPoints, 6000, jmxPort = -1)) brokers } -- 1.9.3 (Apple Git-50) From a35a92b67ef4990f5b7542ec4d3836dda4fcd395 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Fri, 5 Dec 2014 12:21:09 -0800 Subject: [PATCH 13/41] changed topicmetadata to include brokerendpoints and fixed few unit tests --- core/src/main/scala/kafka/admin/AdminUtils.scala | 14 +++++++------- .../scala/kafka/api/ConsumerMetadataResponse.scala | 2 +- core/src/main/scala/kafka/api/TopicMetadata.scala | 22 +++++----------------- .../scala/kafka/api/TopicMetadataResponse.scala | 6 +----- core/src/main/scala/kafka/cluster/Broker.scala | 6 +----- .../main/scala/kafka/cluster/BrokerEndPoint.scala | 20 ++++---------------- .../kafka/consumer/ConsumerFetcherManager.scala | 6 +++--- .../main/scala/kafka/javaapi/TopicMetadata.scala | 6 +++--- .../main/scala/kafka/producer/ProducerPool.scala | 4 ++-- .../kafka/server/AbstractFetcherManager.scala | 6 +++--- core/src/main/scala/kafka/server/KafkaApis.scala | 2 +- .../main/scala/kafka/server/MetadataCache.scala | 22 ++++++++++++++-------- .../scala/kafka/server/ReplicaFetcherManager.scala | 4 ++-- .../scala/kafka/server/ReplicaFetcherThread.scala | 6 +++--- .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../main/scala/kafka/tools/GetOffsetShell.scala | 4 ++-- .../scala/kafka/tools/SimpleConsumerShell.scala | 18 +++++++++--------- core/src/test/resources/log4j.properties | 4 ++-- .../kafka/api/ProducerFailureHandlingTest.scala | 3 +-- .../api/RequestResponseSerializationTest.scala | 12 +++++++----- .../test/scala/unit/kafka/cluster/BrokerTest.scala | 1 - .../unit/kafka/producer/AsyncProducerTest.scala | 8 ++++---- .../unit/kafka/server/LeaderElectionTest.scala | 3 +-- 23 files changed, 77 insertions(+), 104 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index d06775c..7e57b8c 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -294,7 +294,7 @@ object AdminUtils extends Logging { - private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker]): TopicMetadata = { + private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker], protocol: ProtocolType = ProtocolType.PLAINTEXT): TopicMetadata = { if(ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) { val topicPartitionAssignment = ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic).get val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) @@ -305,22 +305,22 @@ object AdminUtils extends Logging { val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition) debug("replicas = " + replicas + ", in sync replicas = " + inSyncReplicas + ", leader = " + leader) - var leaderInfo: Option[Broker] = None - var replicaInfo: Seq[Broker] = Nil - var isrInfo: Seq[Broker] = Nil + var leaderInfo: Option[BrokerEndpoint] = None + var replicaInfo: Seq[BrokerEndpoint] = Nil + var isrInfo: Seq[BrokerEndpoint] = Nil try { leaderInfo = leader match { case Some(l) => try { - Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head) + Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head.getBrokerEndPoint(protocol)) } catch { case e: Throwable => throw new LeaderNotAvailableException("Leader not available for partition [%s,%d]".format(topic, partition), e) } case None => throw new LeaderNotAvailableException("No leader exists for partition " + partition) } try { - replicaInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, replicas.map(id => id.toInt)) - isrInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, inSyncReplicas) + replicaInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, replicas.map(id => id.toInt)).map(_.getBrokerEndPoint(protocol)) + isrInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, inSyncReplicas).map(_.getBrokerEndPoint(protocol)) } catch { case e: Throwable => throw new ReplicaNotAvailableException(e) } diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala index b116d62..0f94452 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -24,7 +24,7 @@ import kafka.common.ErrorMapping object ConsumerMetadataResponse { val CurrentVersion = 0 - private val NoBrokerOpt = Some(BrokerEndpoint(id = -1, host = "", port = -1, protocolType = ProtocolType.PLAINTEXT)) + private val NoBrokerOpt = Some(BrokerEndpoint(id = -1, host = "", port = -1)) def readFrom(buffer: ByteBuffer) = { val correlationId = buffer.getInt diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index b1ff35e..9aa9119 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -28,7 +28,7 @@ object TopicMetadata { val NoLeaderNodeId = -1 - def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): TopicMetadata = { + def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndpoint]): TopicMetadata = { val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue)) val topic = readShortString(buffer) val numPartitions = readIntInRange(buffer, "number of partitions", (0, Int.MaxValue)) @@ -89,7 +89,7 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat object PartitionMetadata { - def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): PartitionMetadata = { + def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndpoint]): PartitionMetadata = { val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue)) val partitionId = readIntInRange(buffer, "partition id", (0, Int.MaxValue)) /* partition id */ val leaderId = buffer.getInt @@ -110,23 +110,11 @@ object PartitionMetadata { } case class PartitionMetadata(partitionId: Int, - val leader: Option[Broker], - replicas: Seq[Broker], - isr: Seq[Broker] = Seq.empty, + val leader: Option[BrokerEndpoint], + replicas: Seq[BrokerEndpoint], + isr: Seq[BrokerEndpoint] = Seq.empty, errorCode: Short = ErrorMapping.NoError) extends Logging { - /*def this(partitionId: Int, - leader: Option[BrokerEndPoint], - replicas: Seq[BrokerEndPoint], - isr: Seq[BrokerEndPoint] = Seq.empty, - errorCode: Short = ErrorMapping.NoError) = { - this(partitionId, - Some(if (leader.isDefined) Broker.createBroker(leader.get) else Broker.noBroker()), - replicas.map(endpoint=>Broker.createBroker(endpoint)), - isr.map(endpoint=>Broker.createBroker(endpoint)), - errorCode) - }*/ - def sizeInBytes: Int = { 2 /* error code */ + 4 /* partition id */ + diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala index db9f88e..70450be 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala @@ -26,11 +26,7 @@ object TopicMetadataResponse { val correlationId = buffer.getInt val brokerCount = buffer.getInt val brokers = (0 until brokerCount).map(_ => BrokerEndpoint.readFrom(buffer)) - /* The broker list we are using for the TopicMetadataResponse is a collection of end-points (so clients can use them for connections) - * But partitionMetadata has multiple use-cases, some of them seem to require actual brokers - * So converting endpoints to broker objects here to preserve the partitionMetadata as is. - * TODO: check if we can use endpoints in partitionMetadata */ - val brokerMap = brokers.map(b => (b.id, Broker.createBroker(b))).toMap + val brokerMap = brokers.map(b => (b.id, b)).toMap val topicCount = buffer.getInt val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap)) new TopicMetadataResponse(brokers, topicsMetadata, correlationId) diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 684bc32..a84a34e 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -67,10 +67,6 @@ object Broker { } } - def createBroker(endPoint: BrokerEndpoint) = { - Broker(endPoint.id,List(EndPoint(endPoint.host, endPoint.port,endPoint.protocolType))) - } - def noBroker() = { Broker(-1,List()) } @@ -121,7 +117,7 @@ case class Broker(id: Int, endPoints: Seq[EndPoint]) { def getBrokerEndPoint(protocolType: ProtocolType): BrokerEndpoint = { val endpoint = endPoints.map((endpoint)=>(endpoint.protocolType,endpoint)).toMap.get(protocolType) endpoint match { - case Some(endpoint) => new BrokerEndpoint(id,endpoint.host,endpoint.port,endpoint.protocolType) + case Some(endpoint) => new BrokerEndpoint(id,endpoint.host,endpoint.port) case None => throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id)) } diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala index bf6f406..90ea612 100644 --- a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -9,7 +9,7 @@ import kafka.cluster.ProtocolType._ object BrokerEndpoint { def createBrokerEndPoint(brokerId: Int, connectionString: String): BrokerEndpoint = { val endPoint = EndPoint.createEndPoint(connectionString) - new BrokerEndpoint(brokerId,endPoint.host,endPoint.port.toInt,endPoint.protocolType) + new BrokerEndpoint(brokerId,endPoint.host,endPoint.port.toInt) } /** @@ -23,28 +23,16 @@ object BrokerEndpoint { val brokerId = buffer.getInt() val host = readShortString(buffer) val port = buffer.getInt() - BrokerEndpoint(brokerId,host,port,null) + BrokerEndpoint(brokerId,host,port) } } // Utility class, representing a particular method of connecting to a broker // Mostly to be used by clients // This is not a broker and is not stored in ZooKeeper -case class BrokerEndpoint(id: Int, host: String, port: Int, protocolType: ProtocolType) { +case class BrokerEndpoint(id: Int, host: String, port: Int) { - def connectionString(): String = formatEndpoint(host,port,protocolType) - - /** - * Formats broker endpoint as "channel://host:port" address string, - * surrounding IPv6 addresses with braces '[', ']' - * @param host hostname - * @param port port number - * @param protocolType String representing channel type (plain, SSL, KRB) - * @return address string - */ - def formatEndpoint(host: String, port: Integer, protocolType: ProtocolType): String = { - return protocolType + "://" + formatAddress(host, port) - } + def connectionString(): String = formatAddress(host,port) def writeTo(buffer: ByteBuffer): Unit = { buffer.putInt(id) diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index c6700d2..8c7accc 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -53,7 +53,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, private class LeaderFinderThread(name: String) extends ShutdownableThread(name) { // thread responsible for adding the fetcher to the right broker when leader is available override def doWork() { - val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker] + val leaderForPartitionsMap = new HashMap[TopicAndPartition, BrokerEndpoint] lock.lock() try { while (noLeaderPartitionSet.isEmpty) { @@ -114,10 +114,10 @@ class ConsumerFetcherManager(private val consumerIdString: String, } } - override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { + override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndpoint): AbstractFetcherThread = { new ConsumerFetcherThread( "ConsumerFetcherThread-%s-%d-%d".format(consumerIdString, fetcherId, sourceBroker.id), - config, sourceBroker.getBrokerEndPoint(config.securityProtocol), partitionMap, this) + config, sourceBroker, partitionMap, this) } def startConnections(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster) { diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala index 255a9f7..24defbc 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala @@ -52,17 +52,17 @@ class TopicMetadata(private val underlying: kafka.api.TopicMetadata) { class PartitionMetadata(private val underlying: kafka.api.PartitionMetadata) { def partitionId: Int = underlying.partitionId - def leader: Broker = { + def leader: BrokerEndpoint = { import kafka.javaapi.Implicits._ underlying.leader } - def replicas: java.util.List[Broker] = { + def replicas: java.util.List[BrokerEndpoint] = { import JavaConversions._ underlying.replicas } - def isr: java.util.List[Broker] = { + def isr: java.util.List[BrokerEndpoint] = { import JavaConversions._ underlying.isr } diff --git a/core/src/main/scala/kafka/producer/ProducerPool.scala b/core/src/main/scala/kafka/producer/ProducerPool.scala index 72686fc..1ea8752 100644 --- a/core/src/main/scala/kafka/producer/ProducerPool.scala +++ b/core/src/main/scala/kafka/producer/ProducerPool.scala @@ -48,8 +48,8 @@ class ProducerPool(val config: ProducerConfig) extends Logging { topicMetadata.foreach(tmd => { tmd.partitionsMetadata.foreach(pmd => { if(pmd.leader.isDefined) { - val endpoint = pmd.leader.get.endPoints.head - newBrokers += BrokerEndpoint(pmd.leader.get.id, endpoint.host,endpoint.port,config.securityProtocol) + val endpoint = pmd.leader.get + newBrokers += BrokerEndpoint(pmd.leader.get.id, endpoint.host,endpoint.port) } }) }) diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 36f054e..1a5b0f9 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -68,7 +68,7 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri } // to be defined in subclass to create a specific fetcher - def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread + def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndpoint): AbstractFetcherThread def addFetcherForPartitions(partitionAndOffsets: Map[TopicAndPartition, BrokerAndInitialOffset]) { mapLock synchronized { @@ -126,6 +126,6 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri } } -case class BrokerAndFetcherId(broker: Broker, fetcherId: Int) +case class BrokerAndFetcherId(broker: BrokerEndpoint, fetcherId: Int) -case class BrokerAndInitialOffset(broker: Broker, initOffset: Long) \ No newline at end of file +case class BrokerAndInitialOffset(broker: BrokerEndpoint, initOffset: Long) \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index f61d972..83bc825 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -447,7 +447,7 @@ class KafkaApis(val requestChannel: RequestChannel, val response = offsetsTopicMetadata.partitionsMetadata.find(_.partitionId == partition).map { partitionMetadata => partitionMetadata.leader.map { leader => - ConsumerMetadataResponse(Some(leader.getBrokerEndPoint(securityProtocol)), ErrorMapping.NoError, consumerMetadataRequest.correlationId) + ConsumerMetadataResponse(Some(leader), ErrorMapping.NoError, consumerMetadataRequest.correlationId) }.getOrElse(errorResponse) }.getOrElse(errorResponse) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index bf81a1a..3df176c 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -17,9 +17,12 @@ package kafka.server +import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.ProtocolType.ProtocolType + import scala.collection.{Seq, Set, mutable} import kafka.api._ -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, ProtocolType, Broker} import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.utils.Utils._ import kafka.common.{ErrorMapping, ReplicaNotAvailableException, LeaderNotAvailableException} @@ -36,7 +39,7 @@ private[server] class MetadataCache { private var aliveBrokers: Map[Int, Broker] = Map() private val partitionMetadataLock = new ReentrantReadWriteLock() - def getTopicMetadata(topics: Set[String]) = { + def getTopicMetadata(topics: Set[String], protocol: ProtocolType = ProtocolType.PLAINTEXT) = { val isAllTopics = topics.isEmpty val topicsRequested = if(isAllTopics) cache.keySet else topics val topicResponses: mutable.ListBuffer[TopicMetadata] = new mutable.ListBuffer[TopicMetadata] @@ -47,18 +50,21 @@ private[server] class MetadataCache { val partitionMetadata = partitionStateInfos.map { case (partitionId, partitionState) => val replicas = partitionState.allReplicas - val replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq - var leaderInfo: Option[Broker] = None - var isrInfo: Seq[Broker] = Nil + val replicaInfo: Seq[BrokerEndpoint] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq.map(_.getBrokerEndPoint(protocol)) + var leaderInfo: Option[BrokerEndpoint] = None + var leaderBrokerInfo: Option[Broker] = None + var isrInfo: Seq[BrokerEndpoint] = Nil val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch val leader = leaderIsrAndEpoch.leaderAndIsr.leader val isr = leaderIsrAndEpoch.leaderAndIsr.isr val topicPartition = TopicAndPartition(topic, partitionId) try { - leaderInfo = aliveBrokers.get(leader) - if (!leaderInfo.isDefined) + leaderBrokerInfo = aliveBrokers.get(leader) + if (!leaderBrokerInfo.isDefined) throw new LeaderNotAvailableException("Leader not available for %s".format(topicPartition)) - isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null) + else + leaderInfo = Some(leaderBrokerInfo.get.getBrokerEndPoint(protocol)) + isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).map(_.getBrokerEndPoint(protocol)) if (replicaInfo.size < replicas.size) throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala index 351dbba..4d0f998 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala @@ -17,13 +17,13 @@ package kafka.server -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val replicaMgr: ReplicaManager) extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId, "Replica", brokerConfig.numReplicaFetchers) { - override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { + override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndpoint): AbstractFetcherThread = { new ReplicaFetcherThread("ReplicaFetcherThread-%d-%d".format(fetcherId, sourceBroker.id), sourceBroker, brokerConfig, replicaMgr) } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 5cf4cfd..b48e65e 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -18,19 +18,19 @@ package kafka.server import kafka.admin.AdminUtils -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import kafka.log.LogConfig import kafka.message.ByteBufferMessageSet import kafka.api.{OffsetRequest, FetchResponsePartitionData} import kafka.common.{KafkaStorageException, TopicAndPartition} class ReplicaFetcherThread(name:String, - sourceBroker: Broker, + sourceBroker: BrokerEndpoint, brokerConfig: KafkaConfig, replicaMgr: ReplicaManager) extends AbstractFetcherThread(name = name, clientId = name, - sourceBroker = sourceBroker.getBrokerEndPoint(brokerConfig.securityProtocol), + sourceBroker = sourceBroker, socketTimeout = brokerConfig.replicaSocketTimeoutMs, socketBufferSize = brokerConfig.replicaSocketReceiveBufferBytes, fetchSize = brokerConfig.replicaFetchMaxBytes, diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index fb948b9..e90b5dc 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -678,7 +678,7 @@ class ReplicaManager(val config: KafkaConfig, // we do not need to check if the leader exists again since this has been done at the beginning of this process val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map(partition => new TopicAndPartition(partition) -> BrokerAndInitialOffset( - leaders.find(_.id == partition.leaderReplicaIdOpt.get).get, + leaders.find(_.id == partition.leaderReplicaIdOpt.get).get.getBrokerEndPoint(config.securityProtocol), partition.getReplica().get.logEndOffset.messageOffset)).toMap replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset) diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 46078b7..1596516 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -94,8 +94,8 @@ object GetOffsetShell { case Some(metadata) => metadata.leader match { case Some(leader) => - val consumer = new SimpleConsumer(leader.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, - leader.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, 10000, 100000, clientId) + val consumer = new SimpleConsumer(leader.host, + leader.port, 10000, 100000, clientId) val topicAndPartition = TopicAndPartition(topic, partitionId) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index 77c4900..ac86762 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -22,7 +22,7 @@ import kafka.utils._ import kafka.consumer._ import kafka.client.ClientUtils import kafka.api.{OffsetRequest, FetchRequestBuilder, Request} -import kafka.cluster.{ProtocolType, Broker} +import kafka.cluster.{BrokerEndpoint, ProtocolType, Broker} import scala.collection.JavaConversions._ import kafka.common.TopicAndPartition @@ -142,8 +142,8 @@ object SimpleConsumerShell extends Logging { } // validating replica id and initializing target broker - var fetchTargetBroker: Broker = null - var replicaOpt: Option[Broker] = null + var fetchTargetBroker: BrokerEndpoint = null + var replicaOpt: Option[BrokerEndpoint] = null if(replicaId == UseLeaderReplica) { replicaOpt = partitionMetadataOpt.get.leader if(!replicaOpt.isDefined) { @@ -167,8 +167,8 @@ object SimpleConsumerShell extends Logging { System.exit(1) } if (startingOffset < 0) { - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, - fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, + fetchTargetBroker.port, ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize, clientId) try { @@ -191,10 +191,10 @@ object SimpleConsumerShell extends Logging { val replicaString = if(replicaId > 0) "leader" else "replica" info("Starting simple consumer shell to partition [%s, %d], %s [%d], host and port: [%s, %d], from offset [%d]" .format(topic, partitionId, replicaString, replicaId, - fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, - fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, startingOffset)) - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, - fetchTargetBroker.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, + fetchTargetBroker.host, + fetchTargetBroker.port, startingOffset)) + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, + fetchTargetBroker.port, 10000, 64*1024, clientId) val thread = Utils.newThread("kafka-simpleconsumer-shell", new Runnable() { def run() { diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 1b7d5d8..1883f97 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -18,8 +18,8 @@ log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=ERROR -log4j.logger.org.apache.kafka=ERROR +log4j.logger.kafka=TRACE +log4j.logger.org.apache.kafka=TRACE # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 6bc5791..50601ee 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -277,8 +277,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { server.shutdown() server.awaitShutdown() server.startup - - Thread.sleep(2000) + Thread.sleep(5000) } // Make sure the producer do not see any exception diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 1287af0..2d19f65 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -83,10 +83,12 @@ object SerializationTestUtils { private val brokers = List(new Broker(0, List(EndPoint("localhost", 1011, ProtocolType.PLAINTEXT))), new Broker(1, List(EndPoint("localhost", 1012, ProtocolType.PLAINTEXT))), new Broker(2, List(EndPoint("localhost", 1013, ProtocolType.PLAINTEXT)))) - private val partitionMetaData0 = new PartitionMetadata(0, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 0) - private val partitionMetaData1 = new PartitionMetadata(1, Some(brokers.head), replicas = brokers, isr = brokers.tail, errorCode = 1) - private val partitionMetaData2 = new PartitionMetadata(2, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 2) - private val partitionMetaData3 = new PartitionMetadata(3, Some(brokers.head), replicas = brokers, isr = brokers.tail.tail, errorCode = 3) + private val brokerEndpoints = brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)) + + private val partitionMetaData0 = new PartitionMetadata(0, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints, errorCode = 0) + private val partitionMetaData1 = new PartitionMetadata(1, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints.tail, errorCode = 1) + private val partitionMetaData2 = new PartitionMetadata(2, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints, errorCode = 2) + private val partitionMetaData3 = new PartitionMetadata(3, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints.tail.tail, errorCode = 3) private val partitionMetaDataSeq = Seq(partitionMetaData0, partitionMetaData1, partitionMetaData2, partitionMetaData3) private val topicmetaData1 = new TopicMetadata(topic1, partitionMetaDataSeq) private val topicmetaData2 = new TopicMetadata(topic2, partitionMetaDataSeq) @@ -150,7 +152,7 @@ object SerializationTestUtils { } def createTestTopicMetadataResponse: TopicMetadataResponse = { - new TopicMetadataResponse(List(BrokerEndpoint(1,"localhost",9092,null)), Seq(topicmetaData1, topicmetaData2), 1) + new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)).toVector, Seq(topicmetaData1, topicmetaData2), 1) } def createTestOffsetCommitRequestV1: OffsetCommitRequest = { diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index 289d87b..03c6522 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -85,7 +85,6 @@ class BrokerTest extends JUnit3Suite with Logging { val endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) assert(endpoint.host == "localhost") assert(endpoint.port == 9092) - assert(endpoint.protocolType == ProtocolType.PLAINTEXT) } diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index 1db6ac3..450aa59 100644 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -23,7 +23,7 @@ import junit.framework.Assert._ import org.easymock.EasyMock import org.junit.Test import kafka.api._ -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import kafka.common._ import kafka.message._ import kafka.producer.async._ @@ -163,8 +163,8 @@ class AsyncProducerTest extends JUnit3Suite { val props = new Properties() props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) - val broker1 = new Broker(0, "localhost", 9092) - val broker2 = new Broker(1, "localhost", 9093) + val broker1 = new BrokerEndpoint(0, "localhost", 9092) + val broker2 = new BrokerEndpoint(1, "localhost", 9093) // form expected partitions metadata val partition1Metadata = new PartitionMetadata(0, Some(broker1), List(broker1, broker2)) @@ -467,7 +467,7 @@ class AsyncProducerTest extends JUnit3Suite { } private def getTopicMetadata(topic: String, partition: Seq[Int], brokerId: Int, brokerHost: String, brokerPort: Int): TopicMetadata = { - val broker1 = new Broker(brokerId, brokerHost, brokerPort) + val broker1 = new BrokerEndpoint(brokerId, brokerHost, brokerPort) new TopicMetadata(topic, partition.map(new PartitionMetadata(_, Some(broker1), List(broker1)))) } diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index bc6bdd6..79dff89 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -118,8 +118,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // start another controller val controllerId = 2 val controllerConfig = new KafkaConfig(TestUtils.createBrokerConfig(controllerId, TestUtils.choosePort())) - val endpoints = Utils.listenerListToEndPoints(controllerConfig.listeners) - val brokers = servers.map(s => new Broker(s.config.brokerId, endpoints)) + val brokers = servers.map(s => new Broker(s.config.brokerId, Utils.listenerListToEndPoints(s.config.listeners))) val controllerContext = new ControllerContext(zkClient, 6000) controllerContext.liveBrokers = brokers.toSet val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) -- 1.9.3 (Apple Git-50) From ddceab2c3c706a671f92bdae53b58574b9393f6f Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Sat, 6 Dec 2014 11:28:06 -0800 Subject: [PATCH 14/41] fixing systest and support for binding to default address --- config/server.properties | 4 +++- core/src/main/scala/kafka/cluster/EndPoint.scala | 3 ++- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- core/src/test/scala/unit/kafka/KafkaConfigTest.scala | 8 ++++---- core/src/test/scala/unit/kafka/cluster/BrokerTest.scala | 12 +++++++++--- system_test/utils/kafka_system_test_utils.py | 1 + 6 files changed, 20 insertions(+), 10 deletions(-) diff --git a/config/server.properties b/config/server.properties index 1614260..80ee2fc 100644 --- a/config/server.properties +++ b/config/server.properties @@ -21,8 +21,10 @@ broker.id=0 ############################# Socket Server Settings ############################# +listeners=PLAINTEXT://:9092 + # The port the socket server listens on -port=9092 +#port=9092 # Hostname the broker will bind to. If not set, the server will bind to all interfaces #host.name=localhost diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index 085b333..1b5fe8f 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -33,8 +33,9 @@ object EndPoint { } def createEndPoint(connectionString: String): EndPoint = { - val uriParseExp = """^(.*)://([0-9a-z\-.]+):([0-9]+)""".r + val uriParseExp = """^(.*)://([0-9a-z\-.]*):([0-9]+)""".r connectionString match { + case uriParseExp(protocol,"",port) => new EndPoint(null,port.toInt,ProtocolType.withName(protocol)) case uriParseExp(protocol,host,port) => new EndPoint(host,port.toInt,ProtocolType.withName(protocol)) case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 39a67a3..a20fd42 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -103,7 +103,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* Listener List - Comma-separated list of URIs we will listen on and their protocols. * Specify hostname as 0.0.0.0 to bind to all interfaces * Leave hostname empty to bind to default interface */ - val listeners: String = props.getString("listeners", "PLAINTEXT://0.0.0.0:6667") + val listeners: String = props.getString("listeners", "PLAINTEXT://:6667") /* Listeners to publish to ZooKeeper for clients to use, if different than the listeners above. * In IaaS environments, this may need to be different from the interface to which the broker binds. diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 4d36b8b..bc4aef3 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -65,14 +65,14 @@ class KafkaTest { assertEquals(2, config2.brokerId) // We should be also able to set completely new property - val config3 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "port=1987")) + val config3 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "log.cleanup.policy=compact")) assertEquals(1, config3.brokerId) - assertEquals(1987, config3.port) + assertEquals("compact", config3.logCleanupPolicy) // We should be also able to set several properties - val config4 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "port=1987", "--override", "broker.id=2")) + val config4 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "log.cleanup.policy=compact", "--override", "broker.id=2")) assertEquals(2, config4.brokerId) - assertEquals(1987, config4.port) + assertEquals("compact", config4.logCleanupPolicy) } @Test(expected = classOf[ExitCalled]) diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index 03c6522..7d59037 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -80,14 +80,20 @@ class BrokerTest extends JUnit3Suite with Logging { } @Test - def endpointFromURI() = { - val connectionString = "PLAINTEXT://localhost:9092" - val endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) + def testEndpointFromURI() = { + var connectionString = "PLAINTEXT://localhost:9092" + var endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) assert(endpoint.host == "localhost") assert(endpoint.port == 9092) + // also test for default bind + connectionString = "PLAINTEXT://:9092" + endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) + assert(endpoint.host == null) + assert(endpoint.port == 9092) } + } diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index 41d511c..e4b5d80 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -436,6 +436,7 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv addedCSVConfig["kafka.metrics.polling.interval.secs"] = "5" addedCSVConfig["kafka.metrics.reporters"] = "kafka.metrics.KafkaCSVMetricsReporter" addedCSVConfig["kafka.csv.metrics.reporter.enabled"] = "true" + addedCSVConfig["listeners"] = "PLAINTEXT://:"+port if brokerVersion == "0.7": addedCSVConfig["brokerid"] = tcCfg["brokerid"] -- 1.9.3 (Apple Git-50) From 8f8c92ddbbdf92315c475af7a770aa427959f02c Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Wed, 24 Dec 2014 21:34:01 -0800 Subject: [PATCH 15/41] fixed system tests --- clients/src/main/java/org/apache/kafka/common/utils/Utils.java | 6 +++--- config/server.properties | 2 +- core/src/main/scala/kafka/client/ClientUtils.scala | 2 +- system_test/utils/kafka_system_test_utils.py | 10 +++++----- 4 files changed, 10 insertions(+), 10 deletions(-) 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 8a305b0..a97b944 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -27,7 +27,7 @@ import org.apache.kafka.common.KafkaException; public class Utils { - private static final Pattern HOST_PORT_PATTERN = Pattern.compile("\\[?(.+?)\\]?:(\\d+)"); + private static final Pattern HOST_PORT_PATTERN = Pattern.compile("^(.*)://([0-9a-z\\-.]*):([0-9]+)"); public static String NL = System.getProperty("line.separator"); @@ -303,7 +303,7 @@ public class Utils { */ public static String getHost(String address) { Matcher matcher = HOST_PORT_PATTERN.matcher(address); - return matcher.matches() ? matcher.group(1) : null; + return matcher.matches() ? matcher.group(2) : null; } /** @@ -313,7 +313,7 @@ public class Utils { */ public static Integer getPort(String address) { Matcher matcher = HOST_PORT_PATTERN.matcher(address); - return matcher.matches() ? Integer.parseInt(matcher.group(2)) : null; + return matcher.matches() ? Integer.parseInt(matcher.group(3)) : null; } /** diff --git a/config/server.properties b/config/server.properties index 80ee2fc..f4c8d92 100644 --- a/config/server.properties +++ b/config/server.properties @@ -21,7 +21,7 @@ broker.id=0 ############################# Socket Server Settings ############################# -listeners=PLAINTEXT://:9092 +listeners=PLAINTEXT://0.0.0.0:9092 # The port the socket server listens on #port=9092 diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 5acc878..2d53350 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -102,7 +102,7 @@ object ClientUtils extends Logging{ val brokersStr = Utils.parseCsvList(brokerListStr) brokersStr.zipWithIndex.map { case (address, brokerId) => - BrokerEndpoint.createBrokerEndPoint(brokerId,protocolType.toString + "://" + address) + BrokerEndpoint.createBrokerEndPoint(brokerId,address) } } diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index e4b5d80..a0e82c2 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -386,14 +386,14 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv if clusterName == "source": if ( len(testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"]) == 0 ): - testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"] = hostname + ":" + port + testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"] = "PLAINTEXT://" + hostname + ":" + port else: - testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"] += "," + hostname + ":" + port + testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"] += "," + "PLAINTEXT://" + hostname + ":" + port elif clusterName == "target": if ( len(testcaseEnv.userDefinedEnvVarDict["targetBrokerList"]) == 0 ): - testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] = hostname + ":" + port + testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] = "PLAINTEXT://" + hostname + ":" + port else: - testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] += "," + hostname + ":" + port + testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] += "," + "PLAINTEXT://" + hostname + ":" + port else: logger.error("Invalid cluster name: " + clusterName, extra=d) raise Exception("Invalid cluster name : " + clusterName) @@ -436,7 +436,7 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv addedCSVConfig["kafka.metrics.polling.interval.secs"] = "5" addedCSVConfig["kafka.metrics.reporters"] = "kafka.metrics.KafkaCSVMetricsReporter" addedCSVConfig["kafka.csv.metrics.reporter.enabled"] = "true" - addedCSVConfig["listeners"] = "PLAINTEXT://:"+port + addedCSVConfig["listeners"] = "PLAINTEXT://localhost:"+tcCfg["port"] if brokerVersion == "0.7": addedCSVConfig["brokerid"] = tcCfg["brokerid"] -- 1.9.3 (Apple Git-50) From 813960c814e72ddbbfb696ed5285dc619c1a2980 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Wed, 24 Dec 2014 22:43:17 -0800 Subject: [PATCH 16/41] fix default address binding and ipv6 support --- .../main/java/org/apache/kafka/common/utils/Utils.java | 2 +- .../org/apache/kafka/common/utils/ClientUtilsTest.java | 8 ++++---- .../java/org/apache/kafka/common/utils/UtilsTest.java | 16 ++++++++-------- config/server.properties | 2 +- core/src/main/scala/kafka/server/KafkaHealthcheck.scala | 8 +++++++- 5 files changed, 21 insertions(+), 15 deletions(-) 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 a97b944..2dce733 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -27,7 +27,7 @@ import org.apache.kafka.common.KafkaException; public class Utils { - private static final Pattern HOST_PORT_PATTERN = Pattern.compile("^(.*)://([0-9a-z\\-.]*):([0-9]+)"); + private static final Pattern HOST_PORT_PATTERN = Pattern.compile("^(.*)://\\[?([0-9a-z\\-.:]*)\\]?:([0-9]+)"); public static String NL = System.getProperty("line.separator"); diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java index 6e37ea5..c5d70b7 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java @@ -25,10 +25,10 @@ public class ClientUtilsTest { @Test public void testParseAndValidateAddresses() { - check("127.0.0.1:8000"); - check("mydomain.com:8080"); - check("[::1]:8000"); - check("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:1234", "mydomain.com:10000"); + check("PLAINTEXT://127.0.0.1:8000"); + check("PLAINTEXT://mydomain.com:8080"); + check("PLAINTEXT://[::1]:8000"); + check("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:1234", "PLAINTEXT://mydomain.com:10000"); } @Test(expected = ConfigException.class) 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 4c2ea34..f92f5b3 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -30,18 +30,18 @@ public class UtilsTest { @Test public void testGetHost() { - assertEquals("127.0.0.1", getHost("127.0.0.1:8000")); - assertEquals("mydomain.com", getHost("mydomain.com:8080")); - assertEquals("::1", getHost("[::1]:1234")); - assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertEquals("127.0.0.1", getHost("PLAINTEXT://127.0.0.1:8000")); + assertEquals("mydomain.com", getHost("PLAINTEXT://mydomain.com:8080")); + assertEquals("::1", getHost("PLAINTEXT://[::1]:1234")); + assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); } @Test public void testGetPort() { - assertEquals(8000, getPort("127.0.0.1:8000").intValue()); - assertEquals(8080, getPort("mydomain.com:8080").intValue()); - assertEquals(1234, getPort("[::1]:1234").intValue()); - assertEquals(5678, getPort("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678").intValue()); + assertEquals(8000, getPort("PLAINTEXT://127.0.0.1:8000").intValue()); + assertEquals(8080, getPort("PLAINTEXT://mydomain.com:8080").intValue()); + assertEquals(1234, getPort("PLAINTEXT://[::1]:1234").intValue()); + assertEquals(5678, getPort("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678").intValue()); } @Test diff --git a/config/server.properties b/config/server.properties index f4c8d92..80ee2fc 100644 --- a/config/server.properties +++ b/config/server.properties @@ -21,7 +21,7 @@ broker.id=0 ############################# Socket Server Settings ############################# -listeners=PLAINTEXT://0.0.0.0:9092 +listeners=PLAINTEXT://:9092 # The port the socket server listens on #port=9092 diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 3ee1256..00cc696 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -55,7 +55,13 @@ class KafkaHealthcheck(private val brokerId: Int, */ def register() { val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt - ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedEndpoints, zkSessionTimeoutMs, jmxPort) + val updatedEndpoints = advertisedEndpoints.map(endpoint => + if (endpoint.host == null || endpoint.host.trim.isEmpty) + EndPoint(InetAddress.getLocalHost.getCanonicalHostName, endpoint.port, endpoint.protocolType) + else + endpoint + ) + ZkUtils.registerBrokerInZk(zkClient, brokerId, updatedEndpoints, zkSessionTimeoutMs, jmxPort) } /** -- 1.9.3 (Apple Git-50) From 1c93b9fed5c838bc77bc168aaf591a87a69d68af Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 25 Dec 2014 09:43:26 -0800 Subject: [PATCH 17/41] fix some issues regarding endpoint parsing. Also, larger segments for systest make the validation much faster --- .../java/org/apache/kafka/common/utils/Utils.java | 6 +++--- .../apache/kafka/common/utils/ClientUtilsTest.java | 6 +++--- .../org/apache/kafka/common/utils/UtilsTest.java | 8 ++++---- .../main/scala/kafka/cluster/BrokerEndPoint.scala | 15 +++++++++++---- core/src/main/scala/kafka/cluster/EndPoint.scala | 5 ++++- .../test/scala/unit/kafka/cluster/BrokerTest.scala | 22 ++++++++++++++++++++-- .../testcase_1/testcase_1_properties.json | 6 +++--- 7 files changed, 48 insertions(+), 20 deletions(-) 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 2dce733..e4baa86 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -27,7 +27,7 @@ import org.apache.kafka.common.KafkaException; public class Utils { - private static final Pattern HOST_PORT_PATTERN = Pattern.compile("^(.*)://\\[?([0-9a-z\\-.:]*)\\]?:([0-9]+)"); + private static final Pattern HOST_PORT_PATTERN = Pattern.compile(".*?\\[?([0-9a-z\\-.:]*)\\]?:([0-9]+)"); public static String NL = System.getProperty("line.separator"); @@ -303,7 +303,7 @@ public class Utils { */ public static String getHost(String address) { Matcher matcher = HOST_PORT_PATTERN.matcher(address); - return matcher.matches() ? matcher.group(2) : null; + return matcher.matches() ? matcher.group(1) : null; } /** @@ -313,7 +313,7 @@ public class Utils { */ public static Integer getPort(String address) { Matcher matcher = HOST_PORT_PATTERN.matcher(address); - return matcher.matches() ? Integer.parseInt(matcher.group(3)) : null; + return matcher.matches() ? Integer.parseInt(matcher.group(2)) : null; } /** diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java index c5d70b7..be66c87 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java @@ -25,10 +25,10 @@ public class ClientUtilsTest { @Test public void testParseAndValidateAddresses() { - check("PLAINTEXT://127.0.0.1:8000"); + check("127.0.0.1:8000"); check("PLAINTEXT://mydomain.com:8080"); - check("PLAINTEXT://[::1]:8000"); - check("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:1234", "PLAINTEXT://mydomain.com:10000"); + check("[::1]:8000"); + check("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:1234", "mydomain.com:10000"); } @Test(expected = ConfigException.class) 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 f92f5b3..623f910 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -30,18 +30,18 @@ public class UtilsTest { @Test public void testGetHost() { - assertEquals("127.0.0.1", getHost("PLAINTEXT://127.0.0.1:8000")); + assertEquals("127.0.0.1", getHost("127.0.0.1:8000")); assertEquals("mydomain.com", getHost("PLAINTEXT://mydomain.com:8080")); - assertEquals("::1", getHost("PLAINTEXT://[::1]:1234")); + assertEquals("::1", getHost("[::1]:1234")); assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); } @Test public void testGetPort() { assertEquals(8000, getPort("PLAINTEXT://127.0.0.1:8000").intValue()); - assertEquals(8080, getPort("PLAINTEXT://mydomain.com:8080").intValue()); + assertEquals(8080, getPort("mydomain.com:8080").intValue()); assertEquals(1234, getPort("PLAINTEXT://[::1]:1234").intValue()); - assertEquals(5678, getPort("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678").intValue()); + assertEquals(5678, getPort("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678").intValue()); } @Test diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala index 90ea612..66fc47b 100644 --- a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -3,18 +3,25 @@ package kafka.cluster import java.nio.ByteBuffer import kafka.api.ApiUtils._ +import kafka.common.KafkaException import org.apache.kafka.common.utils.Utils._ import kafka.cluster.ProtocolType._ object BrokerEndpoint { def createBrokerEndPoint(brokerId: Int, connectionString: String): BrokerEndpoint = { - val endPoint = EndPoint.createEndPoint(connectionString) - new BrokerEndpoint(brokerId,endPoint.host,endPoint.port.toInt) + val uriParseExp = """.*?\[?([0-9a-z\-.:]*)\]?:([0-9]+)""".r + + connectionString match { + case uriParseExp(host,port) => new BrokerEndpoint(brokerId,host,port.toInt) + case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") + } } /** - * BrokerEndpoint includes the protocol type to allow locating the right endpoint in lists - * but we don't serialize or de-serialize it + * BrokerEndpoint is used to connect to specific host:port pair + * It is typically used by clients (or brokers when connecting to other brokers) + * and contains no information about the security protocol used on the connection + * clients should know which security protocol to use from configuration * this allows us to keep the wire protocol with the clients unchanged where the protocol is not needed * @param buffer * @return diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index 1b5fe8f..1e72927 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -33,7 +33,7 @@ object EndPoint { } def createEndPoint(connectionString: String): EndPoint = { - val uriParseExp = """^(.*)://([0-9a-z\-.]*):([0-9]+)""".r + val uriParseExp = """^(.*)://\[?([0-9a-z\-.:]*)\]?:([0-9]+)""".r connectionString match { case uriParseExp(protocol,"",port) => new EndPoint(null,port.toInt,ProtocolType.withName(protocol)) case uriParseExp(protocol,host,port) => new EndPoint(host,port.toInt,ProtocolType.withName(protocol)) @@ -42,6 +42,9 @@ object EndPoint { } } +/** + * Part of the broker definition - matching host/port pair to a protocol + */ case class EndPoint(host: String, port: Int, protocolType: ProtocolType) { override def toString: String = protocolType + "://" + host + ":" + port diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index 7d59037..c52c211 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -80,16 +80,34 @@ class BrokerTest extends JUnit3Suite with Logging { } @Test + def testBrokerEndpointFromURI() = { + var connectionString = "localhost:9092" + var endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) + assert(endpoint.host == "localhost") + assert(endpoint.port == 9092) + // also test for ipv6 + connectionString = "[::1]:9092" + endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) + assert(endpoint.host == "::1") + assert(endpoint.port == 9092) + } + + @Test def testEndpointFromURI() = { var connectionString = "PLAINTEXT://localhost:9092" - var endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) + var endpoint = EndPoint.createEndPoint(connectionString) assert(endpoint.host == "localhost") assert(endpoint.port == 9092) // also test for default bind connectionString = "PLAINTEXT://:9092" - endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) + endpoint = EndPoint.createEndPoint(connectionString) assert(endpoint.host == null) assert(endpoint.port == 9092) + // also test for ipv6 + connectionString = "PLAINTEXT://[::1]:9092" + endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == "::1") + assert(endpoint.port == 9092) } diff --git a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json index 0c6d7a3..680213f 100644 --- a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json +++ b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -41,7 +41,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -50,7 +50,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" -- 1.9.3 (Apple Git-50) From 8f0a0f8c70dad3fbff5617b80c98338be4a9fa95 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 25 Dec 2014 10:15:16 -0800 Subject: [PATCH 18/41] added link to security wiki in doc --- .../kafka/clients/producer/ProducerConfig.java | 3 ++- core/src/main/scala/kafka/admin/AdminUtils.scala | 6 ++--- .../scala/kafka/api/ConsumerMetadataRequest.scala | 8 +++---- .../scala/kafka/api/ConsumerMetadataResponse.scala | 2 +- .../scala/kafka/api/TopicMetadataRequest.scala | 12 +++++----- .../scala/kafka/api/TopicMetadataResponse.scala | 2 +- core/src/main/scala/kafka/client/ClientUtils.scala | 8 +++---- core/src/main/scala/kafka/cluster/Broker.scala | 8 +++---- .../main/scala/kafka/cluster/BrokerEndPoint.scala | 2 +- core/src/main/scala/kafka/cluster/EndPoint.scala | 10 ++++----- .../main/scala/kafka/cluster/ProtocolType.scala | 26 ---------------------- .../scala/kafka/cluster/SecurityProtocol.scala | 26 ++++++++++++++++++++++ .../main/scala/kafka/consumer/ConsumerConfig.scala | 4 ++-- .../scala/kafka/javaapi/TopicMetadataRequest.scala | 12 +++++----- .../main/scala/kafka/network/SocketServer.scala | 2 +- .../main/scala/kafka/producer/ProducerConfig.scala | 4 ++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 4 ++-- .../main/scala/kafka/server/MetadataCache.scala | 8 +++---- .../scala/kafka/tools/ConsumerOffsetChecker.scala | 4 ++-- .../main/scala/kafka/tools/GetOffsetShell.scala | 2 +- .../scala/kafka/tools/SimpleConsumerShell.scala | 2 +- .../main/scala/kafka/tools/UpdateOffsetsInZK.scala | 6 ++--- core/src/main/scala/kafka/utils/Utils.scala | 2 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 6 ++--- .../test/scala/other/kafka/TestOffsetManager.scala | 6 ++--- .../scala/unit/kafka/admin/AddPartitionsTest.scala | 8 +++---- .../api/RequestResponseSerializationTest.scala | 16 ++++++------- .../test/scala/unit/kafka/cluster/BrokerTest.scala | 14 ++++++------ .../unit/kafka/consumer/ConsumerIteratorTest.scala | 2 +- .../unit/kafka/integration/TopicMetadataTest.scala | 4 ++-- .../unit/kafka/network/SocketServerTest.scala | 6 ++--- .../unit/kafka/producer/SyncProducerTest.scala | 18 +++++++-------- .../unit/kafka/server/AdvertiseBrokerTest.scala | 6 ++--- .../scala/unit/kafka/server/KafkaConfigTest.scala | 6 ++--- .../unit/kafka/server/LeaderElectionTest.scala | 2 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 4 ++-- 36 files changed, 131 insertions(+), 130 deletions(-) delete mode 100644 core/src/main/scala/kafka/cluster/ProtocolType.scala create mode 100644 core/src/main/scala/kafka/cluster/SecurityProtocol.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index bc6cd91..a765b45 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 @@ -167,7 +167,8 @@ public class ProducerConfig extends AbstractConfig { /** security.protocol */ public static final String SECURITY_PROTOCOL = "security.protocol"; - private static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT is supported. SSL and Kerberos are planned for the near future"; + private static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT is supported (and is the default). SSL and Kerberos are planned for the near future. " + + " Details on Kafka's security plans are available on our wiki: https://cwiki.apache.org/confluence/display/KAFKA/Security"; static { config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 7e57b8c..1f18872 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -21,8 +21,8 @@ import kafka.common._ import java.util.Random import java.util.Properties import kafka.api.{TopicMetadata, PartitionMetadata} -import kafka.cluster.ProtocolType.ProtocolType -import kafka.cluster.{BrokerEndpoint, Broker, ProtocolType} +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.{BrokerEndpoint, Broker, SecurityProtocol} import kafka.log.LogConfig import kafka.utils.{Logging, ZkUtils, Json} @@ -294,7 +294,7 @@ object AdminUtils extends Logging { - private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker], protocol: ProtocolType = ProtocolType.PLAINTEXT): TopicMetadata = { + private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): TopicMetadata = { if(ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) { val topicPartitionAssignment = ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic).get val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index ed950af..e08492a 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -18,8 +18,8 @@ package kafka.api import java.nio.ByteBuffer -import kafka.cluster.ProtocolType -import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.SecurityProtocol +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.common.ErrorMapping @@ -35,7 +35,7 @@ object ConsumerMetadataRequest { val clientId = ApiUtils.readShortString(buffer) // request val group = ApiUtils.readShortString(buffer) - val securityProtocol = ProtocolType.withName(ApiUtils.readShortString(buffer)) + val securityProtocol = SecurityProtocol.withName(ApiUtils.readShortString(buffer)) ConsumerMetadataRequest(group, versionId, correlationId, clientId, securityProtocol) } @@ -45,7 +45,7 @@ case class ConsumerMetadataRequest(group: String, versionId: Short = ConsumerMetadataRequest.CurrentVersion, correlationId: Int = 0, clientId: String = ConsumerMetadataRequest.DefaultClientId, - securityProtocol: ProtocolType = ProtocolType.PLAINTEXT) + securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey)) { def sizeInBytes = diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala index 0f94452..861d5f7 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -18,7 +18,7 @@ package kafka.api import java.nio.ByteBuffer -import kafka.cluster.{ProtocolType, BrokerEndpoint, Broker} +import kafka.cluster.{SecurityProtocol, BrokerEndpoint, Broker} import kafka.common.ErrorMapping object ConsumerMetadataResponse { diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 4f1ad1a..423eda6 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -19,9 +19,9 @@ package kafka.api import java.nio.ByteBuffer import kafka.api.ApiUtils._ -import kafka.cluster.ProtocolType -import kafka.cluster.ProtocolType.ProtocolType -import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.SecurityProtocol +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.SecurityProtocol.SecurityProtocol import collection.mutable.ListBuffer import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping @@ -57,19 +57,19 @@ object TopicMetadataRequest extends Logging { if (versionId == 1) { securityProtocol = readShortString(buffer) } - new TopicMetadataRequest(versionId, correlationId, clientId, ProtocolType.withName(securityProtocol), topics.toList) + new TopicMetadataRequest(versionId, correlationId, clientId, SecurityProtocol.withName(securityProtocol), topics.toList) } } case class TopicMetadataRequest(val versionId: Short, val correlationId: Int, val clientId: String, - val securityProtocol: ProtocolType, + val securityProtocol: SecurityProtocol, val topics: Seq[String]) extends RequestOrResponse(Some(RequestKeys.MetadataKey)){ def this(topics: Seq[String], correlationId: Int) = - this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, ProtocolType.PLAINTEXT, topics) + this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, SecurityProtocol.PLAINTEXT, topics) def writeTo(buffer: ByteBuffer) { buffer.putShort(versionId) diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala index 70450be..c55f60c 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala @@ -17,7 +17,7 @@ package kafka.api -import kafka.cluster.{ProtocolType, BrokerEndpoint, Broker} +import kafka.cluster.{SecurityProtocol, BrokerEndpoint, Broker} import java.nio.ByteBuffer object TopicMetadataResponse { diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 2d53350..01618c5 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -16,7 +16,7 @@ */ package kafka.client - import kafka.cluster.ProtocolType.ProtocolType + import kafka.cluster.SecurityProtocol.SecurityProtocol import scala.collection._ import kafka.cluster._ @@ -98,7 +98,7 @@ object ClientUtils extends Logging{ /** * Parse a list of broker urls in the form host1:port1, host2:port2, ... */ - def parseBrokerList(brokerListStr: String, protocolType: ProtocolType = ProtocolType.PLAINTEXT): Seq[BrokerEndpoint] = { + def parseBrokerList(brokerListStr: String, protocolType: SecurityProtocol = SecurityProtocol.PLAINTEXT): Seq[BrokerEndpoint] = { val brokersStr = Utils.parseCsvList(brokerListStr) brokersStr.zipWithIndex.map { case (address, brokerId) => @@ -109,7 +109,7 @@ object ClientUtils extends Logging{ /** * Creates a blocking channel to a random broker */ - def channelToAnyBroker(zkClient: ZkClient, protocolType: ProtocolType, socketTimeoutMs: Int = 3000) : BlockingChannel = { + def channelToAnyBroker(zkClient: ZkClient, protocolType: SecurityProtocol, socketTimeoutMs: Int = 3000) : BlockingChannel = { var channel: BlockingChannel = null var connected = false while (!connected) { @@ -138,7 +138,7 @@ object ClientUtils extends Logging{ /** * Creates a blocking channel to the offset manager of the given group */ - def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000, protocolType: ProtocolType = ProtocolType.PLAINTEXT) = { + def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000, protocolType: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { var queryChannel = channelToAnyBroker(zkClient, protocolType) var offsetManagerChannelOpt: Option[BlockingChannel] = None diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index a84a34e..f81acfa 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -21,7 +21,7 @@ import kafka.utils.Utils._ import kafka.utils.{Utils, Json} import java.nio.ByteBuffer import kafka.common.{BrokerEndPointNotAvailableException, KafkaException, BrokerNotAvailableException} -import kafka.cluster.ProtocolType._ +import kafka.cluster.SecurityProtocol._ /** * A Kafka broker @@ -93,7 +93,7 @@ case class Broker(id: Int, endPoints: Seq[EndPoint]) { override def toString: String = id + " : " + endPoints.mkString("(",",",")") def this(id: Int, host: String, port: Int) = { - this(id,List(EndPoint(host,port,ProtocolType.PLAINTEXT))) + this(id,List(EndPoint(host,port,SecurityProtocol.PLAINTEXT))) } @@ -110,11 +110,11 @@ case class Broker(id: Int, endPoints: Seq[EndPoint]) { 4 + /* number of endPoints */ endPoints.map(_.sizeInBytes).sum /* end points */ - def supportsChannel(protocolType: ProtocolType): Unit = { + def supportsChannel(protocolType: SecurityProtocol): Unit = { endPoints.map((endpoint)=>(endpoint.protocolType)).contains(protocolType) } - def getBrokerEndPoint(protocolType: ProtocolType): BrokerEndpoint = { + def getBrokerEndPoint(protocolType: SecurityProtocol): BrokerEndpoint = { val endpoint = endPoints.map((endpoint)=>(endpoint.protocolType,endpoint)).toMap.get(protocolType) endpoint match { case Some(endpoint) => new BrokerEndpoint(id,endpoint.host,endpoint.port) diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala index 66fc47b..7e6825a 100644 --- a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -5,7 +5,7 @@ import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.common.KafkaException import org.apache.kafka.common.utils.Utils._ -import kafka.cluster.ProtocolType._ +import kafka.cluster.SecurityProtocol._ object BrokerEndpoint { def createBrokerEndPoint(brokerId: Int, connectionString: String): BrokerEndpoint = { diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index 1e72927..364b2f7 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.common.KafkaException -import kafka.cluster.ProtocolType._ +import kafka.cluster.SecurityProtocol._ object EndPoint { @@ -29,14 +29,14 @@ object EndPoint { val port = buffer.getInt() val host = readShortString(buffer) val channel = readShortString(buffer) - EndPoint(host,port,ProtocolType.withName(channel)) + EndPoint(host,port,SecurityProtocol.withName(channel)) } def createEndPoint(connectionString: String): EndPoint = { val uriParseExp = """^(.*)://\[?([0-9a-z\-.:]*)\]?:([0-9]+)""".r connectionString match { - case uriParseExp(protocol,"",port) => new EndPoint(null,port.toInt,ProtocolType.withName(protocol)) - case uriParseExp(protocol,host,port) => new EndPoint(host,port.toInt,ProtocolType.withName(protocol)) + case uriParseExp(protocol,"",port) => new EndPoint(null,port.toInt,SecurityProtocol.withName(protocol)) + case uriParseExp(protocol,host,port) => new EndPoint(host,port.toInt,SecurityProtocol.withName(protocol)) case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") } } @@ -45,7 +45,7 @@ object EndPoint { /** * Part of the broker definition - matching host/port pair to a protocol */ -case class EndPoint(host: String, port: Int, protocolType: ProtocolType) { +case class EndPoint(host: String, port: Int, protocolType: SecurityProtocol) { override def toString: String = protocolType + "://" + host + ":" + port diff --git a/core/src/main/scala/kafka/cluster/ProtocolType.scala b/core/src/main/scala/kafka/cluster/ProtocolType.scala deleted file mode 100644 index f4e6bc3..0000000 --- a/core/src/main/scala/kafka/cluster/ProtocolType.scala +++ /dev/null @@ -1,26 +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.cluster - - -object ProtocolType extends Enumeration { - - type ProtocolType = Value - val PLAINTEXT = Value -} - diff --git a/core/src/main/scala/kafka/cluster/SecurityProtocol.scala b/core/src/main/scala/kafka/cluster/SecurityProtocol.scala new file mode 100644 index 0000000..82b4631 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/SecurityProtocol.scala @@ -0,0 +1,26 @@ +/* + * 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.cluster + + +object SecurityProtocol extends Enumeration { + + type SecurityProtocol = Value + val PLAINTEXT = Value +} + diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index fe89a30..054bd62 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -19,7 +19,7 @@ package kafka.consumer import java.util.Properties import kafka.api.OffsetRequest -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol import kafka.utils._ import kafka.common.{InvalidConfigException, Config} @@ -184,7 +184,7 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( val partitionAssignmentStrategy = props.getString("partition.assignment.strategy", DefaultPartitionAssignmentStrategy) /* plaintext or SSL */ - val securityProtocol = ProtocolType.withName(props.getString("security.protocol", "PLAINTEXT")) + val securityProtocol = SecurityProtocol.withName(props.getString("security.protocol", "PLAINTEXT")) validate(this) } diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index 3ea12c9..de33b04 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -18,9 +18,9 @@ package kafka.javaapi import kafka.api._ import java.nio.ByteBuffer -import kafka.cluster.ProtocolType -import kafka.cluster.ProtocolType.ProtocolType -import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.SecurityProtocol +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.SecurityProtocol.SecurityProtocol import scala.collection.mutable import kafka.network.{BoundedByteBufferSend, RequestChannel} @@ -30,7 +30,7 @@ import kafka.network.RequestChannel.Response class TopicMetadataRequest(val versionId: Short, val correlationId: Int, val clientId: String, - val securityProtocol: ProtocolType, + val securityProtocol: SecurityProtocol, val topics: java.util.List[String]) extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) { @@ -40,10 +40,10 @@ class TopicMetadataRequest(val versionId: Short, } def this(topics: java.util.List[String]) = - this(kafka.api.TopicMetadataRequest.CurrentVersion, 0, kafka.api.TopicMetadataRequest.DefaultClientId, ProtocolType.PLAINTEXT, topics) + this(kafka.api.TopicMetadataRequest.CurrentVersion, 0, kafka.api.TopicMetadataRequest.DefaultClientId, SecurityProtocol.PLAINTEXT, topics) def this(topics: java.util.List[String], correlationId: Int) = - this(kafka.api.TopicMetadataRequest.CurrentVersion, correlationId, kafka.api.TopicMetadataRequest.DefaultClientId, ProtocolType.PLAINTEXT, topics) + this(kafka.api.TopicMetadataRequest.CurrentVersion, correlationId, kafka.api.TopicMetadataRequest.DefaultClientId, SecurityProtocol.PLAINTEXT, topics) def writeTo(buffer: ByteBuffer) = underlying.writeTo(buffer) diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 6f21057..7009030 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -25,7 +25,7 @@ import java.io._ import java.nio.channels._ import kafka.cluster.EndPoint -import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.SecurityProtocol.SecurityProtocol import scala.collection._ import scala.collection.JavaConversions._ diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index bbbf651..9aff401 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -19,7 +19,7 @@ package kafka.producer import async.AsyncProducerConfig import java.util.Properties -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol import kafka.utils.{Utils, VerifiableProperties} import kafka.message.{CompressionCodec, NoCompressionCodec} import kafka.common.{InvalidConfigException, Config} @@ -115,7 +115,7 @@ class ProducerConfig private (val props: VerifiableProperties) val topicMetadataRefreshIntervalMs = props.getInt("topic.metadata.refresh.interval.ms", 600000) /* plaintext or SSL */ - val securityProtocol = ProtocolType.withName(props.getString("security.protocol", "PLAINTEXT")) + val securityProtocol = SecurityProtocol.withName(props.getString("security.protocol", "PLAINTEXT")) validate(this) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index a20fd42..c6257ed 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -18,7 +18,7 @@ package kafka.server import java.util.Properties -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol import kafka.message.{MessageSet, Message} import kafka.consumer.ConsumerConfig import kafka.utils.{VerifiableProperties, ZKConfig, Utils} @@ -130,7 +130,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val connectionsMaxIdleMs = props.getLong("connections.max.idle.ms", 10*60*1000L) /* security protocol used to communicate between brokers */ - val securityProtocol = ProtocolType.withName(props.getString("security.protocol","PLAINTEXT")) + val securityProtocol = SecurityProtocol.withName(props.getString("security.protocol","PLAINTEXT")) /*********** Log Configuration ***********/ diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 3df176c..cb96edb 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -17,12 +17,12 @@ package kafka.server -import kafka.cluster.ProtocolType.ProtocolType -import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.SecurityProtocol.SecurityProtocol import scala.collection.{Seq, Set, mutable} import kafka.api._ -import kafka.cluster.{BrokerEndpoint, ProtocolType, Broker} +import kafka.cluster.{BrokerEndpoint, SecurityProtocol, Broker} import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.utils.Utils._ import kafka.common.{ErrorMapping, ReplicaNotAvailableException, LeaderNotAvailableException} @@ -39,7 +39,7 @@ private[server] class MetadataCache { private var aliveBrokers: Map[Int, Broker] = Map() private val partitionMetadataLock = new ReentrantReadWriteLock() - def getTopicMetadata(topics: Set[String], protocol: ProtocolType = ProtocolType.PLAINTEXT) = { + def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { val isAllTopics = topics.isEmpty val topicsRequested = if(isAllTopics) cache.keySet else topics val topicResponses: mutable.ListBuffer[TopicMetadata] = new mutable.ListBuffer[TopicMetadata] diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index 7198322..03b121d 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -19,7 +19,7 @@ package kafka.tools import joptsimple._ -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol import org.I0Itec.zkclient.ZkClient import kafka.utils._ import kafka.consumer.SimpleConsumer @@ -159,7 +159,7 @@ object ConsumerOffsetChecker extends Logging { topicPidMap = immutable.Map(ZkUtils.getPartitionsForTopics(zkClient, topicList).toSeq:_*) val topicPartitions = topicPidMap.flatMap { case(topic, partitionSeq) => partitionSeq.map(TopicAndPartition(topic, _)) }.toSeq - val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs, ProtocolType.PLAINTEXT) + val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs, SecurityProtocol.PLAINTEXT) debug("Sending offset fetch request to coordinator %s:%d.".format(channel.host, channel.port)) channel.send(OffsetFetchRequest(group, topicPartitions)) diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 1596516..4f0b463 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -24,7 +24,7 @@ import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} import kafka.common.TopicAndPartition import kafka.client.ClientUtils import kafka.utils.{ToolsUtils, CommandLineUtils} -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol object GetOffsetShell { diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index ac86762..84c09c2 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -22,7 +22,7 @@ import kafka.utils._ import kafka.consumer._ import kafka.client.ClientUtils import kafka.api.{OffsetRequest, FetchRequestBuilder, Request} -import kafka.cluster.{BrokerEndpoint, ProtocolType, Broker} +import kafka.cluster.{BrokerEndpoint, SecurityProtocol, Broker} import scala.collection.JavaConversions._ import kafka.common.TopicAndPartition diff --git a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala index 38d3b48..c6c5a88 100644 --- a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala +++ b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala @@ -17,7 +17,7 @@ package kafka.tools -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol import org.I0Itec.zkclient.ZkClient import kafka.consumer.{SimpleConsumer, ConsumerConfig} import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} @@ -66,8 +66,8 @@ object UpdateOffsetsInZK { ZkUtils.getBrokerInfo(zkClient, broker) match { case Some(brokerInfo) => - val consumer = new SimpleConsumer(brokerInfo.getBrokerEndPoint(ProtocolType.PLAINTEXT).host, - brokerInfo.getBrokerEndPoint(ProtocolType.PLAINTEXT).port, + val consumer = new SimpleConsumer(brokerInfo.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host, + brokerInfo.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port, 10000, 100 * 1024, "UpdateOffsetsInZk") val topicAndPartition = TopicAndPartition(topic, partition) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1))) diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index b823c1e..0a96c0f 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -31,7 +31,7 @@ import java.util.Properties import kafka.common.KafkaException import kafka.common.KafkaStorageException import kafka.cluster.EndPoint -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol /** diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index a90fc64..e0a97c0 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -17,7 +17,7 @@ package kafka.utils -import kafka.cluster.ProtocolType.ProtocolType +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.cluster._ import kafka.consumer.{ConsumerThreadId, TopicCount} import org.I0Itec.zkclient.ZkClient @@ -84,7 +84,7 @@ object ZkUtils extends Logging { brokerIds.map(_.toInt).map(getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) } - def getAllBrokerEndPointsForChannel(zkClient: ZkClient, protocolType: ProtocolType): Seq[BrokerEndpoint] = { + def getAllBrokerEndPointsForChannel(zkClient: ZkClient, protocolType: SecurityProtocol): Seq[BrokerEndpoint] = { getAllBrokersInCluster(zkClient).map(_.getBrokerEndPoint(protocolType)) } @@ -176,7 +176,7 @@ object ZkUtils extends Logging { def registerBrokerInZk(zkClient: ZkClient, id: Int, advertisedEndpoints: Seq[EndPoint], timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val defaultEndPoint = advertisedEndpoints.find(_.protocolType == ProtocolType.PLAINTEXT).get + val defaultEndPoint = advertisedEndpoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get val host = defaultEndPoint.host val port = defaultEndPoint.port diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala index 67301ed..28f70b4 100644 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -1,6 +1,6 @@ package other.kafka -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol import org.I0Itec.zkclient.ZkClient import kafka.api._ import kafka.utils.{ShutdownableThread, ZKStringSerializer} @@ -111,7 +111,7 @@ object TestOffsetManager { private val fetchTimer = new KafkaTimer(timer) private val channels = mutable.Map[Int, BlockingChannel]() - private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, ProtocolType.PLAINTEXT, SocketTimeoutMs) + private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SecurityProtocol.PLAINTEXT, SocketTimeoutMs) private val numErrors = new AtomicInteger(0) @@ -157,7 +157,7 @@ object TestOffsetManager { println("Error while querying %s:%d - shutting down query channel.".format(metadataChannel.host, metadataChannel.port)) metadataChannel.disconnect() println("Creating new query channel.") - metadataChannel = ClientUtils.channelToAnyBroker(zkClient, ProtocolType.PLAINTEXT, SocketTimeoutMs) + metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SecurityProtocol.PLAINTEXT, SocketTimeoutMs) } finally { Thread.sleep(fetchIntervalMs) diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 2f338b7..7118b40 100644 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -22,7 +22,7 @@ import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} -import kafka.cluster.{ProtocolType, EndPoint, Broker} +import kafka.cluster.{SecurityProtocol, EndPoint, Broker} import kafka.client.ClientUtils import kafka.server.{KafkaConfig, KafkaServer} @@ -109,7 +109,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 2) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)), "AddPartitionsTest-testIncrementPartitions", + val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testIncrementPartitions", 2000,0).topicsMetadata val metaDataForTopic1 = metadata.filter(p => p.topic.equals(topic1)) val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata @@ -134,7 +134,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 2) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)), "AddPartitionsTest-testManualAssignmentOfReplicas", + val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testManualAssignmentOfReplicas", 2000,0).topicsMetadata val metaDataForTopic2 = metadata.filter(p => p.topic.equals(topic2)) val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata @@ -158,7 +158,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 5) TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 6) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)), "AddPartitionsTest-testReplicaPlacement", + val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testReplicaPlacement", 2000,0).topicsMetadata val metaDataForTopic3 = metadata.filter(p => p.topic.equals(topic3)).head diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 2d19f65..971d055 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -22,7 +22,7 @@ import org.scalatest.junit.JUnitSuite import junit.framework.Assert._ import java.nio.ByteBuffer import kafka.message.{Message, ByteBufferMessageSet} -import kafka.cluster.{BrokerEndpoint, ProtocolType, EndPoint, Broker} +import kafka.cluster.{BrokerEndpoint, SecurityProtocol, EndPoint, Broker} import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError} import kafka.utils.SystemTime import org.apache.kafka.common.requests._ @@ -80,10 +80,10 @@ object SerializationTestUtils { TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100) ) - private val brokers = List(new Broker(0, List(EndPoint("localhost", 1011, ProtocolType.PLAINTEXT))), - new Broker(1, List(EndPoint("localhost", 1012, ProtocolType.PLAINTEXT))), - new Broker(2, List(EndPoint("localhost", 1013, ProtocolType.PLAINTEXT)))) - private val brokerEndpoints = brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)) + private val brokers = List(new Broker(0, List(EndPoint("localhost", 1011, SecurityProtocol.PLAINTEXT))), + new Broker(1, List(EndPoint("localhost", 1012, SecurityProtocol.PLAINTEXT))), + new Broker(2, List(EndPoint("localhost", 1013, SecurityProtocol.PLAINTEXT)))) + private val brokerEndpoints = brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) private val partitionMetaData0 = new PartitionMetadata(0, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints, errorCode = 0) private val partitionMetaData1 = new PartitionMetadata(1, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints.tail, errorCode = 1) @@ -148,11 +148,11 @@ object SerializationTestUtils { } def createTestTopicMetadataRequest: TopicMetadataRequest = { - new TopicMetadataRequest(1, 1, "client 1", ProtocolType.PLAINTEXT, Seq(topic1, topic2)) + new TopicMetadataRequest(1, 1, "client 1", SecurityProtocol.PLAINTEXT, Seq(topic1, topic2)) } def createTestTopicMetadataResponse: TopicMetadataResponse = { - new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(ProtocolType.PLAINTEXT)).toVector, Seq(topicmetaData1, topicmetaData2), 1) + new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)).toVector, Seq(topicmetaData1, topicmetaData2), 1) } def createTestOffsetCommitRequestV1: OffsetCommitRequest = { @@ -196,7 +196,7 @@ object SerializationTestUtils { } def createConsumerMetadataResponse: ConsumerMetadataResponse = { - ConsumerMetadataResponse(Some(brokers.head.getBrokerEndPoint(ProtocolType.PLAINTEXT)), ErrorMapping.NoError, 0) + ConsumerMetadataResponse(Some(brokers.head.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), ErrorMapping.NoError, 0) } def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = { diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index c52c211..7a30f3f 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -30,7 +30,7 @@ class BrokerTest extends JUnit3Suite with Logging { @Test def testSerDe() = { - val endpoint = new EndPoint("myhost",9092,ProtocolType.PLAINTEXT) + val endpoint = new EndPoint("myhost",9092,SecurityProtocol.PLAINTEXT) val listEndPoints = List(endpoint) val origBroker = new Broker(1,listEndPoints) val brokerBytes = ByteBuffer.allocate(origBroker.sizeInBytes) @@ -43,10 +43,10 @@ class BrokerTest extends JUnit3Suite with Logging { @Test def testHashAndEquals() = { - val endpoint1 = new EndPoint("myhost",9092,ProtocolType.PLAINTEXT) - val endpoint2 = new EndPoint("myhost",9092,ProtocolType.PLAINTEXT) - val endpoint3 = new EndPoint("myhost",1111,ProtocolType.PLAINTEXT) - val endpoint4 = new EndPoint("other",1111,ProtocolType.PLAINTEXT) + val endpoint1 = new EndPoint("myhost",9092,SecurityProtocol.PLAINTEXT) + val endpoint2 = new EndPoint("myhost",9092,SecurityProtocol.PLAINTEXT) + val endpoint3 = new EndPoint("myhost",1111,SecurityProtocol.PLAINTEXT) + val endpoint4 = new EndPoint("other",1111,SecurityProtocol.PLAINTEXT) val broker1 = new Broker(1,List(endpoint1)) val broker2 = new Broker(1,List(endpoint2)) val broker3 = new Broker(2,List(endpoint3)) @@ -75,8 +75,8 @@ class BrokerTest extends JUnit3Suite with Logging { "\"endpoints\":\"PLAINTEXT://localhost:9092\"}" val broker = Broker.createBroker(1, brokerInfoStr) assert(broker.id == 1) - assert(broker.getBrokerEndPoint(ProtocolType.PLAINTEXT).host == "localhost") - assert(broker.getBrokerEndPoint(ProtocolType.PLAINTEXT).port == 9092) + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host == "localhost") + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port == 9092) } @Test diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index ec31b34..e48dc0b 100644 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -29,7 +29,7 @@ import kafka.utils.TestUtils._ import kafka.utils._ import org.junit.Test import kafka.serializer._ -import kafka.cluster.{ProtocolType, EndPoint, Broker, Cluster} +import kafka.cluster.{SecurityProtocol, EndPoint, Broker, Cluster} import org.scalatest.junit.JUnit3Suite import kafka.integration.KafkaServerTestHarness diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index ceb39fa..025a349 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -22,7 +22,7 @@ import kafka.zk.ZooKeeperTestHarness import kafka.admin.AdminUtils import java.nio.ByteBuffer import junit.framework.Assert._ -import kafka.cluster.{ProtocolType, EndPoint, Broker} +import kafka.cluster.{SecurityProtocol, EndPoint, Broker} import kafka.utils.{Utils, TestUtils} import kafka.utils.TestUtils._ import kafka.server.{KafkaServer, KafkaConfig} @@ -34,7 +34,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { val props = createBrokerConfigs(1) val configs = props.map(p => new KafkaConfig(p)) private var server1: KafkaServer = null - val brokerEndPoints = configs.map(c => new Broker(c.brokerId,Utils.listenerListToEndPoints(c.listeners)).getBrokerEndPoint(ProtocolType.PLAINTEXT)) + val brokerEndPoints = configs.map(c => new Broker(c.brokerId,Utils.listenerListToEndPoints(c.listeners)).getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) override def setUp() { super.setUp() diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index ab8dd77..1c68456 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -19,7 +19,7 @@ package kafka.network; import java.net._ import java.io._ -import kafka.cluster.{ProtocolType, EndPoint} +import kafka.cluster.{SecurityProtocol, EndPoint} import org.junit._ import org.scalatest.junit.JUnitSuite import java.util.Random @@ -36,7 +36,7 @@ import scala.collection.Map class SocketServerTest extends JUnitSuite { val server: SocketServer = new SocketServer(0, - List(EndPoint(null,kafka.utils.TestUtils.choosePort,ProtocolType.PLAINTEXT)), + List(EndPoint(null,kafka.utils.TestUtils.choosePort,SecurityProtocol.PLAINTEXT)), numProcessorThreads = 1, maxQueuedRequests = 50, sendBufferSize = 300000, @@ -73,7 +73,7 @@ class SocketServerTest extends JUnitSuite { channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) } - def connect(s:SocketServer = server) = new Socket("localhost", server.endpoints.find(_.protocolType == ProtocolType.PLAINTEXT).get.port) + def connect(s:SocketServer = server) = new Socket("localhost", server.endpoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get.port) @After def cleanup() { diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 708fa99..5630ddd 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -21,7 +21,7 @@ import java.net.SocketTimeoutException import java.util.Properties import junit.framework.Assert import kafka.admin.AdminUtils -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol import kafka.integration.KafkaServerTestHarness import kafka.message._ import kafka.server.KafkaConfig @@ -40,7 +40,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testReachableServer() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -76,7 +76,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testEmptyProduceRequest() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 @@ -94,7 +94,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLarge() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -122,7 +122,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLargeWithAckZero() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) props.put("request.required.acks", "0") @@ -150,7 +150,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceCorrectlyReceivesResponse() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -197,7 +197,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { val timeoutMs = 500 val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -224,7 +224,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceRequestWithNoResponse() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 val clientId = SyncProducerConfig.DefaultClientId @@ -240,7 +240,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { def testNotEnoughReplicas() { val topicName = "minisrtest" val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==ProtocolType.PLAINTEXT).get.port + val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) props.put("request.required.acks", "-1") diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala index 49e3933..d503cbb 100644 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -17,7 +17,7 @@ package kafka.server -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import junit.framework.Assert._ @@ -32,7 +32,7 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) - props.put("advertised.listeners",ProtocolType.PLAINTEXT.toString+"://"+advertisedHostName+":"+advertisedPort.toString) + props.put("advertised.listeners",SecurityProtocol.PLAINTEXT.toString+"://"+advertisedHostName+":"+advertisedPort.toString) server = TestUtils.createServer(new KafkaConfig(props)) } @@ -45,7 +45,7 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { def testBrokerAdvertiseToZK { val brokerInfo = ZkUtils.getBrokerInfo(zkClient, brokerId) - val endpoint = brokerInfo.get.endPoints.find(_.protocolType == ProtocolType.PLAINTEXT).get + val endpoint = brokerInfo.get.endPoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get assertEquals(advertisedHostName, endpoint.host) assertEquals(advertisedPort, endpoint.port) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 77bd378..cf3e313 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -17,7 +17,7 @@ package kafka.server -import kafka.cluster.ProtocolType +import kafka.cluster.SecurityProtocol import org.junit.Test import junit.framework.Assert._ import org.scalatest.junit.JUnit3Suite @@ -98,7 +98,7 @@ class KafkaConfigTest extends JUnit3Suite { val serverConfig = new KafkaConfig(props) val endpoints = Utils.listenerListToEndPoints(serverConfig.advertisedListeners) - val endpoint = endpoints.find(_.protocolType == ProtocolType.PLAINTEXT).get + val endpoint = endpoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get assertEquals(endpoint.host, hostName) assertEquals(endpoint.port, port) } @@ -114,7 +114,7 @@ class KafkaConfigTest extends JUnit3Suite { val serverConfig = new KafkaConfig(props) val endpoints = Utils.listenerListToEndPoints(serverConfig.advertisedListeners) - val endpoint = endpoints.find(_.protocolType == ProtocolType.PLAINTEXT).get + val endpoint = endpoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get assertEquals(endpoint.host, advertisedHostName) assertEquals(endpoint.port, advertisedPort) diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 79dff89..785b271 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -23,7 +23,7 @@ import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} -import kafka.cluster.{ProtocolType, Broker} +import kafka.cluster.{SecurityProtocol, Broker} import kafka.common.ErrorMapping import kafka.api._ diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 56a362b..faa697c 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -35,7 +35,7 @@ import kafka.server._ import kafka.producer._ import kafka.message._ import kafka.api._ -import kafka.cluster.{ProtocolType, Broker} +import kafka.cluster.{SecurityProtocol, Broker} import kafka.consumer.{KafkaStream, ConsumerConfig} import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} import kafka.common.TopicAndPartition @@ -148,7 +148,7 @@ object TestUtils extends Logging { def getBrokerListStrFromConfigs(configs: Seq[KafkaConfig]): String = { configs.map(c => { - val endpoint = Utils.listenerListToEndPoints(c.listeners).find(_.protocolType == ProtocolType.PLAINTEXT).get + val endpoint = Utils.listenerListToEndPoints(c.listeners).find(_.protocolType == SecurityProtocol.PLAINTEXT).get formatAddress(endpoint.host, endpoint.port) }).mkString(",") } -- 1.9.3 (Apple Git-50) From edf8713dc4a4c94d86a5bf33d0ff2e5c672eef1a Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 25 Dec 2014 10:31:30 -0800 Subject: [PATCH 19/41] fixing unit test after rename of ProtocolType to SecurityProtocol --- core/src/main/scala/kafka/consumer/ConsumerConfig.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 054bd62..7ebb240 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -46,7 +46,6 @@ object ConsumerConfig extends Config { val OffsetsChannelSocketTimeoutMs = 10000 val OffsetsCommitMaxRetries = 5 val OffsetsStorage = "zookeeper" - val SecurityProtocol = "PLAINTEXT" val MirrorTopicsWhitelistProp = "mirror.topics.whitelist" val MirrorTopicsBlacklistProp = "mirror.topics.blacklist" -- 1.9.3 (Apple Git-50) From 7c4a52b477c8debc807e60baf4cf9197f09e6d24 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Sat, 27 Dec 2014 12:02:34 -0800 Subject: [PATCH 20/41] Following Joe's advice, added security protocol enum on client side, and modified protocol to use ID instead of string. --- .../org/apache/kafka/clients/NetworkClient.java | 5 ++- .../kafka/clients/producer/KafkaProducer.java | 3 +- .../org/apache/kafka/common/protocol/Protocol.java | 2 +- .../kafka/common/protocol/SecurityProtocol.java | 47 ++++++++++++++++++++++ .../kafka/common/requests/MetadataRequest.java | 13 +++--- .../apache/kafka/clients/NetworkClientTest.java | 3 +- .../kafka/common/requests/RequestResponseTest.java | 3 +- .../scala/kafka/api/ConsumerMetadataRequest.scala | 6 +-- .../scala/kafka/api/ConsumerMetadataResponse.scala | 2 +- .../scala/kafka/api/TopicMetadataRequest.scala | 8 ++-- .../scala/kafka/api/TopicMetadataResponse.scala | 2 +- 11 files changed, 73 insertions(+), 21 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java 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 9a2318d..26391aa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; @@ -81,7 +82,7 @@ public class NetworkClient implements KafkaClient { private long lastNoNodeAvailableMs; /* protocol used for communication to brokers */ - private String securityProtocol; + private SecurityProtocol securityProtocol; public NetworkClient(Selectable selector, Metadata metadata, @@ -90,7 +91,7 @@ public class NetworkClient implements KafkaClient { long reconnectBackoffMs, int socketSendBuffer, int socketReceiveBuffer, - String securityProtocol) { + SecurityProtocol securityProtocol) { this.selector = selector; this.metadata = metadata; this.clientId = clientId; 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 0986726..3b7e741 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 @@ -41,6 +41,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Records; @@ -196,7 +197,7 @@ public class KafkaProducer implements Producer { config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG), config.getInt(ProducerConfig.SEND_BUFFER_CONFIG), config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG), - config.getString(ProducerConfig.SECURITY_PROTOCOL)); + SecurityProtocol.valueOf(config.getString(ProducerConfig.SECURITY_PROTOCOL))); this.sender = new Sender(client, this.metadata, this.accumulator, diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 29ce7af..f1a6db3 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -50,7 +50,7 @@ public class Protocol { public static Schema METADATA_REQUEST_V1 = new Schema(new Field("topics", new ArrayOf(STRING), "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics."), - new Field("security_protocol",STRING,"The response should contain broker endpoints that correspond to this protocol")); + new Field("security_protocol",INT16,"The response should contain broker endpoints that correspond to this protocol")); public static Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."), new Field("host", STRING, "The hostname of the broker."), diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java new file mode 100644 index 0000000..1d789ad --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.protocol; + +import java.util.HashMap; +import java.util.Map; + +public enum SecurityProtocol { + PLAINTEXT(0,"PLAINTEXT"); + + private static Map codeToSecurityProtocol = new HashMap(); + + static { + for (SecurityProtocol proto: SecurityProtocol.values()) { + codeToSecurityProtocol.put(proto.id,proto); + } + } + + /** the permenant and immutable id of a security protocol -- this can't change, and must match kafka.cluster.SecurityProtocol */ + public final short id; + + /** a name of the security protocol. This may be used by client configuration */ + public final String name; + + private SecurityProtocol(int id, String name) { + this.id = (short) id; + this.name = name; + } + + public static String getName(int id) { + return codeToSecurityProtocol.get(id).name; + } +} 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 833797e..2ad6ec6 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,7 @@ import java.util.List; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; @@ -25,13 +26,13 @@ public class MetadataRequest extends AbstractRequestResponse { private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); - public static String DEFAULT_PROTOCOL = "PLAINTEXT"; + public static Short DEFAULT_PROTOCOL = SecurityProtocol.PLAINTEXT.id; private static String TOPICS_KEY_NAME = "topics"; private static String PROTOCOL_KEY_NAME = "security_protocol"; private final List topics; - private String protocol; + private short protocol; /* Constructor for V0 */ public MetadataRequest(List topics) { @@ -41,12 +42,12 @@ public class MetadataRequest extends AbstractRequestResponse { } /* Constructor for V1 */ - public MetadataRequest(List topics, String protocol) { + public MetadataRequest(List topics, SecurityProtocol protocol) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.METADATA.id,1))); struct.set(TOPICS_KEY_NAME, topics.toArray()); - struct.set(PROTOCOL_KEY_NAME, protocol); + struct.set(PROTOCOL_KEY_NAME, protocol.id); this.topics = topics; - this.protocol = protocol; + this.protocol = protocol.id; } public MetadataRequest(Struct struct) { @@ -57,7 +58,7 @@ public class MetadataRequest extends AbstractRequestResponse { topics.add((String) topicObj); } if (struct.hasField(PROTOCOL_KEY_NAME)) { - protocol = struct.getString(PROTOCOL_KEY_NAME); + protocol = struct.getShort(PROTOCOL_KEY_NAME); } } 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 2f9a46d..604c776 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -16,6 +16,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.ProduceRequest; @@ -36,7 +37,7 @@ public class NetworkClientTest { private int nodeId = 1; private Cluster cluster = TestUtils.singletonCluster("test", nodeId); private Node node = cluster.nodes().get(0); - private NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024,"PLAINTEXT"); + private NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024, SecurityProtocol.PLAINTEXT); @Before public void setup() { 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 58b5c3e..59edcf0 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -17,6 +17,7 @@ import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.SecurityProtocol; import org.junit.Test; import java.lang.reflect.Method; @@ -124,7 +125,7 @@ public class RequestResponseTest { } private AbstractRequestResponse createMetadataRequest() { - return new MetadataRequest(Arrays.asList("topic1"), "PLAINTEXT"); + return new MetadataRequest(Arrays.asList("topic1"), SecurityProtocol.PLAINTEXT); } private AbstractRequestResponse createMetadataResponse() { diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index e08492a..edbe9e1 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -35,7 +35,7 @@ object ConsumerMetadataRequest { val clientId = ApiUtils.readShortString(buffer) // request val group = ApiUtils.readShortString(buffer) - val securityProtocol = SecurityProtocol.withName(ApiUtils.readShortString(buffer)) + val securityProtocol = SecurityProtocol(ApiUtils.readShortInRange(buffer,"security protocol id",(0,SecurityProtocol.maxId.toShort))) ConsumerMetadataRequest(group, versionId, correlationId, clientId, securityProtocol) } @@ -53,7 +53,7 @@ case class ConsumerMetadataRequest(group: String, 4 + /* correlationId */ ApiUtils.shortStringLength(clientId) + ApiUtils.shortStringLength(group) + - ApiUtils.shortStringLength(securityProtocol.toString) + 2 def writeTo(buffer: ByteBuffer) { // envelope @@ -63,7 +63,7 @@ case class ConsumerMetadataRequest(group: String, // consumer metadata request ApiUtils.writeShortString(buffer, group) - ApiUtils.writeShortString(buffer, securityProtocol.toString) + buffer.putShort(securityProtocol.id.toShort) } override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala index 861d5f7..a3587e4 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -18,7 +18,7 @@ package kafka.api import java.nio.ByteBuffer -import kafka.cluster.{SecurityProtocol, BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint import kafka.common.ErrorMapping object ConsumerMetadataResponse { diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 423eda6..7572233 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -55,9 +55,9 @@ object TopicMetadataRequest extends Logging { var securityProtocol = org.apache.kafka.common.requests.MetadataRequest.DEFAULT_PROTOCOL if (versionId == 1) { - securityProtocol = readShortString(buffer) + securityProtocol = readShortInRange(buffer,"security protocol id",(0,SecurityProtocol.maxId.toShort)) } - new TopicMetadataRequest(versionId, correlationId, clientId, SecurityProtocol.withName(securityProtocol), topics.toList) + new TopicMetadataRequest(versionId, correlationId, clientId, SecurityProtocol(securityProtocol.toInt), topics.toList) } } @@ -77,7 +77,7 @@ case class TopicMetadataRequest(val versionId: Short, writeShortString(buffer, clientId) buffer.putInt(topics.size) topics.foreach(topic => writeShortString(buffer, topic)) - writeShortString(buffer, securityProtocol.toString) + buffer.putShort(securityProtocol.id.toShort) } def sizeInBytes(): Int = { @@ -86,7 +86,7 @@ case class TopicMetadataRequest(val versionId: Short, shortStringLength(clientId) + /* client id */ 4 + /* number of topics */ topics.foldLeft(0)(_ + shortStringLength(_)) + /* topics */ - shortStringLength(securityProtocol.toString) /* security protocol */ + 2 /* security protocol */ } override def toString(): String = { diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala index c55f60c..4de566b 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala @@ -17,7 +17,7 @@ package kafka.api -import kafka.cluster.{SecurityProtocol, BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint import java.nio.ByteBuffer object TopicMetadataResponse { -- 1.9.3 (Apple Git-50) From 251cd55388a02abce1281b6918cec20c59f5ac33 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Sat, 27 Dec 2014 12:22:50 -0800 Subject: [PATCH 21/41] validate producer config against enum --- .../java/org/apache/kafka/clients/producer/ProducerConfig.java | 3 ++- .../java/org/apache/kafka/common/protocol/SecurityProtocol.java | 9 +++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index a765b45..964a243 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 @@ -23,6 +23,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.protocol.SecurityProtocol; /** * Configuration for the Kafka Producer. Documentation for these configurations can be found in the codeToSecurityProtocol = new HashMap(); + private static List names = new ArrayList(); static { for (SecurityProtocol proto: SecurityProtocol.values()) { codeToSecurityProtocol.put(proto.id,proto); + names.add(proto.name); } } @@ -44,4 +48,9 @@ public enum SecurityProtocol { public static String getName(int id) { return codeToSecurityProtocol.get(id).name; } + + public static List getNames() { + return names; + } + } -- 1.9.3 (Apple Git-50) From 8af21068f3eed78f66c2661863ff319d557132ae Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 29 Dec 2014 12:11:20 -0800 Subject: [PATCH 22/41] add a second protocol for testing and modify SocketServerTests to check on multi-ports --- .../kafka/common/protocol/SecurityProtocol.java | 5 ++- .../scala/kafka/cluster/SecurityProtocol.scala | 1 + .../unit/kafka/network/SocketServerTest.scala | 51 ++++++++++++++++++---- 3 files changed, 47 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java index 8980258..f2f69e0 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java @@ -22,7 +22,10 @@ import java.util.List; import java.util.Map; public enum SecurityProtocol { - PLAINTEXT(0,"PLAINTEXT"); + /** Currently identical to PLAINTEXT and used for testing. Plan is to add instrumentation in the future */ + TRACE(0,"TRACE"), + /** Un-authenticated, non-encrypted channel */ + PLAINTEXT(1,"PLAINTEXT"); private static Map codeToSecurityProtocol = new HashMap(); private static List names = new ArrayList(); diff --git a/core/src/main/scala/kafka/cluster/SecurityProtocol.scala b/core/src/main/scala/kafka/cluster/SecurityProtocol.scala index 82b4631..ca7105c 100644 --- a/core/src/main/scala/kafka/cluster/SecurityProtocol.scala +++ b/core/src/main/scala/kafka/cluster/SecurityProtocol.scala @@ -21,6 +21,7 @@ package kafka.cluster object SecurityProtocol extends Enumeration { type SecurityProtocol = Value + val TRACE = Value val PLAINTEXT = Value } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 1c68456..235371d 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -19,6 +19,8 @@ package kafka.network; import java.net._ import java.io._ +import kafka.cluster.SecurityProtocol +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.cluster.{SecurityProtocol, EndPoint} import org.junit._ import org.scalatest.junit.JUnitSuite @@ -35,8 +37,12 @@ import scala.collection.Map class SocketServerTest extends JUnitSuite { + val ports = kafka.utils.TestUtils.choosePorts(2) + val plaintextPort = ports.head + val tracePort = ports.last val server: SocketServer = new SocketServer(0, - List(EndPoint(null,kafka.utils.TestUtils.choosePort,SecurityProtocol.PLAINTEXT)), + List(EndPoint(null, plaintextPort, SecurityProtocol.PLAINTEXT), + EndPoint(null, tracePort, SecurityProtocol.TRACE)), numProcessorThreads = 1, maxQueuedRequests = 50, sendBufferSize = 300000, @@ -73,7 +79,13 @@ class SocketServerTest extends JUnitSuite { channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) } - def connect(s:SocketServer = server) = new Socket("localhost", server.endpoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get.port) + def connect(s:SocketServer = server) : java.net.Socket = { + connect(SecurityProtocol.PLAINTEXT) + } + + def connect(protocol: SecurityProtocol, s:SocketServer = server) = { + new Socket("localhost", server.endpoints.find(_.protocolType == protocol).get.port) + } @After def cleanup() { @@ -81,7 +93,8 @@ class SocketServerTest extends JUnitSuite { } @Test def simpleRequest() { - val socket = connect() + val plainSocket = connect(SecurityProtocol.PLAINTEXT) + val traceSocket = connect(SecurityProtocol.TRACE) val correlationId = -1 val clientId = SyncProducerConfig.DefaultClientId val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs @@ -95,9 +108,15 @@ class SocketServerTest extends JUnitSuite { val serializedBytes = new Array[Byte](byteBuffer.remaining) byteBuffer.get(serializedBytes) - sendRequest(socket, 0, serializedBytes) + // Test PLAINTEXT socket + sendRequest(plainSocket, 0, serializedBytes) + processRequest(server.requestChannel) + assertEquals(serializedBytes.toSeq, receiveResponse(plainSocket).toSeq) + + // Test TRACE socket + sendRequest(traceSocket, 0, serializedBytes) processRequest(server.requestChannel) - assertEquals(serializedBytes.toSeq, receiveResponse(socket).toSeq) + assertEquals(serializedBytes.toSeq, receiveResponse(traceSocket).toSeq) } @Test(expected = classOf[IOException]) @@ -129,18 +148,31 @@ class SocketServerTest extends JUnitSuite { "Socket key should be available for reads") } - @Test(expected = classOf[IOException]) + @Test def testSocketsCloseOnShutdown() { // open a connection - val socket = connect() + val plainSocket = connect(SecurityProtocol.PLAINTEXT) + val traceSocket = connect(SecurityProtocol.TRACE) val bytes = new Array[Byte](40) // send a request first to make sure the connection has been picked up by the socket server - sendRequest(socket, 0, bytes) + sendRequest(plainSocket, 0, bytes) + sendRequest(traceSocket, 0, bytes) processRequest(server.requestChannel) // then shutdown the server server.shutdown() // doing a subsequent send should throw an exception as the connection should be closed. - sendRequest(socket, 0, bytes) + try { + sendRequest(plainSocket, 0, bytes) + fail("expected exception when writing to closed socket") + } catch { + case e: IOException => // expected + } + try { + sendRequest(traceSocket, 0, bytes) + fail("expected exception when writing to closed socket") + } catch { + case e: IOException => // expected + } } @Test @@ -177,4 +209,5 @@ class SocketServerTest extends JUnitSuite { assertEquals(-1, conn.getInputStream.read()) overrideServer.shutdown() } + } -- 1.9.3 (Apple Git-50) From f8705d5217f3d972be1491198fe6b9679d9bf379 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 30 Dec 2014 11:25:12 -0800 Subject: [PATCH 23/41] Reverted the metadata request changes and removed the explicit security protocol argument. Instead the socketserver will determine the protocol based on the port and add this to the request --- .../org/apache/kafka/clients/NetworkClient.java | 2 +- .../org/apache/kafka/common/protocol/Protocol.java | 9 ++------ .../kafka/common/requests/MetadataRequest.java | 26 +--------------------- .../kafka/common/requests/RequestResponseTest.java | 2 +- .../scala/kafka/api/ConsumerMetadataRequest.scala | 13 +++-------- .../scala/kafka/api/TopicMetadataRequest.scala | 17 +++----------- core/src/main/scala/kafka/client/ClientUtils.scala | 2 +- .../controller/ControllerChannelManager.scala | 2 +- .../scala/kafka/controller/KafkaController.scala | 2 +- .../scala/kafka/javaapi/TopicMetadataRequest.scala | 2 +- .../main/scala/kafka/network/RequestChannel.scala | 6 +++-- .../main/scala/kafka/network/SocketServer.scala | 22 ++++++++++++++---- .../kafka/server/AbstractFetcherManager.scala | 2 +- .../scala/kafka/server/AbstractFetcherThread.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 5 ++--- core/src/main/scala/kafka/server/KafkaConfig.scala | 4 ++-- core/src/main/scala/kafka/server/KafkaServer.scala | 4 ++-- .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../api/RequestResponseSerializationTest.scala | 2 +- 19 files changed, 47 insertions(+), 79 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 26391aa..1c942e6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -441,7 +441,7 @@ public class NetworkClient implements KafkaClient { * Create a metadata request for the given topics */ private ClientRequest metadataRequest(long now, int node, Set topics) { - MetadataRequest metadata = new MetadataRequest(new ArrayList(topics),securityProtocol); + MetadataRequest metadata = new MetadataRequest(new ArrayList(topics)); RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct()); return new ClientRequest(now, true, send, null); } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index f1a6db3..7be2840 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -47,11 +47,6 @@ public class Protocol { new ArrayOf(STRING), "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics.")); - public static Schema METADATA_REQUEST_V1 = new Schema(new Field("topics", - new ArrayOf(STRING), - "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics."), - new Field("security_protocol",INT16,"The response should contain broker endpoints that correspond to this protocol")); - public static Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."), new Field("host", STRING, "The hostname of the broker."), new Field("port", INT32, "The port on which the broker accepts requests.")); @@ -82,8 +77,8 @@ public class Protocol { "Host and port information for all brokers."), new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V0))); - public static Schema[] METADATA_REQUEST = new Schema[] { METADATA_REQUEST_V0, METADATA_REQUEST_V1 }; - public static Schema[] METADATA_RESPONSE = new Schema[] { METADATA_RESPONSE_V0, METADATA_RESPONSE_V0 }; + public static Schema[] METADATA_REQUEST = new Schema[] { METADATA_REQUEST_V0}; + public static Schema[] METADATA_RESPONSE = new Schema[] { METADATA_RESPONSE_V0}; /* Produce api */ 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 2ad6ec6..cb97732 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 @@ -26,30 +26,17 @@ public class MetadataRequest extends AbstractRequestResponse { private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); - public static Short DEFAULT_PROTOCOL = SecurityProtocol.PLAINTEXT.id; private static String TOPICS_KEY_NAME = "topics"; - private static String PROTOCOL_KEY_NAME = "security_protocol"; private final List topics; - private short protocol; - /* Constructor for V0 */ public MetadataRequest(List topics) { super(new Struct(CURRENT_SCHEMA)); struct.set(TOPICS_KEY_NAME, topics.toArray()); this.topics = topics; } - /* Constructor for V1 */ - public MetadataRequest(List topics, SecurityProtocol protocol) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.METADATA.id,1))); - struct.set(TOPICS_KEY_NAME, topics.toArray()); - struct.set(PROTOCOL_KEY_NAME, protocol.id); - this.topics = topics; - this.protocol = protocol.id; - } - public MetadataRequest(Struct struct) { super(struct); Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); @@ -57,9 +44,6 @@ public class MetadataRequest extends AbstractRequestResponse { for (Object topicObj: topicArray) { topics.add((String) topicObj); } - if (struct.hasField(PROTOCOL_KEY_NAME)) { - protocol = struct.getShort(PROTOCOL_KEY_NAME); - } } public List topics() { @@ -67,15 +51,7 @@ public class MetadataRequest extends AbstractRequestResponse { } public static MetadataRequest parse(ByteBuffer buffer) { -<<<<<<< HEAD return new MetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer))); -======= - return new MetadataRequest(((Struct) curSchema.read(buffer))); } - - public static MetadataRequest parse(ByteBuffer buffer, int versionId) { - Schema schema = ProtoUtils.requestSchema(ApiKeys.METADATA.id,versionId); - return new MetadataRequest(((Struct) schema.read(buffer))); ->>>>>>> first commit of refactoring. - } + } 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 59edcf0..a4eb08b 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 @@ -125,7 +125,7 @@ public class RequestResponseTest { } private AbstractRequestResponse createMetadataRequest() { - return new MetadataRequest(Arrays.asList("topic1"), SecurityProtocol.PLAINTEXT); + return new MetadataRequest(Arrays.asList("topic1")); } private AbstractRequestResponse createMetadataResponse() { diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index edbe9e1..1e3a9f2 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -18,8 +18,6 @@ package kafka.api import java.nio.ByteBuffer -import kafka.cluster.SecurityProtocol -import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.common.ErrorMapping @@ -35,8 +33,7 @@ object ConsumerMetadataRequest { val clientId = ApiUtils.readShortString(buffer) // request val group = ApiUtils.readShortString(buffer) - val securityProtocol = SecurityProtocol(ApiUtils.readShortInRange(buffer,"security protocol id",(0,SecurityProtocol.maxId.toShort))) - ConsumerMetadataRequest(group, versionId, correlationId, clientId, securityProtocol) + ConsumerMetadataRequest(group, versionId, correlationId, clientId) } } @@ -44,16 +41,14 @@ object ConsumerMetadataRequest { case class ConsumerMetadataRequest(group: String, versionId: Short = ConsumerMetadataRequest.CurrentVersion, correlationId: Int = 0, - clientId: String = ConsumerMetadataRequest.DefaultClientId, - securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) + clientId: String = ConsumerMetadataRequest.DefaultClientId) extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey)) { def sizeInBytes = 2 + /* versionId */ 4 + /* correlationId */ ApiUtils.shortStringLength(clientId) + - ApiUtils.shortStringLength(group) + - 2 + ApiUtils.shortStringLength(group) def writeTo(buffer: ByteBuffer) { // envelope @@ -63,7 +58,6 @@ case class ConsumerMetadataRequest(group: String, // consumer metadata request ApiUtils.writeShortString(buffer, group) - buffer.putShort(securityProtocol.id.toShort) } override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { @@ -79,7 +73,6 @@ case class ConsumerMetadataRequest(group: String, consumerMetadataRequest.append("; CorrelationId: " + correlationId) consumerMetadataRequest.append("; ClientId: " + clientId) consumerMetadataRequest.append("; Group: " + group) - consumerMetadataRequest.append("; SecurityProtocol" + securityProtocol.toString) consumerMetadataRequest.toString() } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 7572233..8d21a93 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -19,9 +19,6 @@ package kafka.api import java.nio.ByteBuffer import kafka.api.ApiUtils._ -import kafka.cluster.SecurityProtocol -import kafka.cluster.SecurityProtocol.SecurityProtocol -import kafka.cluster.SecurityProtocol.SecurityProtocol import collection.mutable.ListBuffer import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping @@ -53,23 +50,18 @@ object TopicMetadataRequest extends Logging { for(i <- 0 until numTopics) topics += readShortString(buffer) - var securityProtocol = org.apache.kafka.common.requests.MetadataRequest.DEFAULT_PROTOCOL - if (versionId == 1) { - securityProtocol = readShortInRange(buffer,"security protocol id",(0,SecurityProtocol.maxId.toShort)) - } - new TopicMetadataRequest(versionId, correlationId, clientId, SecurityProtocol(securityProtocol.toInt), topics.toList) + new TopicMetadataRequest(versionId, correlationId, clientId, topics.toList) } } case class TopicMetadataRequest(val versionId: Short, val correlationId: Int, val clientId: String, - val securityProtocol: SecurityProtocol, val topics: Seq[String]) extends RequestOrResponse(Some(RequestKeys.MetadataKey)){ def this(topics: Seq[String], correlationId: Int) = - this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, SecurityProtocol.PLAINTEXT, topics) + this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics) def writeTo(buffer: ByteBuffer) { buffer.putShort(versionId) @@ -77,7 +69,6 @@ case class TopicMetadataRequest(val versionId: Short, writeShortString(buffer, clientId) buffer.putInt(topics.size) topics.foreach(topic => writeShortString(buffer, topic)) - buffer.putShort(securityProtocol.id.toShort) } def sizeInBytes(): Int = { @@ -85,8 +76,7 @@ case class TopicMetadataRequest(val versionId: Short, 4 + /* correlation id */ shortStringLength(clientId) + /* client id */ 4 + /* number of topics */ - topics.foldLeft(0)(_ + shortStringLength(_)) + /* topics */ - 2 /* security protocol */ + topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */ } override def toString(): String = { @@ -107,7 +97,6 @@ case class TopicMetadataRequest(val versionId: Short, topicMetadataRequest.append("; Version: " + versionId) topicMetadataRequest.append("; CorrelationId: " + correlationId) topicMetadataRequest.append("; ClientId: " + clientId) - topicMetadataRequest.append("; SecurityProtocol" + securityProtocol.toString) if(details) topicMetadataRequest.append("; Topics: " + topics.mkString(",")) topicMetadataRequest.toString() diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 01618c5..6e067df 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -47,7 +47,7 @@ object ClientUtils extends Logging{ def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndpoint], producerConfig: ProducerConfig, correlationId: Int): TopicMetadataResponse = { var fetchMetaDataSucceeded: Boolean = false var i: Int = 0 - val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, producerConfig.clientId, producerConfig.securityProtocol, topics.toSeq) + val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, producerConfig.clientId, topics.toSeq) var topicMetadataResponse: TopicMetadataResponse = null var t: Throwable = null // shuffle the list of brokers before sending metadata requests so that most requests don't get routed to the diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index f9fd523..1f9aa11 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -80,7 +80,7 @@ class ControllerChannelManager (private val controllerContext: ControllerContext private def addNewBroker(broker: Broker) { val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize) debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id)) - val brokerEndPoint = broker.getBrokerEndPoint(config.securityProtocol) + val brokerEndPoint = broker.getBrokerEndPoint(config.replicationSecurityProtocol) val channel = new BlockingChannel(brokerEndPoint.host, brokerEndPoint.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 4fbcdd3..1f8cb1d 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -214,7 +214,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt def clientId = { val listeners = listenerListToEndPoints(config.listeners) - val controllerListener = listeners.find(endpoint => endpoint.protocolType == config.securityProtocol) + val controllerListener = listeners.find(endpoint => endpoint.protocolType == config.replicationSecurityProtocol) "id_%d-host_%s-port_%d".format(config.brokerId, controllerListener.get.host, controllerListener.get.port) } diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index de33b04..63148d3 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -36,7 +36,7 @@ class TopicMetadataRequest(val versionId: Short, val underlying: kafka.api.TopicMetadataRequest = { import scala.collection.JavaConversions._ - new kafka.api.TopicMetadataRequest(versionId, correlationId, clientId, securityProtocol, topics: mutable.Buffer[String]) + new kafka.api.TopicMetadataRequest(versionId, correlationId, clientId, topics: mutable.Buffer[String]) } def this(topics: java.util.List[String]) = diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 7b1db3d..c4bad46 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -18,6 +18,8 @@ package kafka.network import java.util.concurrent._ +import kafka.cluster.SecurityProtocol +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge import java.nio.ByteBuffer @@ -30,7 +32,7 @@ import org.apache.log4j.Logger object RequestChannel extends Logging { - val AllDone = new Request(1, 2, getShutdownReceive(), 0) + val AllDone = new Request(processor = 1, requestKey = 2, buffer = getShutdownReceive(), startTimeMs = 0, securityProtocol = SecurityProtocol.PLAINTEXT) def getShutdownReceive() = { val emptyProducerRequest = new ProducerRequest(0, 0, "", 0, 0, collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]()) @@ -41,7 +43,7 @@ object RequestChannel extends Logging { byteBuffer } - case class Request(processor: Int, requestKey: Any, private var buffer: ByteBuffer, startTimeMs: Long, remoteAddress: SocketAddress = new InetSocketAddress(0)) { + case class Request(processor: Int, requestKey: Any, private var buffer: ByteBuffer, startTimeMs: Long, remoteAddress: SocketAddress = new InetSocketAddress(0), securityProtocol: SecurityProtocol) { @volatile var requestDequeueTimeMs = -1L @volatile var apiLocalCompleteTimeMs = -1L @volatile var responseCompleteTimeMs = -1L diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 7009030..abaab83 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -24,7 +24,8 @@ import java.net._ import java.io._ import java.nio.channels._ -import kafka.cluster.EndPoint +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.{SecurityProtocol, EndPoint} import kafka.cluster.SecurityProtocol.SecurityProtocol import scala.collection._ @@ -60,6 +61,15 @@ class SocketServer(val brokerId: Int, /* a meter to track the average free capacity of the network processors */ private val aggregateIdleMeter = newMeter("NetworkProcessorAvgIdlePercent", "percent", TimeUnit.NANOSECONDS) + + /* 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. + we'll probably have a more elegant way of doing this once we patch the request channel + to include more information about security and authentication. + TODO: re-consider this code when working on JIRA-1683 + */ + val portToProtocol = endpoints.map(ep => (ep.port -> ep.protocolType )).toMap + /** * Start the socket server */ @@ -74,7 +84,8 @@ class SocketServer(val brokerId: Int, numProcessorThreads, requestChannel, quotas, - connectionsMaxIdleMs) + connectionsMaxIdleMs, + portToProtocol) Utils.newThread("kafka-network-thread-%d".format(i), processors(i), false).start() } @@ -311,7 +322,8 @@ private[kafka] class Processor(val id: Int, val totalProcessorThreads: Int, val requestChannel: RequestChannel, connectionQuotas: ConnectionQuotas, - val connectionsMaxIdleMs: Long) extends AbstractServerThread(connectionQuotas) { + val connectionsMaxIdleMs: Long, + val portToProtocol: Map[Int,SecurityProtocol]) extends AbstractServerThread(connectionQuotas) { private val newConnections = new ConcurrentLinkedQueue[SocketChannel]() private val connectionsMaxIdleNanos = connectionsMaxIdleMs * 1000 * 1000 @@ -457,7 +469,9 @@ private[kafka] class Processor(val id: Int, if(read < 0) { close(key) } else if(receive.complete) { - val req = RequestChannel.Request(processor = id, requestKey = key, buffer = receive.buffer, startTimeMs = time.milliseconds, remoteAddress = address) + val port = address.asInstanceOf[InetSocketAddress].getPort + val protocol = portToProtocol.getOrElse(port,SecurityProtocol.PLAINTEXT) + 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 diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 1a5b0f9..94aa952 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.collection.Set import scala.collection.Map import kafka.utils.{Utils, Logging} -import kafka.cluster.{BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint import kafka.metrics.KafkaMetricsGroup import kafka.common.TopicAndPartition import com.yammer.metrics.core.Gauge diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index d770768..210369a 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -17,7 +17,7 @@ package kafka.server -import kafka.cluster.{BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint import kafka.utils.{Pool, ShutdownableThread} import kafka.consumer.{PartitionTopicInfo, SimpleConsumer} import kafka.api.{FetchRequest, FetchResponse, FetchResponsePartitionData, FetchRequestBuilder} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 83bc825..bef4a8f 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -22,6 +22,7 @@ import org.apache.kafka.common.requests.JoinGroupResponse import org.apache.kafka.common.requests.HeartbeatResponse import org.apache.kafka.common.requests.ResponseHeader import org.apache.kafka.common.protocol.types.Struct +import java.net.InetSocketAddress import kafka.api._ import kafka.common._ @@ -401,7 +402,7 @@ class KafkaApis(val requestChannel: RequestChannel, val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet) 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(metadataRequest.securityProtocol)), topicMetadata, metadataRequest.correlationId) + val response = new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(request.securityProtocol)), topicMetadata, metadataRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } @@ -440,8 +441,6 @@ class KafkaApis(val requestChannel: RequestChannel, // get metadata (and create the topic if necessary) val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName)).head - val securityProtocol = consumerMetadataRequest.securityProtocol - val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) val response = diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index c6257ed..f1de723 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -129,8 +129,8 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* idle connections timeout: the server socket processor threads close the connections that idle more than this */ val connectionsMaxIdleMs = props.getLong("connections.max.idle.ms", 10*60*1000L) - /* security protocol used to communicate between brokers */ - val securityProtocol = SecurityProtocol.withName(props.getString("security.protocol","PLAINTEXT")) + /* security protocol used to communicate between brokers. Defaults to plain text. */ + val replicationSecurityProtocol = SecurityProtocol.withName(props.getString("replication,security.protocol","PLAINTEXT")) /*********** Log Configuration ***********/ diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 63ac879..a5e2edf 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -207,8 +207,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (channel != null) { channel.disconnect() } - channel = new BlockingChannel(broker.getBrokerEndPoint(config.securityProtocol).host, - broker.getBrokerEndPoint(config.securityProtocol).port, + channel = new BlockingChannel(broker.getBrokerEndPoint(config.replicationSecurityProtocol).host, + broker.getBrokerEndPoint(config.replicationSecurityProtocol).port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index e90b5dc..8037a75 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -678,7 +678,7 @@ class ReplicaManager(val config: KafkaConfig, // we do not need to check if the leader exists again since this has been done at the beginning of this process val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map(partition => new TopicAndPartition(partition) -> BrokerAndInitialOffset( - leaders.find(_.id == partition.leaderReplicaIdOpt.get).get.getBrokerEndPoint(config.securityProtocol), + leaders.find(_.id == partition.leaderReplicaIdOpt.get).get.getBrokerEndPoint(config.replicationSecurityProtocol), partition.getReplica().get.logEndOffset.messageOffset)).toMap replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset) diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 971d055..3ae51dd 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -148,7 +148,7 @@ object SerializationTestUtils { } def createTestTopicMetadataRequest: TopicMetadataRequest = { - new TopicMetadataRequest(1, 1, "client 1", SecurityProtocol.PLAINTEXT, Seq(topic1, topic2)) + new TopicMetadataRequest(1, 1, "client 1", Seq(topic1, topic2)) } def createTestTopicMetadataResponse: TopicMetadataResponse = { -- 1.9.3 (Apple Git-50) From 62d318438384f6fa40b8206287c461eb39e518c4 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 30 Dec 2014 18:48:22 -0800 Subject: [PATCH 24/41] bump version for UpdateMetadataRequest and added support for rolling upgrades with new config --- .../scala/kafka/api/ConsumerMetadataRequest.scala | 2 +- .../scala/kafka/api/UpdateMetadataRequest.scala | 24 +++++++++++++++++----- core/src/main/scala/kafka/cluster/Broker.scala | 4 ++++ .../controller/ControllerChannelManager.scala | 6 ++++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 3 +++ .../unit/kafka/server/ServerShutdownTest.scala | 2 +- 6 files changed, 32 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index 1e3a9f2..5ca1994 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -23,7 +23,7 @@ import kafka.network.RequestChannel.Response import kafka.common.ErrorMapping object ConsumerMetadataRequest { - val CurrentVersion = 1.shortValue + val CurrentVersion = 0.shortValue val DefaultClientId = "" def readFrom(buffer: ByteBuffer) = { diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala index 530982e..66d4f5c 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -17,15 +17,15 @@ package kafka.api import java.nio.ByteBuffer +import kafka.cluster.{SecurityProtocol, BrokerEndpoint, Broker} import kafka.api.ApiUtils._ -import kafka.cluster.Broker -import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.common.{KafkaException, ErrorMapping, TopicAndPartition} import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.network.RequestChannel.Response import collection.Set object UpdateMetadataRequest { - val CurrentVersion = 0.shortValue + val CurrentVersion = 1.shortValue val IsInit: Boolean = true val NotInit: Boolean = false val DefaultAckTimeout: Int = 1000 @@ -48,7 +48,14 @@ object UpdateMetadataRequest { } val numAliveBrokers = buffer.getInt - val aliveBrokers = for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer) + + val aliveBrokers = versionId match { + case 0 => for(i <- 0 until numAliveBrokers) yield new Broker(BrokerEndpoint.readFrom(buffer)) + case 1 => for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer) + case v => throw new KafkaException( "Version " + v.toString + " is invalid for UpdateMetadataRequest. Valid versions are 0 or 1.") + } + + new UpdateMetadataRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos.toMap, aliveBrokers.toSet) } @@ -82,7 +89,14 @@ case class UpdateMetadataRequest (versionId: Short, value.writeTo(buffer) } buffer.putInt(aliveBrokers.size) - aliveBrokers.foreach(_.writeTo(buffer)) + + versionId match { + case 0 => aliveBrokers.foreach(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).writeTo(buffer)) + case 1 => aliveBrokers.foreach(_.writeTo(buffer)) + case v => throw new KafkaException( "Version " + v.toString + " is invalid for UpdateMetadataRequest. Valid versions are 0 or 1.") + } + + } def sizeInBytes(): Int = { diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index f81acfa..8786cb5 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -96,6 +96,10 @@ case class Broker(id: Int, endPoints: Seq[EndPoint]) { this(id,List(EndPoint(host,port,SecurityProtocol.PLAINTEXT))) } + def this(bep: BrokerEndpoint) = { + this(bep.id,bep.host,bep.port) + } + def writeTo(buffer: ByteBuffer) { buffer.putInt(id) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 1f9aa11..bba03f2 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -294,8 +294,10 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging updateMetadataRequestMap.foreach { m => val broker = m._1 val partitionStateInfos = m._2.toMap - val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, correlationId, clientId, - partitionStateInfos, controllerContext.liveOrShuttingDownBrokers) + + val versionId = if (controller.config.useNewWireProtocol) 1 else 0 + val updateMetadataRequest = new UpdateMetadataRequest(versionId = versionId.toShort, controllerId = controllerId, controllerEpoch = controllerEpoch, + correlationId = correlationId, clientId = clientId, partitionStateInfos = partitionStateInfos, aliveBrokers = controllerContext.liveOrShuttingDownBrokers) partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, correlationId, broker, p._1))) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index f1de723..9235592 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -132,6 +132,9 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* security protocol used to communicate between brokers. Defaults to plain text. */ val replicationSecurityProtocol = SecurityProtocol.withName(props.getString("replication,security.protocol","PLAINTEXT")) + /* allow users to bump the protocol version when they are done upgrading */ + val useNewWireProtocol = props.getBoolean("use.new.wire.protocol",false) + /*********** Log Configuration ***********/ /* the default number of log partitions per topic */ diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index ba1e48e..a615a2f 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -165,7 +165,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue(true); } catch{ - case ex => fail() + case ex: Throwable => fail() } } } -- 1.9.3 (Apple Git-50) From a551a28e4794d23e3e674dcbed4199a6ea4d3770 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 5 Jan 2015 14:23:08 -0800 Subject: [PATCH 25/41] following tests - fixed LeaderAndISR protocol and ZK registration for backward compatibility --- .../main/scala/kafka/api/LeaderAndIsrRequest.scala | 10 +++++----- core/src/main/scala/kafka/cluster/Broker.scala | 19 ++++++++++++++----- .../kafka/controller/ControllerChannelManager.scala | 2 +- .../main/scala/kafka/server/KafkaHealthcheck.scala | 7 +++++-- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- core/src/main/scala/kafka/server/ReplicaManager.scala | 6 +++--- core/src/main/scala/kafka/utils/ZkUtils.scala | 9 +++------ .../kafka/api/RequestResponseSerializationTest.scala | 2 +- .../test/scala/unit/kafka/cluster/BrokerTest.scala | 9 +++++++++ .../scala/unit/kafka/server/LeaderElectionTest.scala | 5 +++-- 10 files changed, 45 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index 4ff7e8f..bf93632 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -21,7 +21,7 @@ package kafka.api import java.nio._ import kafka.utils._ import kafka.api.ApiUtils._ -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import kafka.controller.LeaderIsrAndControllerEpoch import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping @@ -120,9 +120,9 @@ object LeaderAndIsrRequest { } val leadersCount = buffer.getInt - var leaders = Set[Broker]() + var leaders = Set[BrokerEndpoint]() for (i <- 0 until leadersCount) - leaders += Broker.readFrom(buffer) + leaders += BrokerEndpoint.readFrom(buffer) new LeaderAndIsrRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos.toMap, leaders) } @@ -134,10 +134,10 @@ case class LeaderAndIsrRequest (versionId: Short, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[(String, Int), PartitionStateInfo], - leaders: Set[Broker]) + leaders: Set[BrokerEndpoint]) extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) { - def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[Broker], controllerId: Int, + def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[BrokerEndpoint], controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String) = { this(LeaderAndIsrRequest.CurrentVersion, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos, leaders) diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 8786cb5..4c1b52c 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -39,8 +39,9 @@ object Broker { * The current JSON schema for a broker is: * {"version":1, * "host":"localhost", + * "port":9092 * "jmx_port":9999, - * "timestamp":2233345666, + * "timestamp":"2233345666", * "endpoints": [ * {"port":9092, * "protocolType":"plain"}, @@ -55,10 +56,18 @@ object Broker { Json.parseFull(brokerInfoString) match { case Some(m) => val brokerInfo = m.asInstanceOf[Map[String, Any]] - val host = brokerInfo.get("host").get.asInstanceOf[String] - val port = brokerInfo.get("port").get.asInstanceOf[Int] - val endPoints = brokerInfo.get("endpoints").get.asInstanceOf[String] - new Broker(id, Utils.listenerListToEndPoints(endPoints)) + val version = brokerInfo.get("version").get.asInstanceOf[Int] + val endpoints = version match { + case 1 => + val host = brokerInfo.get("host").get.asInstanceOf[String] + val port = brokerInfo.get("port").get.asInstanceOf[Int] + List(new EndPoint(host,port,SecurityProtocol.PLAINTEXT)) + case 2 => + val listeners = brokerInfo.get("endpoints").get.asInstanceOf[String] + Utils.listenerListToEndPoints(listeners) + case _ => throw new KafkaException("Unknown version of broker registration. Only versions 1 and 2 are supported." + brokerInfoString) + } + new Broker(id, endpoints) case None => throw new BrokerNotAvailableException("Broker id %d does not exist".format(id)) } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index bba03f2..d5c24eb 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -279,7 +279,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val broker = m._1 val partitionStateInfos = m._2.toMap val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet - val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)) + val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map(b => b.getBrokerEndPoint(controller.config.replicationSecurityProtocol)) val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId, clientId) for (p <- partitionStateInfos) { val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 00cc696..711620e 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -18,6 +18,7 @@ package kafka.server import kafka.cluster.EndPoint +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.utils._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkStateListener, ZkClient} @@ -35,7 +36,8 @@ import java.net.InetAddress class KafkaHealthcheck(private val brokerId: Int, private val advertisedEndpoints: Seq[EndPoint], private val zkSessionTimeoutMs: Int, - private val zkClient: ZkClient) extends Logging { + private val zkClient: ZkClient, + private val securityProtocol: SecurityProtocol) extends Logging { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId val sessionExpireListener = new SessionExpireListener @@ -61,7 +63,8 @@ class KafkaHealthcheck(private val brokerId: Int, else endpoint ) - ZkUtils.registerBrokerInZk(zkClient, brokerId, updatedEndpoints, zkSessionTimeoutMs, jmxPort) + val defaultEndpoint = updatedEndpoints.find(endpoint => endpoint.protocolType == securityProtocol).get + ZkUtils.registerBrokerInZk(zkClient, brokerId, updatedEndpoints, zkSessionTimeoutMs, jmxPort, defaultEndpoint.host, defaultEndpoint.port) } /** diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index a5e2edf..d090788 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -128,7 +128,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg /* tell everyone we are alive */ val endpoints = Utils.listenerListToEndPoints(config.advertisedListeners) - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId,endpoints,config.zkSessionTimeoutMs, zkClient) + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId,endpoints,config.zkSessionTimeoutMs, zkClient, config.replicationSecurityProtocol) kafkaHealthcheck.startup() registerStats() diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 8037a75..872167d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -19,7 +19,7 @@ package kafka.server import kafka.api._ import kafka.common._ import kafka.utils._ -import kafka.cluster.{Broker, Partition, Replica} +import kafka.cluster.{BrokerEndpoint, Broker, Partition, Replica} import kafka.log.{LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController @@ -610,7 +610,7 @@ class ReplicaManager(val config: KafkaConfig, * 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[Broker], correlationId: Int, responseMap: mutable.Map[(String, Int), Short], + leaders: Set[BrokerEndpoint], correlationId: Int, responseMap: mutable.Map[(String, Int), Short], offsetManager: OffsetManager) { partitionState.foreach { state => stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + @@ -678,7 +678,7 @@ class ReplicaManager(val config: KafkaConfig, // we do not need to check if the leader exists again since this has been done at the beginning of this process val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map(partition => new TopicAndPartition(partition) -> BrokerAndInitialOffset( - leaders.find(_.id == partition.leaderReplicaIdOpt.get).get.getBrokerEndPoint(config.replicationSecurityProtocol), + leaders.find(_.id == partition.leaderReplicaIdOpt.get).get, partition.getReplica().get.logEndOffset.messageOffset)).toMap replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset) diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index e0a97c0..dff8ad6 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -173,14 +173,11 @@ object ZkUtils extends Logging { } } - def registerBrokerInZk(zkClient: ZkClient, id: Int, advertisedEndpoints: Seq[EndPoint], timeout: Int, jmxPort: Int) { + def registerBrokerInZk(zkClient: ZkClient, id: Int, advertisedEndpoints: Seq[EndPoint], timeout: Int, jmxPort: Int, defaultHost: String, defaultPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val defaultEndPoint = advertisedEndpoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get - val host = defaultEndPoint.host - val port = defaultEndPoint.port - val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.mkString(","), "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val brokerInfo = Json.encode(Map("version" -> 2, "host" -> defaultHost, "port" -> defaultPort, "endpoints"->advertisedEndpoints.mkString(","), "jmx_port" -> jmxPort, "timestamp" -> timestamp)) val expectedBroker = new Broker(id, advertisedEndpoints) try { @@ -195,7 +192,7 @@ object ZkUtils extends Logging { + "else you have shutdown this broker and restarted it faster than the zookeeper " + "timeout so it appears to be re-registering.") } - info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, host, port)) + info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, defaultHost, defaultPort)) } def deregisterBrokerInZk(zkClient: ZkClient, id: Int) { diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 3ae51dd..8e7f47f 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -98,7 +98,7 @@ object SerializationTestUtils { val leaderAndIsr2 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader2, 1, isr2, 2), 1) val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, isr1.toSet)), ((topic2, 0), PartitionStateInfo(leaderAndIsr2, isr2.toSet))) - new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 0, 1, 0, "") + new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[BrokerEndpoint](), 0, 1, 0, "") } def createTestLeaderAndIsrResponse() : LeaderAndIsrResponse = { diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index 7a30f3f..f68bd31 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -80,6 +80,15 @@ class BrokerTest extends JUnit3Suite with Logging { } @Test + def testFromOldJSON() = { + val brokerInfoStr = "{\"jmx_port\":-1,\"timestamp\":\"1420485325400\",\"host\":\"172.16.8.243\",\"version\":1,\"port\":9091}" + val broker = Broker.createBroker(1, brokerInfoStr) + assert(broker.id == 1) + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host == "172.16.8.243") + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port == 9091) + } + + @Test def testBrokerEndpointFromURI() = { var connectionString = "localhost:9092" var endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 785b271..61b593f 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -23,7 +23,7 @@ import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} -import kafka.cluster.{SecurityProtocol, Broker} +import kafka.cluster.{BrokerEndpoint, SecurityProtocol, Broker} import kafka.common.ErrorMapping import kafka.api._ @@ -119,6 +119,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val controllerId = 2 val controllerConfig = new KafkaConfig(TestUtils.createBrokerConfig(controllerId, TestUtils.choosePort())) val brokers = servers.map(s => new Broker(s.config.brokerId, Utils.listenerListToEndPoints(s.config.listeners))) + val brokerEndPoints = brokers.map(b => b.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) val controllerContext = new ControllerContext(zkClient, 6000) controllerContext.liveBrokers = brokers.toSet val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) @@ -128,7 +129,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { leaderAndIsr.put((topic, partitionId), new LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId2, List(brokerId1, brokerId2)), 2)) val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, Set(0,1))).toMap - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, controllerId, + val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokerEndPoints.toSet, controllerId, staleControllerEpoch, 0, "") controllerChannelManager.sendRequest(brokerId2, leaderAndIsrRequest, staleControllerEpochCallback) -- 1.9.3 (Apple Git-50) From 49293a26e4f46f7691d65de50d712ef2dc37fce3 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 5 Jan 2015 20:25:01 -0800 Subject: [PATCH 26/41] cleaned up some changes that were actually not necessary. hopefully making this patch slightly easier to review --- clients/src/main/java/org/apache/kafka/clients/NetworkClient.java | 7 +------ .../main/java/org/apache/kafka/clients/producer/KafkaProducer.java | 4 +--- .../java/org/apache/kafka/clients/producer/ProducerConfig.java | 6 ++---- .../src/test/java/org/apache/kafka/clients/NetworkClientTest.java | 3 +-- core/src/main/scala/kafka/cluster/Broker.scala | 5 +---- core/src/test/scala/unit/kafka/utils/TestUtils.scala | 2 +- 6 files changed, 7 insertions(+), 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 1c942e6..ec9dc05 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -81,17 +81,13 @@ public class NetworkClient implements KafkaClient { /* the last timestamp when no broker node is available to connect */ private long lastNoNodeAvailableMs; - /* protocol used for communication to brokers */ - private SecurityProtocol securityProtocol; - public NetworkClient(Selectable selector, Metadata metadata, String clientId, int maxInFlightRequestsPerConnection, long reconnectBackoffMs, int socketSendBuffer, - int socketReceiveBuffer, - SecurityProtocol securityProtocol) { + int socketReceiveBuffer) { this.selector = selector; this.metadata = metadata; this.clientId = clientId; @@ -103,7 +99,6 @@ public class NetworkClient implements KafkaClient { this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE); this.metadataFetchInProgress = false; this.lastNoNodeAvailableMs = 0; - this.securityProtocol = securityProtocol; } /** 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 3b7e741..ebc4c53 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 @@ -41,7 +41,6 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.Selector; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Records; @@ -196,8 +195,7 @@ public class KafkaProducer implements Producer { config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG), config.getInt(ProducerConfig.SEND_BUFFER_CONFIG), - config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG), - SecurityProtocol.valueOf(config.getString(ProducerConfig.SECURITY_PROTOCOL))); + config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG)); this.sender = new Sender(client, this.metadata, this.accumulator, 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 964a243..bc6cd91 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 @@ -23,7 +23,6 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; -import org.apache.kafka.common.protocol.SecurityProtocol; /** * Configuration for the Kafka Producer. Documentation for these configurations can be found in the security.protocol */ public static final String SECURITY_PROTOCOL = "security.protocol"; - private static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT is supported (and is the default). SSL and Kerberos are planned for the near future. " + - " Details on Kafka's security plans are available on our wiki: https://cwiki.apache.org/confluence/display/KAFKA/Security"; + private static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT is supported. SSL and Kerberos are planned for the near future"; static { config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) @@ -223,7 +221,7 @@ public class ProducerConfig extends AbstractConfig { .define(SECURITY_PROTOCOL, Type.STRING, "PLAINTEXT", - in(SecurityProtocol.getNames()), + in(Arrays.asList("PLAINTEXT")), Importance.MEDIUM, SECURITY_PROTOCOL_DOC); } 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 604c776..5debcd6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.ProduceRequest; @@ -37,7 +36,7 @@ public class NetworkClientTest { private int nodeId = 1; private Cluster cluster = TestUtils.singletonCluster("test", nodeId); private Node node = cluster.nodes().get(0); - private NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024, SecurityProtocol.PLAINTEXT); + private NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024); @Before public void setup() { diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 4c1b52c..cd7e511 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -138,9 +138,6 @@ case class Broker(id: Int, endPoints: Seq[EndPoint]) { } - - - /* TODO: Unit test! */ override def equals(obj: Any): Boolean = { obj match { case null => false @@ -150,7 +147,7 @@ case class Broker(id: Int, endPoints: Seq[EndPoint]) { } } - /* TODO: Unit test! */ + override def hashCode(): Int = hashcode(id, endPoints) } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index faa697c..1350f8f 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -450,7 +450,7 @@ object TestUtils extends Logging { def createBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { val brokers = ids.map(id => new Broker(id, "localhost", 6667)) - brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.endPoints, 6000, jmxPort = -1)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.endPoints, 6000, jmxPort = -1,"localhost",6667)) brokers } -- 1.9.3 (Apple Git-50) From 0430e33c2a89579d6385b6f2b74fcef406e1c171 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 5 Jan 2015 21:19:12 -0800 Subject: [PATCH 27/41] undoing some changes that don't belong here --- clients/src/main/java/org/apache/kafka/clients/NetworkClient.java | 1 - .../java/org/apache/kafka/clients/producer/ProducerConfig.java | 1 + .../src/main/java/org/apache/kafka/common/protocol/Protocol.java | 7 +++---- .../java/org/apache/kafka/common/requests/MetadataRequest.java | 6 +----- .../java/org/apache/kafka/common/requests/RequestResponseTest.java | 1 - core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala | 1 + core/src/main/scala/kafka/consumer/ConsumerConfig.scala | 4 ---- core/src/main/scala/kafka/producer/ProducerConfig.scala | 4 ---- core/src/main/scala/kafka/server/MetadataCache.scala | 1 - core/src/main/scala/kafka/tools/GetOffsetShell.scala | 4 +--- core/src/test/resources/log4j.properties | 4 ++-- core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala | 2 +- 12 files changed, 10 insertions(+), 26 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index ec9dc05..5950191 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -27,7 +27,6 @@ import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; 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 bc6cd91..5bdc2e9 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 @@ -157,6 +157,7 @@ public class ProducerConfig extends AbstractConfig { private static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC = "The maximum number of unacknowledged requests the client will send on a single connection before blocking." + " Note that if this setting is set to be greater than 1 and there are failed sends, there is a risk of" + " message re-ordering due to retries (i.e., if retries are enabled)."; + /** key.serializer */ public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; private static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 7be2840..7517b87 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -51,8 +51,7 @@ public class Protocol { new Field("host", STRING, "The hostname of the broker."), new Field("port", INT32, "The port on which the broker accepts requests.")); - - public static Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code", + public static Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code", INT16, "The error code for the partition, if any."), new Field("partition_id", INT32, "The id of the partition."), @@ -77,8 +76,8 @@ public class Protocol { "Host and port information for all brokers."), new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V0))); - public static Schema[] METADATA_REQUEST = new Schema[] { METADATA_REQUEST_V0}; - public static Schema[] METADATA_RESPONSE = new Schema[] { METADATA_RESPONSE_V0}; + public static Schema[] METADATA_REQUEST = new Schema[] { METADATA_REQUEST_V0 }; + public static Schema[] METADATA_RESPONSE = new Schema[] { METADATA_RESPONSE_V0 }; /* Produce api */ 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 cb97732..225afe5 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,17 +18,14 @@ import java.util.List; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; public class MetadataRequest extends AbstractRequestResponse { private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); private static String TOPICS_KEY_NAME = "topics"; - private final List topics; public MetadataRequest(List topics) { @@ -52,6 +49,5 @@ public class MetadataRequest extends AbstractRequestResponse { public static MetadataRequest parse(ByteBuffer buffer) { return new MetadataRequest(((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 a4eb08b..df37fc6 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -17,7 +17,6 @@ import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.junit.Test; import java.lang.reflect.Method; diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index 5ca1994..a3b1b78 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -31,6 +31,7 @@ object ConsumerMetadataRequest { val versionId = buffer.getShort val correlationId = buffer.getInt val clientId = ApiUtils.readShortString(buffer) + // request val group = ApiUtils.readShortString(buffer) ConsumerMetadataRequest(group, versionId, correlationId, clientId) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 7ebb240..9ebbee6 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -19,7 +19,6 @@ package kafka.consumer import java.util.Properties import kafka.api.OffsetRequest -import kafka.cluster.SecurityProtocol import kafka.utils._ import kafka.common.{InvalidConfigException, Config} @@ -181,9 +180,6 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin */ val partitionAssignmentStrategy = props.getString("partition.assignment.strategy", DefaultPartitionAssignmentStrategy) - - /* plaintext or SSL */ - val securityProtocol = SecurityProtocol.withName(props.getString("security.protocol", "PLAINTEXT")) validate(this) } diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index 9aff401..3cdf23d 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -19,7 +19,6 @@ package kafka.producer import async.AsyncProducerConfig import java.util.Properties -import kafka.cluster.SecurityProtocol import kafka.utils.{Utils, VerifiableProperties} import kafka.message.{CompressionCodec, NoCompressionCodec} import kafka.common.{InvalidConfigException, Config} @@ -114,8 +113,5 @@ class ProducerConfig private (val props: VerifiableProperties) */ val topicMetadataRefreshIntervalMs = props.getInt("topic.metadata.refresh.interval.ms", 600000) - /* plaintext or SSL */ - val securityProtocol = SecurityProtocol.withName(props.getString("security.protocol", "PLAINTEXT")) - validate(this) } diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index cb96edb..c709017 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -18,7 +18,6 @@ package kafka.server import kafka.cluster.SecurityProtocol.SecurityProtocol -import kafka.cluster.SecurityProtocol.SecurityProtocol import scala.collection.{Seq, Set, mutable} import kafka.api._ diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 4f0b463..3d9293e 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -24,7 +24,6 @@ import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} import kafka.common.TopicAndPartition import kafka.client.ClientUtils import kafka.utils.{ToolsUtils, CommandLineUtils} -import kafka.cluster.SecurityProtocol object GetOffsetShell { @@ -94,8 +93,7 @@ object GetOffsetShell { case Some(metadata) => metadata.leader match { case Some(leader) => - val consumer = new SimpleConsumer(leader.host, - leader.port, 10000, 100000, clientId) + val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, clientId) val topicAndPartition = TopicAndPartition(topic, partitionId) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 1883f97..1b7d5d8 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -18,8 +18,8 @@ log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=TRACE -log4j.logger.org.apache.kafka=TRACE +log4j.logger.kafka=ERROR +log4j.logger.org.apache.kafka=ERROR # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index a615a2f..ba1e48e 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -165,7 +165,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue(true); } catch{ - case ex: Throwable => fail() + case ex => fail() } } } -- 1.9.3 (Apple Git-50) From 23b5af8eb0f3f6c2cdc2b229ee1dda028cce6b6a Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 5 Jan 2015 21:40:03 -0800 Subject: [PATCH 28/41] bring back config lost in cleanup --- core/src/main/scala/kafka/consumer/ConsumerConfig.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 9ebbee6..7ebb240 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -19,6 +19,7 @@ package kafka.consumer import java.util.Properties import kafka.api.OffsetRequest +import kafka.cluster.SecurityProtocol import kafka.utils._ import kafka.common.{InvalidConfigException, Config} @@ -180,6 +181,9 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin */ val partitionAssignmentStrategy = props.getString("partition.assignment.strategy", DefaultPartitionAssignmentStrategy) + + /* plaintext or SSL */ + val securityProtocol = SecurityProtocol.withName(props.getString("security.protocol", "PLAINTEXT")) validate(this) } -- 1.9.3 (Apple Git-50) From a169b37e08db556c0d0afed3ea0090a5923f3307 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 6 Jan 2015 11:44:51 -0800 Subject: [PATCH 29/41] fixes neccessary for an all non-plaintext cluster to work --- core/src/main/scala/kafka/api/TopicMetadataRequest.scala | 14 +++----------- core/src/main/scala/kafka/network/SocketServer.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 9 +++++---- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- core/src/main/scala/kafka/server/MetadataCache.scala | 2 +- 5 files changed, 11 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 8d21a93..7dca09c 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -26,30 +26,22 @@ import kafka.network.RequestChannel.Response import kafka.utils.Logging object TopicMetadataRequest extends Logging { - val CurrentVersion = 1.shortValue + val CurrentVersion = 0.shortValue val DefaultClientId = "" /** * TopicMetadataRequest has the following format - - * security protocol (2 bytes + protocol.length) - * number of topics (4 bytes) - * list of topics (2 bytes + topic.length per topic) - * detailedMetadata (2 bytes) - * timestamp (8 bytes) - * count (4 bytes) + * number of topics (4 bytes) list of topics (2 bytes + topic.length per topic) detailedMetadata (2 bytes) timestamp (8 bytes) count (4 bytes) */ def readFrom(buffer: ByteBuffer): TopicMetadataRequest = { val versionId = buffer.getShort - assert(versionId == 0 || versionId == 1, - "Version " + versionId + " is invalid for TopicMetadataRequest. Valid versions are 0 or 1.") val correlationId = buffer.getInt val clientId = readShortString(buffer) val numTopics = readIntInRange(buffer, "number of topics", (0, Int.MaxValue)) val topics = new ListBuffer[String]() for(i <- 0 until numTopics) topics += readShortString(buffer) - new TopicMetadataRequest(versionId, correlationId, clientId, topics.toList) } } @@ -76,7 +68,7 @@ case class TopicMetadataRequest(val versionId: Short, 4 + /* correlation id */ shortStringLength(clientId) + /* client id */ 4 + /* number of topics */ - topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */ + topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */ } override def toString(): String = { diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index abaab83..19f55cb 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -469,7 +469,7 @@ private[kafka] class Processor(val id: Int, if(read < 0) { close(key) } else if(receive.complete) { - val port = address.asInstanceOf[InetSocketAddress].getPort + val port = socketChannel.socket().getLocalPort val protocol = portToProtocol.getOrElse(port,SecurityProtocol.PLAINTEXT) val req = RequestChannel.Request(processor = id, requestKey = key, buffer = receive.buffer, startTimeMs = time.milliseconds, remoteAddress = address, securityProtocol = protocol) requestChannel.sendRequest(req) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index bef4a8f..4b1c450 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.protocol.types.Struct import java.net.InetSocketAddress import kafka.api._ +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.common._ import kafka.log._ import kafka.network._ @@ -356,8 +357,8 @@ class KafkaApis(val requestChannel: RequestChannel, ret.toSeq.sortBy(- _) } - private def getTopicMetadata(topics: Set[String]): Seq[TopicMetadata] = { - val topicResponses = metadataCache.getTopicMetadata(topics) + private def getTopicMetadata(topics: Set[String],securityProtocol: SecurityProtocol): Seq[TopicMetadata] = { + val topicResponses = metadataCache.getTopicMetadata(topics,securityProtocol) if (topics.size > 0 && topicResponses.size != topics.size) { val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet val responsesForNonExistentTopics = nonExistentTopics.map { topic => @@ -399,7 +400,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] - val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet) + val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet,request.securityProtocol) 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) @@ -439,7 +440,7 @@ class KafkaApis(val requestChannel: RequestChannel, val partition = offsetManager.partitionFor(consumerMetadataRequest.group) // get metadata (and create the topic if necessary) - val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName)).head + val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName),request.securityProtocol).head val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 9235592..c0a0427 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -130,7 +130,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val connectionsMaxIdleMs = props.getLong("connections.max.idle.ms", 10*60*1000L) /* security protocol used to communicate between brokers. Defaults to plain text. */ - val replicationSecurityProtocol = SecurityProtocol.withName(props.getString("replication,security.protocol","PLAINTEXT")) + val replicationSecurityProtocol = SecurityProtocol.withName(props.getString("replication.security.protocol","PLAINTEXT")) /* allow users to bump the protocol version when they are done upgrading */ val useNewWireProtocol = props.getBoolean("use.new.wire.protocol",false) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index c709017..664b5a2 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -38,7 +38,7 @@ private[server] class MetadataCache { private var aliveBrokers: Map[Int, Broker] = Map() private val partitionMetadataLock = new ReentrantReadWriteLock() - def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { + def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol) = { val isAllTopics = topics.isEmpty val topicsRequested = if(isAllTopics) cache.keySet else topics val topicResponses: mutable.ListBuffer[TopicMetadata] = new mutable.ListBuffer[TopicMetadata] -- 1.9.3 (Apple Git-50) From d27b11ae18d6f949dc02a69f31852afaca32e6e6 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Wed, 7 Jan 2015 18:49:23 -0800 Subject: [PATCH 30/41] minor modifications following comments by Jun --- core/src/main/scala/kafka/client/ClientUtils.scala | 2 +- core/src/main/scala/kafka/cluster/Broker.scala | 4 ---- core/src/main/scala/kafka/cluster/EndPoint.scala | 13 ++++++++----- .../src/main/scala/kafka/javaapi/TopicMetadataRequest.scala | 9 ++------- core/src/main/scala/kafka/network/BlockingChannel.scala | 5 +---- core/src/main/scala/kafka/network/SocketServer.scala | 2 +- core/src/main/scala/kafka/producer/ProducerPool.scala | 3 +-- .../integration/kafka/api/ProducerFailureHandlingTest.scala | 2 +- core/src/test/scala/unit/kafka/cluster/BrokerTest.scala | 4 ++++ 9 files changed, 19 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 6e067df..4963a0d 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -98,7 +98,7 @@ object ClientUtils extends Logging{ /** * Parse a list of broker urls in the form host1:port1, host2:port2, ... */ - def parseBrokerList(brokerListStr: String, protocolType: SecurityProtocol = SecurityProtocol.PLAINTEXT): Seq[BrokerEndpoint] = { + def parseBrokerList(brokerListStr: String): Seq[BrokerEndpoint] = { val brokersStr = Utils.parseCsvList(brokerListStr) brokersStr.zipWithIndex.map { case (address, brokerId) => diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index cd7e511..8215b3d 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -76,10 +76,6 @@ object Broker { } } - def noBroker() = { - Broker(-1,List()) - } - /** * * @param buffer containing serialized broker diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index 364b2f7..3ed7fdc 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -28,8 +28,8 @@ object EndPoint { def readFrom(buffer: ByteBuffer): EndPoint = { val port = buffer.getInt() val host = readShortString(buffer) - val channel = readShortString(buffer) - EndPoint(host,port,SecurityProtocol.withName(channel)) + val protocol = buffer.getShort() + EndPoint(host,port,SecurityProtocol(protocol)) } def createEndPoint(connectionString: String): EndPoint = { @@ -47,16 +47,19 @@ object EndPoint { */ case class EndPoint(host: String, port: Int, protocolType: SecurityProtocol) { - override def toString: String = protocolType + "://" + host + ":" + port + override def toString: String = { + val hostStr = if (host == null || host.contains(":")) "[" + host + "]" else host + protocolType + "://" + hostStr + ":" + port + } def writeTo(buffer: ByteBuffer): Unit = { buffer.putInt(port) writeShortString(buffer,host) - writeShortString(buffer,protocolType.toString) + buffer.putShort(protocolType.id.toShort) } def sizeInBytes: Int = 4 + /* port */ shortStringLength(host) + - shortStringLength(protocolType.toString) + 2 /* protocol id */ } diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index 63148d3..b0b7be1 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -18,10 +18,6 @@ package kafka.javaapi import kafka.api._ import java.nio.ByteBuffer -import kafka.cluster.SecurityProtocol -import kafka.cluster.SecurityProtocol.SecurityProtocol -import kafka.cluster.SecurityProtocol.SecurityProtocol - import scala.collection.mutable import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping @@ -30,7 +26,6 @@ import kafka.network.RequestChannel.Response class TopicMetadataRequest(val versionId: Short, val correlationId: Int, val clientId: String, - val securityProtocol: SecurityProtocol, val topics: java.util.List[String]) extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) { @@ -40,10 +35,10 @@ class TopicMetadataRequest(val versionId: Short, } def this(topics: java.util.List[String]) = - this(kafka.api.TopicMetadataRequest.CurrentVersion, 0, kafka.api.TopicMetadataRequest.DefaultClientId, SecurityProtocol.PLAINTEXT, topics) + this(kafka.api.TopicMetadataRequest.CurrentVersion, 0, kafka.api.TopicMetadataRequest.DefaultClientId, topics) def this(topics: java.util.List[String], correlationId: Int) = - this(kafka.api.TopicMetadataRequest.CurrentVersion, correlationId, kafka.api.TopicMetadataRequest.DefaultClientId, SecurityProtocol.PLAINTEXT, topics) + this(kafka.api.TopicMetadataRequest.CurrentVersion, correlationId, kafka.api.TopicMetadataRequest.DefaultClientId, topics) def writeTo(buffer: ByteBuffer) = underlying.writeTo(buffer) diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala index 70be9c1..6e2a38e 100644 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ b/core/src/main/scala/kafka/network/BlockingChannel.scala @@ -72,10 +72,7 @@ class BlockingChannel( val host: String, connectTimeoutMs)) } catch { - case e: Throwable => { - error(e) - disconnect() - } + case e: Throwable => disconnect() } } } diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 19f55cb..e661aeb 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -68,7 +68,7 @@ class SocketServer(val brokerId: Int, to include more information about security and authentication. TODO: re-consider this code when working on JIRA-1683 */ - val portToProtocol = endpoints.map(ep => (ep.port -> ep.protocolType )).toMap + private val portToProtocol = endpoints.map(ep => (ep.port -> ep.protocolType )).toMap /** * Start the socket server diff --git a/core/src/main/scala/kafka/producer/ProducerPool.scala b/core/src/main/scala/kafka/producer/ProducerPool.scala index 1ea8752..362623f 100644 --- a/core/src/main/scala/kafka/producer/ProducerPool.scala +++ b/core/src/main/scala/kafka/producer/ProducerPool.scala @@ -48,8 +48,7 @@ class ProducerPool(val config: ProducerConfig) extends Logging { topicMetadata.foreach(tmd => { tmd.partitionsMetadata.foreach(pmd => { if(pmd.leader.isDefined) { - val endpoint = pmd.leader.get - newBrokers += BrokerEndpoint(pmd.leader.get.id, endpoint.host,endpoint.port) + newBrokers += pmd.leader.get } }) }) diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 50601ee..cf7be2b 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -277,7 +277,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { server.shutdown() server.awaitShutdown() server.startup - Thread.sleep(5000) + Thread.sleep(2000) } // Make sure the producer do not see any exception diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index f68bd31..79e7109 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -107,20 +107,24 @@ class BrokerTest extends JUnit3Suite with Logging { var endpoint = EndPoint.createEndPoint(connectionString) assert(endpoint.host == "localhost") assert(endpoint.port == 9092) + assert(endpoint.toString == "PLAINTEXT://localhost:9092") // also test for default bind connectionString = "PLAINTEXT://:9092" endpoint = EndPoint.createEndPoint(connectionString) assert(endpoint.host == null) assert(endpoint.port == 9092) + assert(endpoint.toString == "PLAINTEXT://[null]:9092") // also test for ipv6 connectionString = "PLAINTEXT://[::1]:9092" endpoint = EndPoint.createEndPoint(connectionString) assert(endpoint.host == "::1") assert(endpoint.port == 9092) + assert(endpoint.toString == "PLAINTEXT://[::1]:9092") } + } -- 1.9.3 (Apple Git-50) From 2dea2a438b1e3f440f0c1c06c902aa6a3d0833d2 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Wed, 7 Jan 2015 18:56:47 -0800 Subject: [PATCH 31/41] added missing license --- core/src/main/scala/kafka/cluster/BrokerEndPoint.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala index 7e6825a..00f5f2b 100644 --- a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -1,3 +1,19 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package kafka.cluster import java.nio.ByteBuffer -- 1.9.3 (Apple Git-50) From 069cbb63ec7b734bd7847d0cf0d005a6bb95e911 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Wed, 7 Jan 2015 19:16:22 -0800 Subject: [PATCH 32/41] formatting --- .../kafka/common/protocol/SecurityProtocol.java | 6 ++--- core/src/main/scala/kafka/client/ClientUtils.scala | 4 +-- core/src/main/scala/kafka/cluster/Broker.scala | 12 ++++----- .../main/scala/kafka/cluster/BrokerEndPoint.scala | 8 +++--- core/src/main/scala/kafka/cluster/EndPoint.scala | 8 +++--- .../kafka/consumer/ConsumerFetcherManager.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 8 +++--- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- .../test/scala/unit/kafka/cluster/BrokerTest.scala | 30 +++++++++++----------- .../scala/unit/kafka/integration/FetcherTest.scala | 2 +- .../unit/kafka/integration/TopicMetadataTest.scala | 6 ++--- .../unit/kafka/server/AdvertiseBrokerTest.scala | 2 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 4 +-- 13 files changed, 47 insertions(+), 47 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java index f2f69e0..677605c 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java @@ -23,16 +23,16 @@ import java.util.Map; public enum SecurityProtocol { /** Currently identical to PLAINTEXT and used for testing. Plan is to add instrumentation in the future */ - TRACE(0,"TRACE"), + TRACE(0, "TRACE"), /** Un-authenticated, non-encrypted channel */ - PLAINTEXT(1,"PLAINTEXT"); + PLAINTEXT(1, "PLAINTEXT"); private static Map codeToSecurityProtocol = new HashMap(); private static List names = new ArrayList(); static { for (SecurityProtocol proto: SecurityProtocol.values()) { - codeToSecurityProtocol.put(proto.id,proto); + codeToSecurityProtocol.put(proto.id, proto); names.add(proto.name); } } diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 4963a0d..6fca531 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -102,7 +102,7 @@ object ClientUtils extends Logging{ val brokersStr = Utils.parseCsvList(brokerListStr) brokersStr.zipWithIndex.map { case (address, brokerId) => - BrokerEndpoint.createBrokerEndPoint(brokerId,address) + BrokerEndpoint.createBrokerEndPoint(brokerId, address) } } @@ -113,7 +113,7 @@ object ClientUtils extends Logging{ var channel: BlockingChannel = null var connected = false while (!connected) { - val allBrokers = getAllBrokerEndPointsForChannel(zkClient,protocolType) + val allBrokers = getAllBrokerEndPointsForChannel(zkClient, protocolType) Random.shuffle(allBrokers).find { broker => trace("Connecting to broker %s:%d.".format(broker.host, broker.port)) try { diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 8215b3d..60f521d 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -61,7 +61,7 @@ object Broker { case 1 => val host = brokerInfo.get("host").get.asInstanceOf[String] val port = brokerInfo.get("port").get.asInstanceOf[Int] - List(new EndPoint(host,port,SecurityProtocol.PLAINTEXT)) + List(new EndPoint(host, port, SecurityProtocol.PLAINTEXT)) case 2 => val listeners = brokerInfo.get("endpoints").get.asInstanceOf[String] Utils.listenerListToEndPoints(listeners) @@ -87,7 +87,7 @@ object Broker { val id = buffer.getInt val numEndpoints = buffer.getInt - val endpoints = List.range(0,numEndpoints).map(i => EndPoint.readFrom(buffer)) + val endpoints = List.range(0, numEndpoints).map(i => EndPoint.readFrom(buffer)) new Broker(id, endpoints) } @@ -98,11 +98,11 @@ case class Broker(id: Int, endPoints: Seq[EndPoint]) { override def toString: String = id + " : " + endPoints.mkString("(",",",")") def this(id: Int, host: String, port: Int) = { - this(id,List(EndPoint(host,port,SecurityProtocol.PLAINTEXT))) + this(id, List(EndPoint(host, port, SecurityProtocol.PLAINTEXT))) } def this(bep: BrokerEndpoint) = { - this(bep.id,bep.host,bep.port) + this(bep.id, bep.host, bep.port) } @@ -124,9 +124,9 @@ case class Broker(id: Int, endPoints: Seq[EndPoint]) { } def getBrokerEndPoint(protocolType: SecurityProtocol): BrokerEndpoint = { - val endpoint = endPoints.map((endpoint)=>(endpoint.protocolType,endpoint)).toMap.get(protocolType) + val endpoint = endPoints.map((endpoint)=>(endpoint.protocolType, endpoint)).toMap.get(protocolType) endpoint match { - case Some(endpoint) => new BrokerEndpoint(id,endpoint.host,endpoint.port) + case Some(endpoint) => new BrokerEndpoint(id, endpoint.host, endpoint.port) case None => throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id)) } diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala index 00f5f2b..e37e75e 100644 --- a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -28,7 +28,7 @@ object BrokerEndpoint { val uriParseExp = """.*?\[?([0-9a-z\-.:]*)\]?:([0-9]+)""".r connectionString match { - case uriParseExp(host,port) => new BrokerEndpoint(brokerId,host,port.toInt) + case uriParseExp(host, port) => new BrokerEndpoint(brokerId, host, port.toInt) case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") } } @@ -46,7 +46,7 @@ object BrokerEndpoint { val brokerId = buffer.getInt() val host = readShortString(buffer) val port = buffer.getInt() - BrokerEndpoint(brokerId,host,port) + BrokerEndpoint(brokerId, host, port) } } @@ -55,11 +55,11 @@ object BrokerEndpoint { // This is not a broker and is not stored in ZooKeeper case class BrokerEndpoint(id: Int, host: String, port: Int) { - def connectionString(): String = formatAddress(host,port) + def connectionString(): String = formatAddress(host, port) def writeTo(buffer: ByteBuffer): Unit = { buffer.putInt(id) - writeShortString(buffer,host) + writeShortString(buffer, host) buffer.putInt(port) } diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index 3ed7fdc..012d1fd 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -29,14 +29,14 @@ object EndPoint { val port = buffer.getInt() val host = readShortString(buffer) val protocol = buffer.getShort() - EndPoint(host,port,SecurityProtocol(protocol)) + EndPoint(host, port, SecurityProtocol(protocol)) } def createEndPoint(connectionString: String): EndPoint = { val uriParseExp = """^(.*)://\[?([0-9a-z\-.:]*)\]?:([0-9]+)""".r connectionString match { - case uriParseExp(protocol,"",port) => new EndPoint(null,port.toInt,SecurityProtocol.withName(protocol)) - case uriParseExp(protocol,host,port) => new EndPoint(host,port.toInt,SecurityProtocol.withName(protocol)) + case uriParseExp(protocol, "", port) => new EndPoint(null, port.toInt, SecurityProtocol.withName(protocol)) + case uriParseExp(protocol, host, port) => new EndPoint(host, port.toInt, SecurityProtocol.withName(protocol)) case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") } } @@ -54,7 +54,7 @@ case class EndPoint(host: String, port: Int, protocolType: SecurityProtocol) { def writeTo(buffer: ByteBuffer): Unit = { buffer.putInt(port) - writeShortString(buffer,host) + writeShortString(buffer, host) buffer.putShort(protocolType.id.toShort) } diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index 8c7accc..d05d55c 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -62,7 +62,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, } trace("Partitions without leader %s".format(noLeaderPartitionSet)) - val brokers = getAllBrokerEndPointsForChannel(zkClient,config.securityProtocol) + val brokers = getAllBrokerEndPointsForChannel(zkClient, config.securityProtocol) val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, brokers, config.clientId, diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4b1c450..409da74 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -357,8 +357,8 @@ class KafkaApis(val requestChannel: RequestChannel, ret.toSeq.sortBy(- _) } - private def getTopicMetadata(topics: Set[String],securityProtocol: SecurityProtocol): Seq[TopicMetadata] = { - val topicResponses = metadataCache.getTopicMetadata(topics,securityProtocol) + private def getTopicMetadata(topics: Set[String], securityProtocol: SecurityProtocol): Seq[TopicMetadata] = { + val topicResponses = metadataCache.getTopicMetadata(topics, securityProtocol) if (topics.size > 0 && topicResponses.size != topics.size) { val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet val responsesForNonExistentTopics = nonExistentTopics.map { topic => @@ -400,7 +400,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] - val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet,request.securityProtocol) + val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet, request.securityProtocol) 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) @@ -440,7 +440,7 @@ class KafkaApis(val requestChannel: RequestChannel, val partition = offsetManager.partitionFor(consumerMetadataRequest.group) // get metadata (and create the topic if necessary) - val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName),request.securityProtocol).head + val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.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 d090788..302d03c 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -128,7 +128,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg /* tell everyone we are alive */ val endpoints = Utils.listenerListToEndPoints(config.advertisedListeners) - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId,endpoints,config.zkSessionTimeoutMs, zkClient, config.replicationSecurityProtocol) + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, endpoints, config.zkSessionTimeoutMs, zkClient, config.replicationSecurityProtocol) kafkaHealthcheck.startup() registerStats() diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index 79e7109..69cf9b4 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -30,9 +30,9 @@ class BrokerTest extends JUnit3Suite with Logging { @Test def testSerDe() = { - val endpoint = new EndPoint("myhost",9092,SecurityProtocol.PLAINTEXT) + val endpoint = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) val listEndPoints = List(endpoint) - val origBroker = new Broker(1,listEndPoints) + val origBroker = new Broker(1, listEndPoints) val brokerBytes = ByteBuffer.allocate(origBroker.sizeInBytes) origBroker.writeTo(brokerBytes) @@ -43,14 +43,14 @@ class BrokerTest extends JUnit3Suite with Logging { @Test def testHashAndEquals() = { - val endpoint1 = new EndPoint("myhost",9092,SecurityProtocol.PLAINTEXT) - val endpoint2 = new EndPoint("myhost",9092,SecurityProtocol.PLAINTEXT) - val endpoint3 = new EndPoint("myhost",1111,SecurityProtocol.PLAINTEXT) - val endpoint4 = new EndPoint("other",1111,SecurityProtocol.PLAINTEXT) - val broker1 = new Broker(1,List(endpoint1)) - val broker2 = new Broker(1,List(endpoint2)) - val broker3 = new Broker(2,List(endpoint3)) - val broker4 = new Broker(1,List(endpoint4)) + val endpoint1 = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) + val endpoint2 = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) + val endpoint3 = new EndPoint("myhost", 1111, SecurityProtocol.PLAINTEXT) + val endpoint4 = new EndPoint("other", 1111, SecurityProtocol.PLAINTEXT) + val broker1 = new Broker(1, List(endpoint1)) + val broker2 = new Broker(1, List(endpoint2)) + val broker3 = new Broker(2, List(endpoint3)) + val broker4 = new Broker(1, List(endpoint4)) assert(broker1 == broker2) assert(broker1 != broker3) @@ -59,9 +59,9 @@ class BrokerTest extends JUnit3Suite with Logging { assert(broker1.hashCode() != broker3.hashCode()) assert(broker1.hashCode() != broker4.hashCode()) - val hashmap = new mutable.HashMap[Broker,Int]() - hashmap.put(broker1,1) - assert(hashmap.getOrElse(broker1,-1) == 1) + val hashmap = new mutable.HashMap[Broker, Int]() + hashmap.put(broker1, 1) + assert(hashmap.getOrElse(broker1, -1) == 1) } @Test @@ -91,12 +91,12 @@ class BrokerTest extends JUnit3Suite with Logging { @Test def testBrokerEndpointFromURI() = { var connectionString = "localhost:9092" - var endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) + var endpoint = BrokerEndpoint.createBrokerEndPoint(1, connectionString) assert(endpoint.host == "localhost") assert(endpoint.port == 9092) // also test for ipv6 connectionString = "[::1]:9092" - endpoint = BrokerEndpoint.createBrokerEndPoint(1,connectionString) + endpoint = BrokerEndpoint.createBrokerEndPoint(1, connectionString) assert(endpoint.host == "::1") assert(endpoint.port == 9092) } diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index ab139ca..9da447c 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -39,7 +39,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { yield new KafkaConfig(props) val messages = new mutable.HashMap[Int, Seq[Array[Byte]]] val topic = "topic" - val cluster = new Cluster(configs.map(c => new Broker(c.brokerId,Utils.listenerListToEndPoints(c.listeners)))) + val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, Utils.listenerListToEndPoints(c.listeners)))) val shutdown = ZookeeperConsumerConnector.shutdownCommand val queue = new LinkedBlockingQueue[FetchedDataChunk] val topicInfos = configs.map(c => new PartitionTopicInfo(topic, diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 025a349..3e6f9cd 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -67,7 +67,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { val topic = "test" createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokerEndPoints,"TopicMetadataTest-testBasicTopicMetadata", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", 2000,0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) @@ -106,7 +106,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { def testAutoCreateTopic { // auto create topic val topic = "testAutoCreateTopic" - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokerEndPoints,"TopicMetadataTest-testAutoCreateTopic", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testAutoCreateTopic", 2000,0).topicsMetadata assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode) assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) @@ -118,7 +118,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0) // retry the metadata for the auto created topic - topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokerEndPoints,"TopicMetadataTest-testBasicTopicMetadata", + topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", 2000,0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala index d503cbb..fa5c19e 100644 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -32,7 +32,7 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) - props.put("advertised.listeners",SecurityProtocol.PLAINTEXT.toString+"://"+advertisedHostName+":"+advertisedPort.toString) + props.put("advertised.listeners", SecurityProtocol.PLAINTEXT.toString+"://"+advertisedHostName+":"+advertisedPort.toString) server = TestUtils.createServer(new KafkaConfig(props)) } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 1350f8f..280cd24 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -160,7 +160,7 @@ object TestUtils extends Logging { enableControlledShutdown: Boolean = true): Properties = { val props = new Properties if (nodeId >= 0) props.put("broker.id", nodeId.toString) - props.put("listeners","PLAINTEXT://localhost:"+port.toString) + props.put("listeners", "PLAINTEXT://localhost:"+port.toString) props.put("log.dir", TestUtils.tempDir().getAbsolutePath) props.put("zookeeper.connect", TestZKUtils.zookeeperConnect) props.put("replica.socket.timeout.ms", "1500") @@ -450,7 +450,7 @@ object TestUtils extends Logging { def createBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { val brokers = ids.map(id => new Broker(id, "localhost", 6667)) - brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.endPoints, 6000, jmxPort = -1,"localhost",6667)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.endPoints, 6000, jmxPort = -1, "localhost", 6667)) brokers } -- 1.9.3 (Apple Git-50) From 3f69ac98197845f38172db043d5d6e5ad2a142b6 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Wed, 7 Jan 2015 20:02:37 -0800 Subject: [PATCH 33/41] clean up imports --- core/src/main/scala/kafka/admin/AdminUtils.scala | 4 ---- core/src/main/scala/kafka/api/TopicMetadata.scala | 3 +-- core/src/main/scala/kafka/client/ClientUtils.scala | 14 ++++++-------- core/src/main/scala/kafka/cluster/BrokerEndPoint.scala | 1 - .../main/scala/kafka/consumer/ConsumerFetcherManager.scala | 3 +-- .../main/scala/kafka/consumer/ConsumerFetcherThread.scala | 2 +- .../scala/kafka/javaapi/ConsumerMetadataResponse.scala | 2 +- core/src/main/scala/kafka/javaapi/TopicMetadata.scala | 2 +- core/src/main/scala/kafka/network/SocketServer.scala | 1 - core/src/main/scala/kafka/server/MetadataCache.scala | 2 +- .../main/scala/kafka/server/ReplicaFetcherManager.scala | 2 +- .../src/main/scala/kafka/server/ReplicaFetcherThread.scala | 2 +- core/src/main/scala/kafka/server/ReplicaManager.scala | 2 +- .../main/scala/kafka/tools/ReplicaVerificationTool.scala | 2 +- core/src/main/scala/kafka/tools/SimpleConsumerShell.scala | 2 +- core/src/main/scala/kafka/utils/Utils.scala | 1 - .../kafka/api/ProducerFailureHandlingTest.scala | 4 +--- .../test/scala/unit/kafka/admin/AddPartitionsTest.scala | 2 +- .../scala/unit/kafka/consumer/ConsumerIteratorTest.scala | 2 +- .../scala/unit/kafka/integration/TopicMetadataTest.scala | 2 +- .../test/scala/unit/kafka/network/SocketServerTest.scala | 1 - .../test/scala/unit/kafka/server/LeaderElectionTest.scala | 2 +- core/src/test/scala/unit/kafka/utils/TestUtils.scala | 1 - 23 files changed, 22 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 1f18872..50f7bdd 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -18,9 +18,6 @@ package kafka.admin import kafka.common._ -import java.util.Random -import java.util.Properties -import kafka.api.{TopicMetadata, PartitionMetadata} import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.cluster.{BrokerEndpoint, Broker, SecurityProtocol} @@ -30,7 +27,6 @@ import kafka.api.{TopicMetadata, PartitionMetadata} import java.util.Random import java.util.Properties -import scala.Some import scala.Predef._ import scala.collection._ import mutable.ListBuffer diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index 9aa9119..76f8bc9 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -17,12 +17,11 @@ package kafka.api -import kafka.cluster.{BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.utils.Logging import kafka.common._ -import org.apache.kafka.common.utils.Utils._ object TopicMetadata { diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 6fca531..614d4b8 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -16,21 +16,19 @@ */ package kafka.client - import kafka.cluster.SecurityProtocol.SecurityProtocol - - import scala.collection._ +import scala.collection._ import kafka.cluster._ +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.api._ import kafka.producer._ import kafka.common.{ErrorMapping, KafkaException} import kafka.utils.{Utils, Logging} import java.util.Properties import util.Random - import kafka.network.BlockingChannel - import kafka.utils.ZkUtils._ - import org.I0Itec.zkclient.ZkClient - import java.io.IOException -import org.apache.kafka.common.utils.Utils.{getHost, getPort} +import kafka.network.BlockingChannel +import kafka.utils.ZkUtils._ +import org.I0Itec.zkclient.ZkClient +import java.io.IOException /** * Helper functions common to clients (producer, consumer, or admin) diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala index e37e75e..7b9198b 100644 --- a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -21,7 +21,6 @@ import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.common.KafkaException import org.apache.kafka.common.utils.Utils._ -import kafka.cluster.SecurityProtocol._ object BrokerEndpoint { def createBrokerEndPoint(brokerId: Int, connectionString: String): BrokerEndpoint = { diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index d05d55c..59e9876 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -19,9 +19,8 @@ package kafka.consumer import org.I0Itec.zkclient.ZkClient import kafka.server.{BrokerAndInitialOffset, AbstractFetcherThread, AbstractFetcherManager} -import kafka.cluster.{BrokerEndpoint, Cluster, Broker} +import kafka.cluster.{BrokerEndpoint, Cluster} import scala.collection.immutable -import scala.collection.Map import collection.mutable.HashMap import scala.collection.mutable import java.util.concurrent.locks.ReentrantLock diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 7e3816d..a51a678 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -17,7 +17,7 @@ package kafka.consumer -import kafka.cluster.{BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint import kafka.server.AbstractFetcherThread import kafka.message.ByteBufferMessageSet import kafka.api.{Request, OffsetRequest, FetchResponsePartitionData} diff --git a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala index b5c4289..2449750 100644 --- a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala @@ -17,7 +17,7 @@ package kafka.javaapi -import kafka.cluster.{BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadataResponse) { diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala index 24defbc..ebbd589 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala @@ -16,7 +16,7 @@ */ package kafka.javaapi -import kafka.cluster.{BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint import scala.collection.JavaConversions private[javaapi] object MetadataListImplicits { diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index e661aeb..c383662 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -24,7 +24,6 @@ import java.net._ import java.io._ import java.nio.channels._ -import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.cluster.{SecurityProtocol, EndPoint} import kafka.cluster.SecurityProtocol.SecurityProtocol diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 664b5a2..aff86e3 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -21,7 +21,7 @@ import kafka.cluster.SecurityProtocol.SecurityProtocol import scala.collection.{Seq, Set, mutable} import kafka.api._ -import kafka.cluster.{BrokerEndpoint, SecurityProtocol, Broker} +import kafka.cluster.{BrokerEndpoint, Broker} import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.utils.Utils._ import kafka.common.{ErrorMapping, ReplicaNotAvailableException, LeaderNotAvailableException} diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala index 4d0f998..f0a2a5b 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala @@ -17,7 +17,7 @@ package kafka.server -import kafka.cluster.{BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val replicaMgr: ReplicaManager) extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId, diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index b48e65e..b155cd1 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -18,7 +18,7 @@ package kafka.server import kafka.admin.AdminUtils -import kafka.cluster.{BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint import kafka.log.LogConfig import kafka.message.ByteBufferMessageSet import kafka.api.{OffsetRequest, FetchResponsePartitionData} diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 872167d..4685de9 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -19,7 +19,7 @@ package kafka.server import kafka.api._ import kafka.common._ import kafka.utils._ -import kafka.cluster.{BrokerEndpoint, Broker, Partition, Replica} +import kafka.cluster.{BrokerEndpoint, Partition, Replica} import kafka.log.{LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index ca06c1e..d1050b4 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -18,7 +18,7 @@ package kafka.tools import joptsimple.OptionParser -import kafka.cluster.{BrokerEndpoint, Broker} +import kafka.cluster.BrokerEndpoint import kafka.message.{MessageSet, MessageAndOffset, ByteBufferMessageSet} import java.util.concurrent.CountDownLatch import java.util.concurrent.atomic.AtomicReference diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index 84c09c2..7379fe3 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -22,7 +22,7 @@ import kafka.utils._ import kafka.consumer._ import kafka.client.ClientUtils import kafka.api.{OffsetRequest, FetchRequestBuilder, Request} -import kafka.cluster.{BrokerEndpoint, SecurityProtocol, Broker} +import kafka.cluster.BrokerEndpoint import scala.collection.JavaConversions._ import kafka.common.TopicAndPartition diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 0a96c0f..1eae83a 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -31,7 +31,6 @@ import java.util.Properties import kafka.common.KafkaException import kafka.common.KafkaStorageException import kafka.cluster.EndPoint -import kafka.cluster.SecurityProtocol /** diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index cf7be2b..56e18ca 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -25,12 +25,10 @@ import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} import kafka.api.FetchRequestBuilder import kafka.common.Topic -import kafka.consumer.SimpleConsumer import kafka.server.KafkaConfig import kafka.integration.KafkaServerTestHarness -import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils,Utils} +import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils, Utils} import kafka.consumer.SimpleConsumer -import kafka.utils.Utils import org.apache.kafka.common.KafkaException import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasException} diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 7118b40..f0b4135 100644 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -22,7 +22,7 @@ import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} -import kafka.cluster.{SecurityProtocol, EndPoint, Broker} +import kafka.cluster.{SecurityProtocol, Broker} import kafka.client.ClientUtils import kafka.server.{KafkaConfig, KafkaServer} diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index e48dc0b..a77ab49 100644 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -29,7 +29,7 @@ import kafka.utils.TestUtils._ import kafka.utils._ import org.junit.Test import kafka.serializer._ -import kafka.cluster.{SecurityProtocol, EndPoint, Broker, Cluster} +import kafka.cluster.{Broker, Cluster} import org.scalatest.junit.JUnit3Suite import kafka.integration.KafkaServerTestHarness diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 3e6f9cd..66aedda 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -22,7 +22,7 @@ import kafka.zk.ZooKeeperTestHarness import kafka.admin.AdminUtils import java.nio.ByteBuffer import junit.framework.Assert._ -import kafka.cluster.{SecurityProtocol, EndPoint, Broker} +import kafka.cluster.{SecurityProtocol, Broker} import kafka.utils.{Utils, TestUtils} import kafka.utils.TestUtils._ import kafka.server.{KafkaServer, KafkaConfig} diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 235371d..d19ea71 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -19,7 +19,6 @@ package kafka.network; import java.net._ import java.io._ -import kafka.cluster.SecurityProtocol import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.cluster.{SecurityProtocol, EndPoint} import org.junit._ diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 61b593f..aecf1f8 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -23,7 +23,7 @@ import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} -import kafka.cluster.{BrokerEndpoint, SecurityProtocol, Broker} +import kafka.cluster.{SecurityProtocol, Broker} import kafka.common.ErrorMapping import kafka.api._ diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 280cd24..1b0adc6 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -24,7 +24,6 @@ import java.nio.channels._ import java.util.Random import java.util.Properties -import kafka.utils import org.apache.kafka.common.utils.Utils._ import collection.mutable.ListBuffer -- 1.9.3 (Apple Git-50) From 461d7301a7571641f3ce605e3b80901cc137d9b4 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 8 Jan 2015 17:51:37 -0800 Subject: [PATCH 34/41] cleaned up V2 to not include host+port field. Using use.new.protocol flag to decide which version to serialize --- core/src/main/scala/kafka/cluster/Broker.scala | 38 +++++++++++-------- .../main/scala/kafka/server/KafkaHealthcheck.scala | 15 +++++--- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 43 +++++++++++++++++++--- .../test/scala/unit/kafka/cluster/BrokerTest.scala | 3 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 2 +- 6 files changed, 72 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 60f521d..ef9d3c6 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -33,22 +33,28 @@ import kafka.cluster.SecurityProtocol._ object Broker { /** - * Create a broker object from id and JSON string - * @param id - * @param brokerInfoString - * The current JSON schema for a broker is: - * {"version":1, - * "host":"localhost", - * "port":9092 - * "jmx_port":9999, - * "timestamp":"2233345666", - * "endpoints": [ - * {"port":9092, - * "protocolType":"plain"}, - * {"port":9093, - * "protocolType":"ssl"}] - * @return - */ + * Create a broker object from id and JSON string + * @param id + * @param brokerInfoString + * + * Version 1 JSON schema for a broker is: + * {"version":1, + * "host":"localhost", + * "port":9092 + * "jmx_port":9999, + * "timestamp":"2233345666" } + * + * The current JSON schema for a broker is: + * {"version":2, + * "jmx_port":9999, + * "timestamp":"2233345666", + * "endpoints": [ + * {"port":9092, + * "protocolType":"plain"}, + * {"port":9093, + * "protocolType":"ssl"}]} + * @return + */ def createBroker(id: Int, brokerInfoString: String): Broker = { if(brokerInfoString == null) throw new BrokerNotAvailableException("Broker id %s does not exist".format(id)) diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 711620e..ae51400 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -17,8 +17,7 @@ package kafka.server -import kafka.cluster.EndPoint -import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.{SecurityProtocol, EndPoint} import kafka.utils._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkStateListener, ZkClient} @@ -37,7 +36,7 @@ class KafkaHealthcheck(private val brokerId: Int, private val advertisedEndpoints: Seq[EndPoint], private val zkSessionTimeoutMs: Int, private val zkClient: ZkClient, - private val securityProtocol: SecurityProtocol) extends Logging { + private val useNewWriteProtocol: Boolean) extends Logging { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId val sessionExpireListener = new SessionExpireListener @@ -63,8 +62,14 @@ class KafkaHealthcheck(private val brokerId: Int, else endpoint ) - val defaultEndpoint = updatedEndpoints.find(endpoint => endpoint.protocolType == securityProtocol).get - ZkUtils.registerBrokerInZk(zkClient, brokerId, updatedEndpoints, zkSessionTimeoutMs, jmxPort, defaultEndpoint.host, defaultEndpoint.port) + + if (useNewWriteProtocol) { + ZkUtils.registerBrokerInZk(zkClient, brokerId, updatedEndpoints, zkSessionTimeoutMs, jmxPort) + } else { + // before we upgrade to the new protocol, only PLAINTEXT is supported + val defaultEndpoint = updatedEndpoints.find(endpoint => endpoint.protocolType == SecurityProtocol.PLAINTEXT).get + ZkUtils.registerBrokerInZk(zkClient, brokerId, defaultEndpoint.host, defaultEndpoint.port, zkSessionTimeoutMs, jmxPort) + } } /** diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 302d03c..2f44a4c 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -128,7 +128,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg /* tell everyone we are alive */ val endpoints = Utils.listenerListToEndPoints(config.advertisedListeners) - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, endpoints, config.zkSessionTimeoutMs, zkClient, config.replicationSecurityProtocol) + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, endpoints, config.zkSessionTimeoutMs, zkClient, config.useNewWireProtocol) kafkaHealthcheck.startup() registerStats() diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index dff8ad6..10ac320 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -173,13 +173,45 @@ object ZkUtils extends Logging { } } - def registerBrokerInZk(zkClient: ZkClient, id: Int, advertisedEndpoints: Seq[EndPoint], timeout: Int, jmxPort: Int, defaultHost: String, defaultPort: Int) { + /** + * Register brokers with v2 json format (which includes multiple endpoints) + * @param zkClient + * @param id + * @param advertisedEndpoints + * @param timeout + * @param jmxPort + */ + def registerBrokerInZk(zkClient: ZkClient, id: Int, advertisedEndpoints: Seq[EndPoint], timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 2, "host" -> defaultHost, "port" -> defaultPort, "endpoints"->advertisedEndpoints.mkString(","), "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val brokerInfo = Json.encode(Map("version" -> 2, "endpoints"->advertisedEndpoints.mkString(","), "jmx_port" -> jmxPort, "timestamp" -> timestamp)) val expectedBroker = new Broker(id, advertisedEndpoints) + registerBrokerInZk(zkClient, brokerIdPath, brokerInfo, expectedBroker, timeout) + + info("Registered broker %d at path %s with addresses: %s".format(id, brokerIdPath, advertisedEndpoints.mkString(","))) + } + + /** + * Register brokers with v1 json format - single host and port pair + * @param zkClient + * @param id + * @param timeout + * @param jmxPort + */ + def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, timeout: Int, jmxPort: Int) { + val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id + val timestamp = SystemTime.milliseconds.toString + val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val expectedBroker = new Broker(id, List(new EndPoint(host, port, SecurityProtocol.PLAINTEXT))) + + registerBrokerInZk(zkClient, brokerIdPath, brokerInfo, expectedBroker, timeout) + + info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, host, port)) + } + + def registerBrokerInZk(zkClient: ZkClient, brokerIdPath: String, brokerInfo: String, expectedBroker: Broker, timeout: Int) { try { createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, (brokerString: String, broker: Any) => Broker.createBroker(broker.asInstanceOf[Broker].id, brokerString).equals(broker.asInstanceOf[Broker]), @@ -188,11 +220,10 @@ object ZkUtils extends Logging { } catch { case e: ZkNodeExistsException => throw new RuntimeException("A broker is already registered on the path " + brokerIdPath - + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " - + "else you have shutdown this broker and restarted it faster than the zookeeper " - + "timeout so it appears to be re-registering.") + + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " + + "else you have shutdown this broker and restarted it faster than the zookeeper " + + "timeout so it appears to be re-registering.") } - info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, defaultHost, defaultPort)) } def deregisterBrokerInZk(zkClient: ZkClient, id: Int) { diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index 69cf9b4..9830c07 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -66,8 +66,7 @@ class BrokerTest extends JUnit3Suite with Logging { @Test def testFromJSON() = { - //val brokerInfoStr = "{\"jmx_port\":-1,\"timestamp\":\"1416974968782\",\"host\":\"kafkaf-4.ent.cloudera.com\",\"version\":1,\"port\":9092}" - val brokerInfoStr = "{\"version\":1," + + val brokerInfoStr = "{\"version\":2," + "\"host\":\"localhost\"," + "\"port\":9092," + "\"jmx_port\":9999," + diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 1b0adc6..837413c 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -449,7 +449,7 @@ object TestUtils extends Logging { def createBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { val brokers = ids.map(id => new Broker(id, "localhost", 6667)) - brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.endPoints, 6000, jmxPort = -1, "localhost", 6667)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.endPoints, 6000, jmxPort = -1)) brokers } -- 1.9.3 (Apple Git-50) From c536c0d928308a91c4f0784f2f9545566d587e64 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Fri, 9 Jan 2015 11:50:43 -0800 Subject: [PATCH 35/41] change endpoints collection in Broker to Map[protocol, endpoint], mostly to be clear that we intend to have one endpoint per protocol --- .../scala/kafka/api/UpdateMetadataRequest.scala | 2 +- core/src/main/scala/kafka/cluster/Broker.scala | 24 +++++++++++----------- .../scala/kafka/controller/KafkaController.scala | 2 +- .../main/scala/kafka/network/SocketServer.scala | 15 +++++++------- .../main/scala/kafka/server/KafkaHealthcheck.scala | 7 ++++--- core/src/main/scala/kafka/utils/Utils.scala | 6 ++++-- core/src/main/scala/kafka/utils/ZkUtils.scala | 6 +++--- .../kafka/api/ProducerFailureHandlingTest.scala | 4 ++-- .../integration/kafka/api/ProducerSendTest.scala | 4 ++-- .../api/RequestResponseSerializationTest.scala | 6 +++--- .../test/scala/unit/kafka/cluster/BrokerTest.scala | 10 ++++----- .../unit/kafka/network/SocketServerTest.scala | 7 ++++--- .../unit/kafka/producer/SyncProducerTest.scala | 16 +++++++-------- .../unit/kafka/server/AdvertiseBrokerTest.scala | 2 +- .../scala/unit/kafka/server/KafkaConfigTest.scala | 4 ++-- .../test/scala/unit/kafka/utils/TestUtils.scala | 6 +++--- 16 files changed, 63 insertions(+), 58 deletions(-) diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala index 66d4f5c..cef3ec9 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -50,7 +50,7 @@ object UpdateMetadataRequest { val numAliveBrokers = buffer.getInt val aliveBrokers = versionId match { - case 0 => for(i <- 0 until numAliveBrokers) yield new Broker(BrokerEndpoint.readFrom(buffer)) + case 0 => for(i <- 0 until numAliveBrokers) yield new Broker(BrokerEndpoint.readFrom(buffer),SecurityProtocol.PLAINTEXT) case 1 => for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer) case v => throw new KafkaException( "Version " + v.toString + " is invalid for UpdateMetadataRequest. Valid versions are 0 or 1.") } diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index ef9d3c6..759ef49 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -67,7 +67,7 @@ object Broker { case 1 => val host = brokerInfo.get("host").get.asInstanceOf[String] val port = brokerInfo.get("port").get.asInstanceOf[Int] - List(new EndPoint(host, port, SecurityProtocol.PLAINTEXT)) + Map(SecurityProtocol.PLAINTEXT -> new EndPoint(host, port, SecurityProtocol.PLAINTEXT)) case 2 => val listeners = brokerInfo.get("endpoints").get.asInstanceOf[String] Utils.listenerListToEndPoints(listeners) @@ -94,28 +94,28 @@ object Broker { val numEndpoints = buffer.getInt val endpoints = List.range(0, numEndpoints).map(i => EndPoint.readFrom(buffer)) - + .map(ep => ep.protocolType -> ep).toMap new Broker(id, endpoints) } } -case class Broker(id: Int, endPoints: Seq[EndPoint]) { +case class Broker(id: Int, endPoints: Map[SecurityProtocol, EndPoint]) { - override def toString: String = id + " : " + endPoints.mkString("(",",",")") + override def toString: String = id + " : " + endPoints.values.mkString("(",",",")") - def this(id: Int, host: String, port: Int) = { - this(id, List(EndPoint(host, port, SecurityProtocol.PLAINTEXT))) + def this(id: Int, host: String, port: Int, protocol: SecurityProtocol) = { + this(id, Map(protocol -> EndPoint(host, port, protocol))) } - def this(bep: BrokerEndpoint) = { - this(bep.id, bep.host, bep.port) + def this(bep: BrokerEndpoint, protocol: SecurityProtocol) = { + this(bep.id, bep.host, bep.port, protocol) } def writeTo(buffer: ByteBuffer) { buffer.putInt(id) buffer.putInt(endPoints.size) - for(endpoint <- endPoints) { + for(endpoint <- endPoints.values) { endpoint.writeTo(buffer) } } @@ -123,14 +123,14 @@ case class Broker(id: Int, endPoints: Seq[EndPoint]) { def sizeInBytes: Int = 4 + /* broker id*/ 4 + /* number of endPoints */ - endPoints.map(_.sizeInBytes).sum /* end points */ + endPoints.values.map(_.sizeInBytes).sum /* end points */ def supportsChannel(protocolType: SecurityProtocol): Unit = { - endPoints.map((endpoint)=>(endpoint.protocolType)).contains(protocolType) + endPoints.contains(protocolType) } def getBrokerEndPoint(protocolType: SecurityProtocol): BrokerEndpoint = { - val endpoint = endPoints.map((endpoint)=>(endpoint.protocolType, endpoint)).toMap.get(protocolType) + val endpoint = endPoints.get(protocolType) endpoint match { case Some(endpoint) => new BrokerEndpoint(id, endpoint.host, endpoint.port) case None => diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 1f8cb1d..619bcbb 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -214,7 +214,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt def clientId = { val listeners = listenerListToEndPoints(config.listeners) - val controllerListener = listeners.find(endpoint => endpoint.protocolType == config.replicationSecurityProtocol) + val controllerListener = listeners.get(config.replicationSecurityProtocol) "id_%d-host_%s-port_%d".format(config.brokerId, controllerListener.get.host, controllerListener.get.port) } diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index c383662..bdd4400 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -42,7 +42,7 @@ import com.yammer.metrics.core.{Gauge, Meter} * M Handler threads that handle requests and produce responses back to the processor threads for writing. */ class SocketServer(val brokerId: Int, - val endpoints: Seq[EndPoint], + val endpoints: Map[SecurityProtocol, EndPoint], val numProcessorThreads: Int, val maxQueuedRequests: Int, val sendBufferSize: Int, @@ -67,7 +67,8 @@ class SocketServer(val brokerId: Int, to include more information about security and authentication. TODO: re-consider this code when working on JIRA-1683 */ - private val portToProtocol = endpoints.map(ep => (ep.port -> ep.protocolType )).toMap + private val portToProtocol: Map[Int, SecurityProtocol] = + endpoints.map{ case (protocol: SecurityProtocol, endpoint: EndPoint) => (endpoint.port -> protocol )} /** * Start the socket server @@ -75,12 +76,12 @@ class SocketServer(val brokerId: Int, def startup() { val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides) for(i <- 0 until numProcessorThreads) { - processors(i) = new Processor(i, - time, - maxRequestSize, + processors(i) = new Processor(i, + time, + maxRequestSize, aggregateIdleMeter, newMeter("IdlePercent", "percent", TimeUnit.NANOSECONDS, Map("networkProcessor" -> i.toString)), - numProcessorThreads, + numProcessorThreads, requestChannel, quotas, connectionsMaxIdleMs, @@ -99,7 +100,7 @@ class SocketServer(val brokerId: Int, // right now we will use the same processors for all ports, since we didn't implement different protocols // in the future, we may implement different processors for SSL and Kerberos - endpoints.foreach(endpoint => { + endpoints.values.foreach(endpoint => { val acceptor = new Acceptor(endpoint.host, endpoint.port, processors, sendBufferSize, recvBufferSize, quotas) acceptors.put(endpoint,acceptor) Utils.newThread("kafka-socket-acceptor-%s-%d".format(endpoint.protocolType.toString, endpoint.port), acceptor, false).start() diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index ae51400..be07ffd 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -18,6 +18,7 @@ package kafka.server import kafka.cluster.{SecurityProtocol, EndPoint} +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.utils._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkStateListener, ZkClient} @@ -33,7 +34,7 @@ import java.net.InetAddress * we are dead. */ class KafkaHealthcheck(private val brokerId: Int, - private val advertisedEndpoints: Seq[EndPoint], + private val advertisedEndpoints: Map[SecurityProtocol, EndPoint], private val zkSessionTimeoutMs: Int, private val zkClient: ZkClient, private val useNewWriteProtocol: Boolean) extends Logging { @@ -56,7 +57,7 @@ class KafkaHealthcheck(private val brokerId: Int, */ def register() { val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt - val updatedEndpoints = advertisedEndpoints.map(endpoint => + val updatedEndpoints = advertisedEndpoints.mapValues(endpoint => if (endpoint.host == null || endpoint.host.trim.isEmpty) EndPoint(InetAddress.getLocalHost.getCanonicalHostName, endpoint.port, endpoint.protocolType) else @@ -67,7 +68,7 @@ class KafkaHealthcheck(private val brokerId: Int, ZkUtils.registerBrokerInZk(zkClient, brokerId, updatedEndpoints, zkSessionTimeoutMs, jmxPort) } else { // before we upgrade to the new protocol, only PLAINTEXT is supported - val defaultEndpoint = updatedEndpoints.find(endpoint => endpoint.protocolType == SecurityProtocol.PLAINTEXT).get + val defaultEndpoint = updatedEndpoints.get(SecurityProtocol.PLAINTEXT).get ZkUtils.registerBrokerInZk(zkClient, brokerId, defaultEndpoint.host, defaultEndpoint.port, zkSessionTimeoutMs, jmxPort) } } diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 1eae83a..b2dec5c 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -25,6 +25,8 @@ import java.util.concurrent.locks.{ReadWriteLock, Lock} import java.lang.management._ import javax.management._ +import kafka.cluster.SecurityProtocol.SecurityProtocol + import scala.collection._ import scala.collection.mutable import java.util.Properties @@ -610,8 +612,8 @@ object Utils extends Logging { .keys } - def listenerListToEndPoints(listeners: String): Seq[EndPoint] = { + def listenerListToEndPoints(listeners: String): immutable.Map[SecurityProtocol, EndPoint] = { val listenerList = parseCsvList(listeners) - listenerList.map(listener => EndPoint.createEndPoint(listener)) + listenerList.map(listener => EndPoint.createEndPoint(listener)).map(ep => ep.protocolType -> ep).toMap } } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 10ac320..6d682ac 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -181,11 +181,11 @@ object ZkUtils extends Logging { * @param timeout * @param jmxPort */ - def registerBrokerInZk(zkClient: ZkClient, id: Int, advertisedEndpoints: Seq[EndPoint], timeout: Int, jmxPort: Int) { + def registerBrokerInZk(zkClient: ZkClient, id: Int, advertisedEndpoints: immutable.Map[SecurityProtocol, EndPoint], timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 2, "endpoints"->advertisedEndpoints.mkString(","), "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val brokerInfo = Json.encode(Map("version" -> 2, "endpoints"->advertisedEndpoints.values.mkString(","), "jmx_port" -> jmxPort, "timestamp" -> timestamp)) val expectedBroker = new Broker(id, advertisedEndpoints) registerBrokerInZk(zkClient, brokerIdPath, brokerInfo, expectedBroker, timeout) @@ -204,7 +204,7 @@ object ZkUtils extends Logging { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) - val expectedBroker = new Broker(id, List(new EndPoint(host, port, SecurityProtocol.PLAINTEXT))) + val expectedBroker = new Broker(id, immutable.Map(SecurityProtocol.PLAINTEXT -> new EndPoint(host, port, SecurityProtocol.PLAINTEXT))) registerBrokerInZk(zkClient, brokerIdPath, brokerInfo, expectedBroker, timeout) diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 56e18ca..fc2d5a3 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -74,8 +74,8 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { // TODO: we need to migrate to new consumers when 0.9 is final - val endpoint1 = Utils.listenerListToEndPoints(configs(0).listeners).head - val endpoint2 = Utils.listenerListToEndPoints(configs(1).listeners).head + val endpoint1 = Utils.listenerListToEndPoints(configs(0).listeners).values.head + val endpoint2 = Utils.listenerListToEndPoints(configs(1).listeners).values.head consumer1 = new SimpleConsumer("localhost", endpoint1.port, 100, 1024*1024, "") consumer2 = new SimpleConsumer("localhost", endpoint2.port, 100, 1024*1024, "") diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index 78d8da4..ee71fd5 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -55,8 +55,8 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { super.setUp() // TODO: we need to migrate to new consumers when 0.9 is final - val endpoint1 = Utils.listenerListToEndPoints(configs(0).listeners).head - val endpoint2 = Utils.listenerListToEndPoints(configs(1).listeners).head + val endpoint1 = Utils.listenerListToEndPoints(configs(0).listeners).values.head + val endpoint2 = Utils.listenerListToEndPoints(configs(1).listeners).values.head consumer1 = new SimpleConsumer("localhost", endpoint1.port, 100, 1024*1024, "") consumer2 = new SimpleConsumer("localhost", endpoint2.port, 100, 1024*1024, "") } diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 8e7f47f..9dbd430 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -80,9 +80,9 @@ object SerializationTestUtils { TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100) ) - private val brokers = List(new Broker(0, List(EndPoint("localhost", 1011, SecurityProtocol.PLAINTEXT))), - new Broker(1, List(EndPoint("localhost", 1012, SecurityProtocol.PLAINTEXT))), - new Broker(2, List(EndPoint("localhost", 1013, SecurityProtocol.PLAINTEXT)))) + private val brokers = List(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> EndPoint("localhost", 1011, SecurityProtocol.PLAINTEXT))), + new Broker(1, Map(SecurityProtocol.PLAINTEXT -> EndPoint("localhost", 1012, SecurityProtocol.PLAINTEXT))), + new Broker(2, Map(SecurityProtocol.PLAINTEXT -> EndPoint("localhost", 1013, SecurityProtocol.PLAINTEXT)))) private val brokerEndpoints = brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) private val partitionMetaData0 = new PartitionMetadata(0, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints, errorCode = 0) diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index 9830c07..b9c7fd1 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -31,7 +31,7 @@ class BrokerTest extends JUnit3Suite with Logging { def testSerDe() = { val endpoint = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) - val listEndPoints = List(endpoint) + val listEndPoints = Map(SecurityProtocol.PLAINTEXT -> endpoint) val origBroker = new Broker(1, listEndPoints) val brokerBytes = ByteBuffer.allocate(origBroker.sizeInBytes) @@ -47,10 +47,10 @@ class BrokerTest extends JUnit3Suite with Logging { val endpoint2 = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) val endpoint3 = new EndPoint("myhost", 1111, SecurityProtocol.PLAINTEXT) val endpoint4 = new EndPoint("other", 1111, SecurityProtocol.PLAINTEXT) - val broker1 = new Broker(1, List(endpoint1)) - val broker2 = new Broker(1, List(endpoint2)) - val broker3 = new Broker(2, List(endpoint3)) - val broker4 = new Broker(1, List(endpoint4)) + val broker1 = new Broker(1, Map(SecurityProtocol.PLAINTEXT -> endpoint1)) + val broker2 = new Broker(1, Map(SecurityProtocol.PLAINTEXT -> endpoint2)) + val broker3 = new Broker(2, Map(SecurityProtocol.PLAINTEXT -> endpoint3)) + val broker4 = new Broker(1, Map(SecurityProtocol.PLAINTEXT -> endpoint4)) assert(broker1 == broker2) assert(broker1 != broker3) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index d19ea71..186b145 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -40,8 +40,8 @@ class SocketServerTest extends JUnitSuite { val plaintextPort = ports.head val tracePort = ports.last val server: SocketServer = new SocketServer(0, - List(EndPoint(null, plaintextPort, SecurityProtocol.PLAINTEXT), - EndPoint(null, tracePort, SecurityProtocol.TRACE)), + Map(SecurityProtocol.PLAINTEXT -> EndPoint(null, plaintextPort, SecurityProtocol.PLAINTEXT), + SecurityProtocol.TRACE -> EndPoint(null, tracePort, SecurityProtocol.TRACE)), numProcessorThreads = 1, maxQueuedRequests = 50, sendBufferSize = 300000, @@ -82,8 +82,9 @@ class SocketServerTest extends JUnitSuite { connect(SecurityProtocol.PLAINTEXT) } + def connect(protocol: SecurityProtocol, s:SocketServer = server) = { - new Socket("localhost", server.endpoints.find(_.protocolType == protocol).get.port) + new Socket("localhost", server.endpoints.get(protocol).get.port) } @After diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 5630ddd..80b5243 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -40,7 +40,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testReachableServer() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -76,7 +76,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testEmptyProduceRequest() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 @@ -94,7 +94,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLarge() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -122,7 +122,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLargeWithAckZero() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) props.put("request.required.acks", "0") @@ -150,7 +150,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceCorrectlyReceivesResponse() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -197,7 +197,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { val timeoutMs = 500 val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -224,7 +224,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceRequestWithNoResponse() { val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 val clientId = SyncProducerConfig.DefaultClientId @@ -240,7 +240,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { def testNotEnoughReplicas() { val topicName = "minisrtest" val server = servers.head - val port = server.socketServer.endpoints.find(_.protocolType==SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port val props = TestUtils.getSyncProducerConfig(port) props.put("request.required.acks", "-1") diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala index fa5c19e..d56503f 100644 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -45,7 +45,7 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { def testBrokerAdvertiseToZK { val brokerInfo = ZkUtils.getBrokerInfo(zkClient, brokerId) - val endpoint = brokerInfo.get.endPoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get + val endpoint = brokerInfo.get.endPoints.get(SecurityProtocol.PLAINTEXT).get assertEquals(advertisedHostName, endpoint.host) assertEquals(advertisedPort, endpoint.port) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index cf3e313..32ef5f5 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -98,7 +98,7 @@ class KafkaConfigTest extends JUnit3Suite { val serverConfig = new KafkaConfig(props) val endpoints = Utils.listenerListToEndPoints(serverConfig.advertisedListeners) - val endpoint = endpoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get assertEquals(endpoint.host, hostName) assertEquals(endpoint.port, port) } @@ -114,7 +114,7 @@ class KafkaConfigTest extends JUnit3Suite { val serverConfig = new KafkaConfig(props) val endpoints = Utils.listenerListToEndPoints(serverConfig.advertisedListeners) - val endpoint = endpoints.find(_.protocolType == SecurityProtocol.PLAINTEXT).get + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get assertEquals(endpoint.host, advertisedHostName) assertEquals(endpoint.port, advertisedPort) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 837413c..fec4f04 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -147,7 +147,7 @@ object TestUtils extends Logging { def getBrokerListStrFromConfigs(configs: Seq[KafkaConfig]): String = { configs.map(c => { - val endpoint = Utils.listenerListToEndPoints(c.listeners).find(_.protocolType == SecurityProtocol.PLAINTEXT).get + val endpoint = Utils.listenerListToEndPoints(c.listeners).get(SecurityProtocol.PLAINTEXT).get formatAddress(endpoint.host, endpoint.port) }).mkString(",") } @@ -448,13 +448,13 @@ object TestUtils extends Logging { } def createBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { - val brokers = ids.map(id => new Broker(id, "localhost", 6667)) + val brokers = ids.map(id => new Broker(id, "localhost", 6667, SecurityProtocol.PLAINTEXT)) brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.endPoints, 6000, jmxPort = -1)) brokers } def deleteBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { - val brokers = ids.map(id => new Broker(id, "localhost", 6667)) + val brokers = ids.map(id => new Broker(id, "localhost", 6667, SecurityProtocol.PLAINTEXT)) brokers.foreach(b => ZkUtils.deletePath(zkClient, ZkUtils.BrokerIdsPath + "/" + b)) brokers } -- 1.9.3 (Apple Git-50) From 185c20cb1a038160bc57fe98230f67bac33a6662 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 13 Jan 2015 13:47:57 -0800 Subject: [PATCH 36/41] validate that listeners and advertised listeners have unique ports and protocols --- core/src/main/scala/kafka/server/KafkaConfig.scala | 20 ++++++++++++- .../test/scala/unit/kafka/KafkaConfigTest.scala | 35 ++++++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index c0a0427..94fb6d9 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -18,7 +18,8 @@ package kafka.server import java.util.Properties -import kafka.cluster.SecurityProtocol +import kafka.cluster.{EndPoint, SecurityProtocol} +import kafka.common.InvalidConfigException import kafka.message.{MessageSet, Message} import kafka.consumer.ConsumerConfig import kafka.utils.{VerifiableProperties, ZKConfig, Utils} @@ -30,6 +31,20 @@ import kafka.message.BrokerCompressionCodec */ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(props) { + def validate(config: KafkaConfig) { + validateUniquePortAndProtocol(config.advertisedListeners) + } + + def validateUniquePortAndProtocol(listeners: String) { + val listenerList = Utils.parseCsvList(listeners) + val endpoints = listenerList.map(listener => EndPoint.createEndPoint(listener)) + val distinctPorts = endpoints.map(ep => ep.port).distinct + val distinctProtocols = endpoints.map(ep => ep.protocolType).distinct + + if (distinctPorts.size < endpoints.size) throw new InvalidConfigException("Only one listener is allowed per port. Please check listeners for duplicates") + if (distinctProtocols.size < endpoints.size) throw new InvalidConfigException("Only one listener is allowed per protocol. Please check listeners for duplicates") + } + def this(originalProps: Properties) { this(new VerifiableProperties(originalProps)) props.verify() @@ -359,4 +374,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val compressionType = props.getString("compression.type", "producer").toLowerCase() require(BrokerCompressionCodec.isValid(compressionType), "compression.type : "+compressionType + " is not valid." + " Valid options are "+BrokerCompressionCodec.brokerCompressionOptions.mkString(",")) + + validate(this) + } diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index bc4aef3..c8d1437 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -18,8 +18,11 @@ package unit.kafka import java.io.{FileOutputStream, File} import java.security.Permission +import java.util.Properties import kafka.Kafka +import kafka.common.InvalidConfigException +import kafka.server.KafkaConfig import org.junit.{After, Before, Test} import junit.framework.Assert._ @@ -99,6 +102,38 @@ class KafkaTest { Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "broker.id=1", "--override", "broker.id=2")) } + @Test + def testDuplicateListeners() { + val props = new Properties() + props.put("broker.id","1") + props.put("zookeeper.connect","localhost:2181") + + // listeners with duplicate port + props.put("listeners","PLAINTEXT://localhost:9091,TRACE://localhost:9091") + + assert(!isValidKafkaConfig(props)) + + // listeners with duplicate protocol + props.put("listeners","PLAINTEXT://localhost:9091,PLAINTEXT://localhost:9092") + + assert(!isValidKafkaConfig(props)) + + // advertised listeners with duplicate port + props.put("advertised,listeners","PLAINTEXT://localhost:9091,TRACE://localhost:9091") + + assert(!isValidKafkaConfig(props)) + + } + + def isValidKafkaConfig(props: Properties): Boolean = { + try { + new KafkaConfig(props) + true + } catch { + case e: InvalidConfigException => false + } + } + def prepareDefaultConfig(): String = { prepareConfig(Array("broker.id=1", "zookeeper.connect=somewhere")) } -- 1.9.3 (Apple Git-50) From a5215b50e4fa40bc211b5cd8210f5dc2f004c376 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 13 Jan 2015 15:11:55 -0800 Subject: [PATCH 37/41] support legacy configs --- core/src/main/scala/kafka/server/KafkaConfig.scala | 27 ++++++++++++++++-- .../test/scala/unit/kafka/KafkaConfigTest.scala | 32 ++++++++++++++++++---- core/src/test/scala/unit/kafka/log/LogTest.scala | 2 +- 3 files changed, 53 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 94fb6d9..4a8a09f 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -50,6 +50,29 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro props.verify() } + // If the user did not define listeners but did define host or port, lets use them in backward compatible way + // If none of those are defined, we default to PLAINTEXT://null:6667 + private def getListeners(): String = { + if (props.containsKey("listeners")) { + props.getString("listeners") + } else { + "PLAINTEXT://" + props.getString("host.name", null) + ":" + props.getInt("port", 6667).toString + } + } + + // If the user defined advertised listeners, we use those + // 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(): String = { + if (props.containsKey("advertised.listeners")) { + props.getString("advertised.listeners") + } else if (props.containsKey("advertised.host.name") || props.containsKey("advertised.port") ) { + "PLAINTEXT://" + props.getString("advertised.host.name", props.getString("host.name", null)) + ":" + props.getInt("advertised.port", props.getInt("port", 6667)).toString + } else { + getListeners() + } + } + private def getLogRetentionTimeMillis(): Long = { val millisInMinute = 60L * 1000L val millisInHour = 60L * millisInMinute @@ -118,12 +141,12 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* Listener List - Comma-separated list of URIs we will listen on and their protocols. * Specify hostname as 0.0.0.0 to bind to all interfaces * Leave hostname empty to bind to default interface */ - val listeners: String = props.getString("listeners", "PLAINTEXT://:6667") + val listeners: String = getListeners() /* Listeners to publish to ZooKeeper for clients to use, if different than the listeners above. * In IaaS environments, this may need to be different from the interface to which the broker binds. * If this is not set, it will use the value for "listeners" */ - val advertisedListeners: String = props.getString("advertised.listeners", listeners) + val advertisedListeners: String = getAdvertisedListeners() /* the SO_SNDBUFF buffer of the socket sever sockets */ diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index c8d1437..fc49d91 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -105,26 +105,48 @@ class KafkaTest { @Test def testDuplicateListeners() { val props = new Properties() - props.put("broker.id","1") - props.put("zookeeper.connect","localhost:2181") + props.put("broker.id", "1") + props.put("zookeeper.connect", "localhost:2181") // listeners with duplicate port - props.put("listeners","PLAINTEXT://localhost:9091,TRACE://localhost:9091") + props.put("listeners", "PLAINTEXT://localhost:9091,TRACE://localhost:9091") assert(!isValidKafkaConfig(props)) // listeners with duplicate protocol - props.put("listeners","PLAINTEXT://localhost:9091,PLAINTEXT://localhost:9092") + props.put("listeners", "PLAINTEXT://localhost:9091,PLAINTEXT://localhost:9092") assert(!isValidKafkaConfig(props)) // advertised listeners with duplicate port - props.put("advertised,listeners","PLAINTEXT://localhost:9091,TRACE://localhost:9091") + props.put("advertised,listeners", "PLAINTEXT://localhost:9091,TRACE://localhost:9091") assert(!isValidKafkaConfig(props)) } + @Test + def testListenerDefaults() { + val props = new Properties() + props.put("broker.id", "1") + props.put("zookeeper.connect", "localhost:2181") + + // configuration with host and port, but no listeners + props.put("host.name", "myhost") + props.put("port", "1111") + + val conf = new KafkaConfig(props) + assertEquals(conf.listeners, "PLAINTEXT://myhost:1111") + + // configuration with advertised host and port, and no advertised listeners + props.put("advertised.host.name", "otherhost") + props.put("advertised.port", "2222") + + val conf2 = new KafkaConfig(props) + assertEquals(conf2.advertisedListeners, "PLAINTEXT://otherhost:2222") + + } + def isValidKafkaConfig(props: Properties): Boolean = { try { new KafkaConfig(props) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index c2dd8eb..9b1df44 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -37,7 +37,7 @@ class LogTest extends JUnitSuite { @Before def setUp() { logDir = TestUtils.tempDir() - val props = TestUtils.createBrokerConfig(0, -1) + val props = TestUtils.createBrokerConfig(0, 1) config = new KafkaConfig(props) } -- 1.9.3 (Apple Git-50) From 862215a3d264639c1accb9fe758f91b49fd5ee3c Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 13 Jan 2015 18:16:07 -0800 Subject: [PATCH 38/41] some fixes following rebase --- .../scala/unit/kafka/network/SocketServerTest.scala | 21 ++++++++------------- 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 186b145..be553e3 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -78,12 +78,8 @@ class SocketServerTest extends JUnitSuite { channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) } - def connect(s:SocketServer = server) : java.net.Socket = { - connect(SecurityProtocol.PLAINTEXT) - } - - def connect(protocol: SecurityProtocol, s:SocketServer = server) = { + def connect(s:SocketServer = server, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { new Socket("localhost", server.endpoints.get(protocol).get.port) } @@ -93,8 +89,8 @@ class SocketServerTest extends JUnitSuite { } @Test def simpleRequest() { - val plainSocket = connect(SecurityProtocol.PLAINTEXT) - val traceSocket = connect(SecurityProtocol.TRACE) + val plainSocket = connect(protocol = SecurityProtocol.PLAINTEXT) + val traceSocket = connect(protocol = SecurityProtocol.TRACE) val correlationId = -1 val clientId = SyncProducerConfig.DefaultClientId val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs @@ -151,8 +147,8 @@ class SocketServerTest extends JUnitSuite { @Test def testSocketsCloseOnShutdown() { // open a connection - val plainSocket = connect(SecurityProtocol.PLAINTEXT) - val traceSocket = connect(SecurityProtocol.TRACE) + val plainSocket = connect(protocol = SecurityProtocol.PLAINTEXT) + val traceSocket = connect(protocol = SecurityProtocol.TRACE) val bytes = new Array[Byte](40) // send a request first to make sure the connection has been picked up by the socket server sendRequest(plainSocket, 0, bytes) @@ -163,13 +159,13 @@ class SocketServerTest extends JUnitSuite { // doing a subsequent send should throw an exception as the connection should be closed. try { sendRequest(plainSocket, 0, bytes) - fail("expected exception when writing to closed socket") + fail("expected exception when writing to closed plain socket") } catch { case e: IOException => // expected } try { sendRequest(traceSocket, 0, bytes) - fail("expected exception when writing to closed socket") + fail("expected exception when writing to closed trace socket") } catch { case e: IOException => // expected } @@ -190,8 +186,7 @@ class SocketServerTest extends JUnitSuite { val overrideNum = 6 val overrides: Map[String, Int] = Map("localhost" -> overrideNum) val overrideServer: SocketServer = new SocketServer(0, - host = null, - port = kafka.utils.TestUtils.choosePort, + Map(SecurityProtocol.PLAINTEXT -> EndPoint(null, kafka.utils.TestUtils.choosePort, SecurityProtocol.PLAINTEXT)), numProcessorThreads = 1, maxQueuedRequests = 50, sendBufferSize = 300000, -- 1.9.3 (Apple Git-50) From 8b6798cb87054caf3b82dc02bf56bb7b01ce7d5c Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Wed, 28 Jan 2015 10:25:39 -0800 Subject: [PATCH 39/41] Reverted to backward compatible zk registration, changed use.new.protocol to support multiple versions and few minor fixes --- .../apache/kafka/common/protocol/ApiVersion.java | 38 ++++++++++++++++++++++ .../apache/kafka/common/utils/ClientUtilsTest.java | 4 +-- .../org/apache/kafka/common/utils/UtilsTest.java | 4 +-- core/src/main/scala/kafka/cluster/Broker.scala | 11 +++---- .../controller/ControllerChannelManager.scala | 3 +- .../main/scala/kafka/network/SocketServer.scala | 4 +-- core/src/main/scala/kafka/server/KafkaConfig.scala | 19 ++++++++--- .../main/scala/kafka/server/KafkaHealthcheck.scala | 14 +++----- core/src/main/scala/kafka/server/KafkaServer.scala | 4 ++- core/src/main/scala/kafka/utils/ZkUtils.scala | 23 ++----------- .../unit/kafka/producer/SyncProducerTest.scala | 2 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 2 +- system_test/utils/kafka_system_test_utils.py | 8 ++--- 13 files changed, 81 insertions(+), 55 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java new file mode 100644 index 0000000..5390993 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.protocol; + +public enum ApiVersion { + KAFKA_0820, + KAFKA_0830, + KAFKA_0900; + + public boolean onOrAfter(ApiVersion other) { + return compareTo(other) >= 0; + } + + public static ApiVersion parseConfig(String version) { + String[] vals = version.split("\\."); + StringBuilder parsed = new StringBuilder(); + parsed.append("KAFKA_"); + + for (String v: vals) { + parsed.append(v); + } + return ApiVersion.valueOf(parsed.toString()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java index be66c87..6e37ea5 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java @@ -26,9 +26,9 @@ public class ClientUtilsTest { @Test public void testParseAndValidateAddresses() { check("127.0.0.1:8000"); - check("PLAINTEXT://mydomain.com:8080"); + check("mydomain.com:8080"); check("[::1]:8000"); - check("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:1234", "mydomain.com:10000"); + check("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:1234", "mydomain.com:10000"); } @Test(expected = ConfigException.class) 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 623f910..dc69d14 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -38,9 +38,9 @@ public class UtilsTest { @Test public void testGetPort() { - assertEquals(8000, getPort("PLAINTEXT://127.0.0.1:8000").intValue()); + assertEquals(8000, getPort("127.0.0.1:8000").intValue()); assertEquals(8080, getPort("mydomain.com:8080").intValue()); - assertEquals(1234, getPort("PLAINTEXT://[::1]:1234").intValue()); + assertEquals(1234, getPort("[::1]:1234").intValue()); assertEquals(5678, getPort("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678").intValue()); } diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 759ef49..4104114 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -27,7 +27,7 @@ import kafka.cluster.SecurityProtocol._ * A Kafka broker * A broker has an id, a host and a collection of end-points * each end-point is (port,protocolType) - * currently the only channel type is PlainText + * currently the only protocol type is PlainText * but we will add SSL and Kerberos in the future */ object Broker { @@ -46,14 +46,11 @@ object Broker { * * The current JSON schema for a broker is: * {"version":2, + * "host","localhost", + * "port",9092 * "jmx_port":9999, * "timestamp":"2233345666", - * "endpoints": [ - * {"port":9092, - * "protocolType":"plain"}, - * {"port":9093, - * "protocolType":"ssl"}]} - * @return + * "endpoints": "{PLAINTEXT://host1:9092,SSL://host1:9093"} */ def createBroker(id: Int, brokerInfoString: String): Broker = { if(brokerInfoString == null) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index d5c24eb..635b0fe 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -18,6 +18,7 @@ package kafka.controller import kafka.network.{Receive, BlockingChannel} import kafka.utils.{Utils, Logging, ShutdownableThread} +import org.apache.kafka.common.protocol.ApiVersion import collection.mutable.HashMap import kafka.cluster.Broker import java.util.concurrent.{LinkedBlockingQueue, BlockingQueue} @@ -295,7 +296,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val broker = m._1 val partitionStateInfos = m._2.toMap - val versionId = if (controller.config.useNewWireProtocol) 1 else 0 + val versionId = if (controller.config.intraBrokerProtocolVersion.onOrAfter(ApiVersion.KAFKA_0830)) 1 else 0 val updateMetadataRequest = new UpdateMetadataRequest(versionId = versionId.toShort, controllerId = controllerId, controllerEpoch = controllerEpoch, correlationId = correlationId, clientId = clientId, partitionStateInfos = partitionStateInfos, aliveBrokers = controllerContext.liveOrShuttingDownBrokers) partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index bdd4400..ff0c1ab 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -65,7 +65,7 @@ class SocketServer(val brokerId: Int, so the processor can put the correct protocol in the request channel. we'll probably have a more elegant way of doing this once we patch the request channel to include more information about security and authentication. - TODO: re-consider this code when working on JIRA-1683 + TODO: re-consider this code when working on KAFKA-1683 */ private val portToProtocol: Map[Int, SecurityProtocol] = endpoints.map{ case (protocol: SecurityProtocol, endpoint: EndPoint) => (endpoint.port -> protocol )} @@ -470,7 +470,7 @@ private[kafka] class Processor(val id: Int, close(key) } else if(receive.complete) { val port = socketChannel.socket().getLocalPort - val protocol = portToProtocol.getOrElse(port,SecurityProtocol.PLAINTEXT) + val protocol = portToProtocol(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) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 4a8a09f..e4854cc 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -25,6 +25,7 @@ import kafka.consumer.ConsumerConfig import kafka.utils.{VerifiableProperties, ZKConfig, Utils} import kafka.message.NoCompressionCodec import kafka.message.BrokerCompressionCodec +import org.apache.kafka.common.protocol.ApiVersion /** * Configuration settings for the kafka server @@ -140,7 +141,12 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* Listener List - Comma-separated list of URIs we will listen on and their protocols. * Specify hostname as 0.0.0.0 to bind to all interfaces - * Leave hostname empty to bind to default interface */ + * Leave hostname empty to bind to default interface + * + * examples of legal listener lists: + * PLAINTEXT://myhost:9092,TRACE://:9091 + * PLAINTEXT://0.0.0.0:9092, TRACE://localhost:9093 + * */ val listeners: String = getListeners() /* Listeners to publish to ZooKeeper for clients to use, if different than the listeners above. @@ -148,7 +154,6 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro * If this is not set, it will use the value for "listeners" */ val advertisedListeners: String = getAdvertisedListeners() - /* the SO_SNDBUFF buffer of the socket sever sockets */ val socketSendBufferBytes: Int = props.getInt("socket.send.buffer.bytes", 100*1024) @@ -168,10 +173,14 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val connectionsMaxIdleMs = props.getLong("connections.max.idle.ms", 10*60*1000L) /* security protocol used to communicate between brokers. Defaults to plain text. */ - val replicationSecurityProtocol = SecurityProtocol.withName(props.getString("replication.security.protocol","PLAINTEXT")) + val replicationSecurityProtocol = SecurityProtocol.withName(props.getString("intra.broker.security.protocol","PLAINTEXT")) + + /* specify which version of the inter-broker protocol will be used + this is typically bumped after all brokers were upgraded to a new version + valid values: 0.8.2.0, 0.8.3.0, 0.9.0.0 + */ - /* allow users to bump the protocol version when they are done upgrading */ - val useNewWireProtocol = props.getBoolean("use.new.wire.protocol",false) + val intraBrokerProtocolVersion = ApiVersion.parseConfig(props.getString("use.intra.broker.protocol.version","0.8.3.0")) /*********** Log Configuration ***********/ diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index be07ffd..b6ef750 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -36,8 +36,7 @@ import java.net.InetAddress class KafkaHealthcheck(private val brokerId: Int, private val advertisedEndpoints: Map[SecurityProtocol, EndPoint], private val zkSessionTimeoutMs: Int, - private val zkClient: ZkClient, - private val useNewWriteProtocol: Boolean) extends Logging { + private val zkClient: ZkClient) extends Logging { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId val sessionExpireListener = new SessionExpireListener @@ -64,13 +63,10 @@ class KafkaHealthcheck(private val brokerId: Int, endpoint ) - if (useNewWriteProtocol) { - ZkUtils.registerBrokerInZk(zkClient, brokerId, updatedEndpoints, zkSessionTimeoutMs, jmxPort) - } else { - // before we upgrade to the new protocol, only PLAINTEXT is supported - val defaultEndpoint = updatedEndpoints.get(SecurityProtocol.PLAINTEXT).get - ZkUtils.registerBrokerInZk(zkClient, brokerId, defaultEndpoint.host, defaultEndpoint.port, zkSessionTimeoutMs, jmxPort) - } + // the default host and port are here for compatibility with older client + // only PLAINTEXT is supported as default + val defaultEndpoint = updatedEndpoints(SecurityProtocol.PLAINTEXT) + ZkUtils.registerBrokerInZk(zkClient, brokerId, defaultEndpoint.host, defaultEndpoint.port, updatedEndpoints, zkSessionTimeoutMs, jmxPort) } /** diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 2f44a4c..418cda0 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -25,6 +25,8 @@ import kafka.utils._ import java.util.concurrent._ import atomic.{AtomicInteger, AtomicBoolean} import java.io.File +import org.apache.kafka.common.protocol.ApiVersion + import collection.mutable import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} @@ -128,7 +130,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg /* tell everyone we are alive */ val endpoints = Utils.listenerListToEndPoints(config.advertisedListeners) - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, endpoints, config.zkSessionTimeoutMs, zkClient, config.useNewWireProtocol) + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, endpoints, config.zkSessionTimeoutMs, zkClient) kafkaHealthcheck.startup() registerStats() diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 6d682ac..2de1223 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -175,17 +175,18 @@ object ZkUtils extends Logging { /** * Register brokers with v2 json format (which includes multiple endpoints) + * This format also includes default endpoints for compatibility with older clients * @param zkClient * @param id * @param advertisedEndpoints * @param timeout * @param jmxPort */ - def registerBrokerInZk(zkClient: ZkClient, id: Int, advertisedEndpoints: immutable.Map[SecurityProtocol, EndPoint], timeout: Int, jmxPort: Int) { + def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, advertisedEndpoints: immutable.Map[SecurityProtocol, EndPoint], timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 2, "endpoints"->advertisedEndpoints.values.mkString(","), "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val brokerInfo = Json.encode(Map("version" -> 2, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.values.mkString(","), "jmx_port" -> jmxPort, "timestamp" -> timestamp)) val expectedBroker = new Broker(id, advertisedEndpoints) registerBrokerInZk(zkClient, brokerIdPath, brokerInfo, expectedBroker, timeout) @@ -193,24 +194,6 @@ object ZkUtils extends Logging { info("Registered broker %d at path %s with addresses: %s".format(id, brokerIdPath, advertisedEndpoints.mkString(","))) } - /** - * Register brokers with v1 json format - single host and port pair - * @param zkClient - * @param id - * @param timeout - * @param jmxPort - */ - def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, timeout: Int, jmxPort: Int) { - val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id - val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) - val expectedBroker = new Broker(id, immutable.Map(SecurityProtocol.PLAINTEXT -> new EndPoint(host, port, SecurityProtocol.PLAINTEXT))) - - registerBrokerInZk(zkClient, brokerIdPath, brokerInfo, expectedBroker, timeout) - - info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, host, port)) - } - def registerBrokerInZk(zkClient: ZkClient, brokerIdPath: String, brokerInfo: String, expectedBroker: Broker, timeout: Int) { try { createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 80b5243..fe456e6 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -76,7 +76,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testEmptyProduceRequest() { val server = servers.head - val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints(SecurityProtocol.PLAINTEXT).port val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index fec4f04..bfec0d0 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -449,7 +449,7 @@ object TestUtils extends Logging { def createBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { val brokers = ids.map(id => new Broker(id, "localhost", 6667, SecurityProtocol.PLAINTEXT)) - brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.endPoints, 6000, jmxPort = -1)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, "localhost", 6667, b.endPoints, 6000, jmxPort = -1)) brokers } diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index a0e82c2..a9b73f7 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -386,14 +386,14 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv if clusterName == "source": if ( len(testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"]) == 0 ): - testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"] = "PLAINTEXT://" + hostname + ":" + port + testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"] = hostname + ":" + port else: - testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"] += "," + "PLAINTEXT://" + hostname + ":" + port + testcaseEnv.userDefinedEnvVarDict["sourceBrokerList"] += "," + hostname + ":" + port elif clusterName == "target": if ( len(testcaseEnv.userDefinedEnvVarDict["targetBrokerList"]) == 0 ): - testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] = "PLAINTEXT://" + hostname + ":" + port + testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] = hostname + ":" + port else: - testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] += "," + "PLAINTEXT://" + hostname + ":" + port + testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] += "," + hostname + ":" + port else: logger.error("Invalid cluster name: " + clusterName, extra=d) raise Exception("Invalid cluster name : " + clusterName) -- 1.9.3 (Apple Git-50) From c7e08a665cead2e67828307982f37410005efafd Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 2 Feb 2015 11:19:22 -0800 Subject: [PATCH 40/41] fixing some issues after rebase --- .../java/org/apache/kafka/clients/producer/ProducerConfig.java | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 6 ------ .../test/scala/unit/kafka/integration/KafkaServerTestHarness.scala | 7 ++++++- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 5bdc2e9..bb0f56e 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 @@ -222,7 +222,7 @@ public class ProducerConfig extends AbstractConfig { .define(SECURITY_PROTOCOL, Type.STRING, "PLAINTEXT", - in(Arrays.asList("PLAINTEXT")), + in("PLAINTEXT"), Importance.MEDIUM, SECURITY_PROTOCOL_DOC); } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 409da74..b8a7093 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -20,9 +20,6 @@ package kafka.server import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.JoinGroupResponse import org.apache.kafka.common.requests.HeartbeatResponse -import org.apache.kafka.common.requests.ResponseHeader -import org.apache.kafka.common.protocol.types.Struct -import java.net.InetSocketAddress import kafka.api._ import kafka.cluster.SecurityProtocol.SecurityProtocol @@ -34,9 +31,6 @@ import kafka.network.RequestChannel.Response import kafka.controller.KafkaController import kafka.utils.{SystemTime, Logging} -import java.nio.ByteBuffer -import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic._ import scala.collection._ import org.I0Itec.zkclient.ZkClient diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index ef4c9ae..334242e 100644 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -18,6 +18,8 @@ package kafka.integration import java.util.Arrays +import kafka.cluster.SecurityProtocol + import scala.collection.mutable.Buffer import kafka.server._ import kafka.utils.{Utils, TestUtils} @@ -38,7 +40,10 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { def serverForId(id: Int) = servers.find(s => s.config.brokerId == id) - def bootstrapUrl = configs.map(c => c.hostName + ":" + c.port).mkString(",") + def bootstrapUrl = configs.map(c => { + val endpoint = Utils.listenerListToEndPoints(c.listeners)(SecurityProtocol.PLAINTEXT) + endpoint.host + ":" + endpoint.port + }).mkString(",") override def setUp() { super.setUp -- 1.9.3 (Apple Git-50) From 8ab9c477f39e058a41da40c47518a6a35affdec6 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 2 Feb 2015 11:54:20 -0800 Subject: [PATCH 41/41] modified inter.broker.protocol config to start with security per feedback --- core/src/main/scala/kafka/controller/ControllerChannelManager.scala | 4 ++-- core/src/main/scala/kafka/controller/KafkaController.scala | 2 +- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- core/src/main/scala/kafka/server/KafkaServer.scala | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 635b0fe..5864967 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -81,7 +81,7 @@ class ControllerChannelManager (private val controllerContext: ControllerContext private def addNewBroker(broker: Broker) { val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize) debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id)) - val brokerEndPoint = broker.getBrokerEndPoint(config.replicationSecurityProtocol) + val brokerEndPoint = broker.getBrokerEndPoint(config.interBrokerSecurityProtocol) val channel = new BlockingChannel(brokerEndPoint.host, brokerEndPoint.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, @@ -280,7 +280,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val broker = m._1 val partitionStateInfos = m._2.toMap val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet - val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map(b => b.getBrokerEndPoint(controller.config.replicationSecurityProtocol)) + val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map(b => b.getBrokerEndPoint(controller.config.interBrokerSecurityProtocol)) val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId, clientId) for (p <- partitionStateInfos) { val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 619bcbb..6cf4c84 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -214,7 +214,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt def clientId = { val listeners = listenerListToEndPoints(config.listeners) - val controllerListener = listeners.get(config.replicationSecurityProtocol) + val controllerListener = listeners.get(config.interBrokerSecurityProtocol) "id_%d-host_%s-port_%d".format(config.brokerId, controllerListener.get.host, controllerListener.get.port) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index e4854cc..bcc3596 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -173,7 +173,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val connectionsMaxIdleMs = props.getLong("connections.max.idle.ms", 10*60*1000L) /* security protocol used to communicate between brokers. Defaults to plain text. */ - val replicationSecurityProtocol = SecurityProtocol.withName(props.getString("intra.broker.security.protocol","PLAINTEXT")) + val interBrokerSecurityProtocol = SecurityProtocol.withName(props.getString("security.intra.broker.protocol","PLAINTEXT")) /* specify which version of the inter-broker protocol will be used this is typically bumped after all brokers were upgraded to a new version diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 418cda0..8a60b0f 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -209,8 +209,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (channel != null) { channel.disconnect() } - channel = new BlockingChannel(broker.getBrokerEndPoint(config.replicationSecurityProtocol).host, - broker.getBrokerEndPoint(config.replicationSecurityProtocol).port, + channel = new BlockingChannel(broker.getBrokerEndPoint(config.interBrokerSecurityProtocol).host, + broker.getBrokerEndPoint(config.interBrokerSecurityProtocol).port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) -- 1.9.3 (Apple Git-50)