Index: core/src/test/scala/unit/kafka/utils/TestUtils.scala =================================================================== --- core/src/test/scala/unit/kafka/utils/TestUtils.scala (revision 1371523) +++ core/src/test/scala/unit/kafka/utils/TestUtils.scala (working copy) @@ -21,8 +21,7 @@ import java.net._ import java.nio._ import java.nio.channels._ -import java.util.Random -import java.util.Properties +import java.util.{Random, Properties} import junit.framework.Assert._ import kafka.server._ import kafka.producer._ @@ -301,8 +300,28 @@ } } + def waitUntilTrue(condition: () => Boolean, waitTime: Long): Boolean = { + val startTime = System.currentTimeMillis() + while (true) { + if (condition()) + return true + if (System.currentTimeMillis() > startTime + waitTime) + return false + } + // should never hit here + throw new RuntimeException("unexpected error") + } + + def checkSetEqual(actual : Seq[Tuple2[String,String]], expected : Seq[Tuple2[String,String]]) { + assertEquals(expected.length, actual.length) + for (i <- 0 until expected.length) { + assertEquals(expected(i)._1, actual(i)._1) + assertEquals(expected(i)._2, actual(i)._2) + } + } + } object TestZKUtils { - val zookeeperConnect = "127.0.0.1:2182" + val zookeeperConnect = "127.0.0.1:2182" } Index: core/src/test/scala/unit/kafka/integration/FetcherTest.scala =================================================================== --- core/src/test/scala/unit/kafka/integration/FetcherTest.scala (revision 1371523) +++ core/src/test/scala/unit/kafka/integration/FetcherTest.scala (working copy) @@ -40,7 +40,7 @@ val messages = new mutable.HashMap[Int, ByteBufferMessageSet] val topic = "topic" val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, c.brokerId.toString, "localhost", c.port))) - val shutdown = ZookeeperConsumerConnector.shutdownCommand + val shutdown = SimpleConsumerConnector.shutdownCommand val queue = new LinkedBlockingQueue[FetchedDataChunk] val topicInfos = configs.map(c => new PartitionTopicInfo(topic, c.brokerId, @@ -54,7 +54,7 @@ override def setUp() { super.setUp - fetcher = new Fetcher(new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), null) + fetcher = new Fetcher(new ConsumerConfig(TestUtils.createConsumerProperties("", "", ""))) fetcher.stopConnectionsToAllBrokers fetcher.startConnections(topicInfos, cluster) } Index: core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala =================================================================== --- core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala (revision 1371523) +++ core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala (working copy) @@ -40,6 +40,7 @@ val numMessages = 10 val largeOffset = 10000 val smallOffset = -1 + val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandlers]) @@ -70,7 +71,9 @@ var consumerProps = TestUtils.createConsumerProperties(zkConnect, group, testConsumer) consumerProps.put("autooffset.reset", "smallest") consumerProps.put("consumer.timeout.ms", "2000") - val consumerConfig = new ConsumerConfig(consumerProps) + val consumerConfig = new ConsumerConfig(consumerProps) { + override val initBrokerList = "{ \"1\" : \"localhost," + kafkaConfig.port.toString + "\"}" + } TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0-0", largeOffset) info("Updated consumer offset to " + largeOffset) @@ -124,7 +127,9 @@ var consumerProps = TestUtils.createConsumerProperties(zkConnect, group, testConsumer) consumerProps.put("autooffset.reset", "smallest") consumerProps.put("consumer.timeout.ms", "2000") - val consumerConfig = new ConsumerConfig(consumerProps) + val consumerConfig = new ConsumerConfig(consumerProps) { + override val initBrokerList = "{ \"1\" : \"localhost," + kafkaConfig.port.toString + "\"}" + } TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0-0", smallOffset) info("Updated consumer offset to " + smallOffset) @@ -177,7 +182,9 @@ var consumerProps = TestUtils.createConsumerProperties(zkConnect, group, testConsumer) consumerProps.put("autooffset.reset", "largest") consumerProps.put("consumer.timeout.ms", "2000") - val consumerConfig = new ConsumerConfig(consumerProps) + val consumerConfig = new ConsumerConfig(consumerProps) { + override val initBrokerList = "{ \"1\" : \"localhost," + kafkaConfig.port.toString + "\"}" + } TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0-0", largeOffset) info("Updated consumer offset to " + largeOffset) Index: core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala =================================================================== --- core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala (revision 1371523) +++ core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala (working copy) @@ -25,7 +25,7 @@ import kafka.common.InvalidMessageSizeException import kafka.zk.ZooKeeperTestHarness import kafka.utils.{TestZKUtils, TestUtils} -import kafka.consumer.{ZookeeperConsumerConnector, ConsumerConfig} +import kafka.consumer.{SimpleConsumerConnector, ConsumerConfig} import org.scalatest.junit.JUnit3Suite import kafka.integration.ProducerConsumerTestHarness import kafka.integration.KafkaServerTestHarness @@ -33,7 +33,8 @@ import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness with ZooKeeperTestHarness { - val zkConnect = TestZKUtils.zookeeperConnect + + val zkConnect = TestZKUtils.zookeeperConnect val port = TestUtils.choosePort val props = TestUtils.createBrokerConfig(0, port) val config = new KafkaConfig(props) { @@ -41,6 +42,7 @@ override val enableZookeeper = true } val configs = List(config) + val topic = "test" val partition = 0 @@ -88,10 +90,12 @@ case e: InvalidMessageSizeException => println("This is good") } - // test ZookeeperConsumer + // test SimpleConsumerConnector val consumerConfig1 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, "group1", "consumer1", 10000)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1) + TestUtils.createConsumerProperties(zkConnect, "group1", "consumer1", 10000)) { + override val initBrokerList = "{ \"1\" : \"localhost," + config.port.toString + "\"}" + } + val zkConsumerConnector1 = new SimpleConsumerConnector(consumerConfig1) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1)) try { for ((topic, messageStreams) <- topicMessageStreams1) Index: core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala =================================================================== --- core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala (revision 1371523) +++ core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala (working copy) @@ -1,127 +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.zk - -import junit.framework.Assert._ -import java.util.Collections -import kafka.consumer.{ConsumerConfig, ZookeeperConsumerConnector} -import java.lang.Thread -import org.scalatest.junit.JUnit3Suite -import kafka.utils.{TestUtils, ZkUtils, ZKGroupTopicDirs, TestZKUtils} - -class ZKLoadBalanceTest extends JUnit3Suite with ZooKeeperTestHarness { - val zkConnect = TestZKUtils.zookeeperConnect - var dirs : ZKGroupTopicDirs = null - val topic = "topic1" - val group = "group1" - val firstConsumer = "consumer1" - val secondConsumer = "consumer2" - - override def setUp() { - super.setUp() - - dirs = new ZKGroupTopicDirs(group, topic) - } - - def testLoadBalance() { - // create the first partition - ZkUtils.setupPartition(zkClient, 400, "broker1", 1111, "topic1", 1) - // add the first consumer - val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, firstConsumer)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, false) - zkConsumerConnector1.createMessageStreams(Map(topic -> 1)) - - { - // check Partition Owner Registry - val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_1 = List( ("400-0", "group1_consumer1-0") ) - checkSetEqual(actual_1, expected_1) - } - - // add a second consumer - val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, secondConsumer)) - val ZKConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, false) - ZKConsumerConnector2.createMessageStreams(Map(topic -> 1)) - // wait a bit to make sure rebalancing logic is triggered - Thread.sleep(200) - - { - // check Partition Owner Registry - val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_2 = List( ("400-0", "group1_consumer1-0") ) - checkSetEqual(actual_2, expected_2) - } - - { - // add a few more partitions - val brokers = List( - (200, "broker2", 1111, "topic1", 2), - (300, "broker3", 1111, "topic1", 2) ) - - for ((brokerID, host, port, topic, nParts) <- brokers) - ZkUtils.setupPartition(zkClient, brokerID, host, port, topic, nParts) - - - // wait a bit to make sure rebalancing logic is triggered - Thread.sleep(1000) - // check Partition Owner Registry - val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_3 = List( ("200-0", "group1_consumer1-0"), - ("200-1", "group1_consumer1-0"), - ("300-0", "group1_consumer1-0"), - ("300-1", "group1_consumer2-0"), - ("400-0", "group1_consumer2-0") ) - checkSetEqual(actual_3, expected_3) - } - - { - // now delete a partition - ZkUtils.deletePartition(zkClient, 400, "topic1") - - // wait a bit to make sure rebalancing logic is triggered - Thread.sleep(500) - // check Partition Owner Registry - val actual_4 = getZKChildrenValues(dirs.consumerOwnerDir) - val expected_4 = List( ("200-0", "group1_consumer1-0"), - ("200-1", "group1_consumer1-0"), - ("300-0", "group1_consumer2-0"), - ("300-1", "group1_consumer2-0") ) - checkSetEqual(actual_4, expected_4) - } - - zkConsumerConnector1.shutdown - ZKConsumerConnector2.shutdown - } - - private def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = { - import scala.collection.JavaConversions - val children = zkClient.getChildren(path) - Collections.sort(children) - val childrenAsSeq : Seq[java.lang.String] = JavaConversions.asBuffer(children) - childrenAsSeq.map(partition => - (partition, zkClient.readData(path + "/" + partition).asInstanceOf[String])) - } - - private def checkSetEqual(actual : Seq[Tuple2[String,String]], expected : Seq[Tuple2[String,String]]) { - assertEquals(expected.length, actual.length) - for (i <- 0 until expected.length) { - assertEquals(expected(i)._1, actual(i)._1) - assertEquals(expected(i)._2, actual(i)._2) - } - } -} Index: core/src/test/scala/unit/kafka/server/CoordinatorRebalanceTest.scala =================================================================== --- core/src/test/scala/unit/kafka/server/CoordinatorRebalanceTest.scala (revision 0) +++ core/src/test/scala/unit/kafka/server/CoordinatorRebalanceTest.scala (revision 0) @@ -0,0 +1,440 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server + +import kafka.producer.{SyncProducer, SyncProducerConfig} +import java.util.{Collections, Properties} +import org.scalatest.junit.{JUnit3Suite, JUnitSuite} +import org.junit.Test +import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} +import kafka.utils._ +import junit.framework.Assert._ +import kafka.consumer._ +import kafka.consumer.Whitelist +import kafka.zk.ZooKeeperTestHarness + + +class CoordinatorRebalanceTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { + + // Global parameters + val zkConnect = TestZKUtils.zookeeperConnect + val zkSessionTimeoutMs = 1000 + val zkConnectionTimeoutMs = 1000 + + val topic1 = "topic1" + val topic2 = "topic2" + + // Broker parameters + val brokerId1 = 1 + val brokerId2 = 2 + val brokerId3 = 3 + + val brokerPort1 = TestUtils.choosePort() + val brokerPort2 = TestUtils.choosePort() + val brokerPort3 = TestUtils.choosePort() + + val brokerConfigProps1 = TestUtils.createBrokerConfig(brokerId1, brokerPort1) + val brokerConfigProps2 = TestUtils.createBrokerConfig(brokerId2, brokerPort2) + val brokerConfigProps3 = TestUtils.createBrokerConfig(brokerId3, brokerPort3) + + // Consumer parameters + val group = "group1" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val consumer3 = "consumer3" + val consumer4 = "consumer4" + + val consumerConfig1 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer1)) { + override val coordinatorSessionTimeoutMs = 2000 + override val consumerTimeoutMs = 1000 + override val fetchSize = 80 + override val autoCommit = true + override val initBrokerList = "{ \"1\" : \"localhost," + brokerPort1.toString + "\"," + + "\"2\" : \"localhost," + brokerPort2.toString + "\"," + + "\"3\" : \"localhost," + brokerPort3.toString + "\"}" + } + val consumerConfig2 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { + override val coordinatorSessionTimeoutMs = 2000 + override val consumerTimeoutMs = 1000 + override val fetchSize = 80 + override val autoCommit = true + override val initBrokerList = "{ \"1\" : \"localhost," + brokerPort1.toString + "\"," + + "\"2\" : \"localhost," + brokerPort2.toString + "\"," + + "\"3\" : \"localhost," + brokerPort3.toString + "\"}" + } + val consumerConfig3 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer3)) { + override val coordinatorSessionTimeoutMs = 2000 + override val consumerTimeoutMs = 1000 + override val fetchSize = 80 + override val autoCommit = true + override val initBrokerList = "{ \"1\" : \"localhost," + brokerPort1.toString + "\"," + + "\"2\" : \"localhost," + brokerPort2.toString + "\"," + + "\"3\" : \"localhost," + brokerPort3.toString + "\"}" + } + val consumerConfig4 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer4)) { + override val coordinatorSessionTimeoutMs = 2000 + override val consumerTimeoutMs = 1000 + override val fetchSize = 80 + override val autoCommit = true + override val initBrokerList = "{ \"1\" : \"localhost," + brokerPort1.toString + "\"," + + "\"2\" : \"localhost," + brokerPort2.toString + "\"," + + "\"3\" : \"localhost," + brokerPort3.toString + "\"}" + } + + // Producer parameters + val producerProps1 = new Properties() + producerProps1.put("host", "localhost") + producerProps1.put("port", brokerPort1.toString) + + val producerProps2 = new Properties() + producerProps2.put("host", "localhost") + producerProps2.put("port", brokerPort2.toString) + + val producerProps3 = new Properties() + producerProps3.put("host", "localhost") + producerProps3.put("port", brokerPort3.toString) + + @Test + def testCoordinatorRebalance() { + // Create and start servers + var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] + + val server1 = TestUtils.createServer(new KafkaConfig(brokerConfigProps1){override val numPartitions = 2}) + + Thread.sleep(500) // make sure server1 is the coordinator + + val server2 = TestUtils.createServer(new KafkaConfig(brokerConfigProps2){override val numPartitions = 2}) + val server3 = TestUtils.createServer(new KafkaConfig(brokerConfigProps3){override val numPartitions = 2}) + + servers ++= List(server1, server2, server3) + + var curCoordId = zkClient.readData(ZkUtils.ConsumerCoordinatorPath).toString.toInt + assertEquals("Coordinator is not server 1", curCoordId, 1) + + var curCood = (if (curCoordId == 1) server1 else if (curCoordId == 2) server2 else server3) + + // Create one producer per broker and send/create topics: + // three brokers, with each broker maintaining 2 partitions for each topic + // + // Broker1: topic1: {partition0, partition1} + // topic2: {partition0, partition1} + // + // Broker2: topic1: {partition0, partition1} + // topic2: {partition0, partition1} + // + // Broker3: topic1: {partition0, partition1} + // topic2: {partition0, partition1} + + val producer1 = new SyncProducer(new SyncProducerConfig(producerProps1)) + val producer2 = new SyncProducer(new SyncProducerConfig(producerProps2)) + val producer3 = new SyncProducer(new SyncProducerConfig(producerProps3)) + + producer1.send(topic1, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker1:topic1-partition0".getBytes()))) + producer1.send(topic1, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker1:topic2-partition1".getBytes()))) + producer1.send(topic2, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker1:topic1-partition0".getBytes()))) + producer1.send(topic2, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker1:topic2-partition1".getBytes()))) + + producer2.send(topic1, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker2:topic1-partition0".getBytes()))) + producer2.send(topic1, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker2:topic2-partition1".getBytes()))) + producer2.send(topic2, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker2:topic1-partition0".getBytes()))) + producer2.send(topic2, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker2:topic2-partition1".getBytes()))) + + producer3.send(topic1, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker3:topic1-partition0".getBytes()))) + producer3.send(topic1, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker3:topic2-partition1".getBytes()))) + producer3.send(topic2, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker3:topic1-partition0".getBytes()))) + producer3.send(topic2, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker3:topic2-partition1".getBytes()))) + + val groupTopic1 = new ZKGroupTopicDirs(group, topic1) + val groupTopic2 = new ZKGroupTopicDirs(group, topic2) + + // Create consumers + val consumer1 = new SimpleConsumerConnector(consumerConfig1, true) + val consumer2 = new SimpleConsumerConnector(consumerConfig2, true) + val consumer3 = new SimpleConsumerConnector(consumerConfig3, true) + + consumer1.createMessageStreams(Predef.Map(topic1 -> 1, topic2 -> 1)) + consumer2.createMessageStreams(Predef.Map(topic1 -> 1, topic2 -> 1)) + consumer3.createMessageStreams(Predef.Map(topic1 -> 1, topic2 -> 1)) + + // Check the rebalance logic + // + // Group1: + // Topic1: + // Consumer1-1 : {Broker1-0, Broker1-1} + // Consumer2-1 : {Broker2-0, Broker2-1} + // Consumer3-1 : {Broker3-0, Broker3-1} + // + // Topic2: + // Consumer1-1 : {Broker1-0, Broker1-1} + // Consumer2-1 : {Broker2-0, Broker2-1} + // Consumer3-1 : {Broker3-0, Broker3-1} + var expected_1 = List( ("1-0", "group1_consumer1-0"), + ("1-1", "group1_consumer1-0"), + ("2-0", "group1_consumer2-0"), + ("2-1", "group1_consumer2-0"), + ("3-0", "group1_consumer3-0"), + ("3-1", "group1_consumer3-0")) + + var expected_2 = List( ("1-0", "group1_consumer1-0"), + ("1-1", "group1_consumer1-0"), + ("2-0", "group1_consumer2-0"), + ("2-1", "group1_consumer2-0"), + ("3-0", "group1_consumer3-0"), + ("3-1", "group1_consumer3-0")) + + + assertTrue( + TestUtils.waitUntilTrue(() => { + val actual_1 = ZkUtils.getZKChildrenValues(zkClient, groupTopic1.consumerOwnerDir) + val actual_2 = ZkUtils.getZKChildrenValues(zkClient, groupTopic2.consumerOwnerDir) + + actual_1.toSet == expected_1.toSet && actual_2.toSet == expected_2.toSet + }, 5000) + ) + + // Shut down the current coordinator and check if the new coordinator does the work + //------------------------------------------------------------------------------// + curCood.shutdown() + + // Check the rebalance logic + // + // Group1: + // Topic1: + // Consumer1-1 : {Broker2-0, Broker2-1} + // Consumer2-1 : {Broker3-0} + // Consumer3-1 : {Broker3-1} + // + // Topic2: + // Consumer1-1 : {Broker2-0, Broker2-1} + // Consumer2-1 : {Broker3-0} + // Consumer3-1 : {Broker3-1} + + expected_1 = List( ("2-0", "group1_consumer1-0"), + ("2-1", "group1_consumer1-0"), + ("3-0", "group1_consumer2-0"), + ("3-1", "group1_consumer3-0")) + + expected_2 = List( ("2-0", "group1_consumer1-0"), + ("2-1", "group1_consumer1-0"), + ("3-0", "group1_consumer2-0"), + ("3-1", "group1_consumer3-0")) + + assertTrue( + TestUtils.waitUntilTrue(() => { + val actual_1 = ZkUtils.getZKChildrenValues(zkClient, groupTopic1.consumerOwnerDir) + val actual_2 = ZkUtils.getZKChildrenValues(zkClient, groupTopic2.consumerOwnerDir) + + actual_1.toSet == expected_1.toSet && actual_2.toSet == expected_2.toSet + }, 5000) + ) + + var oldCood = curCood + curCoordId = zkClient.readData(ZkUtils.ConsumerCoordinatorPath).toString.toInt + curCood = (if (curCoordId == 1) server1 else if (curCoordId == 2) server2 else server3) + + // Restart the shutdown server and check if the rebalance can catch up + //------------------------------------------------------------------------------// + oldCood.startup() + + expected_1 = List( ("1-0", "group1_consumer1-0"), + ("1-1", "group1_consumer1-0"), + ("2-0", "group1_consumer2-0"), + ("2-1", "group1_consumer2-0"), + ("3-0", "group1_consumer3-0"), + ("3-1", "group1_consumer3-0")) + + expected_2 = List( ("1-0", "group1_consumer1-0"), + ("1-1", "group1_consumer1-0"), + ("2-0", "group1_consumer2-0"), + ("2-1", "group1_consumer2-0"), + ("3-0", "group1_consumer3-0"), + ("3-1", "group1_consumer3-0")) + + assertTrue( + TestUtils.waitUntilTrue(() => { + val actual_1 = ZkUtils.getZKChildrenValues(zkClient, groupTopic1.consumerOwnerDir) + val actual_2 = ZkUtils.getZKChildrenValues(zkClient, groupTopic2.consumerOwnerDir) + + actual_1.toSet == expected_1.toSet && actual_2.toSet == expected_2.toSet + }, 5000) + ) + + + // Shut down one consumer and check if the new coordinator does the work + //------------------------------------------------------------------------------// + consumer3.shutdown() + + // Check the rebalance logic + // + // Group1: + // Topic1: + // Consumer1-1 : {Broker1-0, Broker1-1, Broker2-0} + // Consumer2-1 : {Broker2-1, Broker3-0, Broker3-1} + // + // Topic2: + // Consumer1-1 : {Broker1-0, Broker1-1, Broker2-0} + // Consumer2-1 : {Broker2-1, Broker3-0, Broker3-1} + + expected_1 = List( ("1-0", "group1_consumer1-0"), + ("1-1", "group1_consumer1-0"), + ("2-0", "group1_consumer1-0"), + ("2-1", "group1_consumer2-0"), + ("3-0", "group1_consumer2-0"), + ("3-1", "group1_consumer2-0")) + + expected_2 = List( ("1-0", "group1_consumer1-0"), + ("1-1", "group1_consumer1-0"), + ("2-0", "group1_consumer1-0"), + ("2-1", "group1_consumer2-0"), + ("3-0", "group1_consumer2-0"), + ("3-1", "group1_consumer2-0")) + + assertTrue( + TestUtils.waitUntilTrue(() => { + val actual_1 = ZkUtils.getZKChildrenValues(zkClient, groupTopic1.consumerOwnerDir) + val actual_2 = ZkUtils.getZKChildrenValues(zkClient, groupTopic2.consumerOwnerDir) + + actual_1.toSet == expected_1.toSet && actual_2.toSet == expected_2.toSet + }, 5000) + ) + + // Restart the shutdown consumer and check if the rebalance can catch up + //------------------------------------------------------------------------------// + val consumer4 = new SimpleConsumerConnector(consumerConfig4, true) + + consumer4.createMessageStreamsByFilter(new Whitelist("""[\p{Alnum}]+"""), 1) + + expected_1 = List( ("1-0", "group1_consumer1-0"), + ("1-1", "group1_consumer1-0"), + ("2-0", "group1_consumer2-0"), + ("2-1", "group1_consumer2-0"), + ("3-0", "group1_consumer4-0"), + ("3-1", "group1_consumer4-0")) + + expected_2 = List( ("1-0", "group1_consumer1-0"), + ("1-1", "group1_consumer1-0"), + ("2-0", "group1_consumer2-0"), + ("2-1", "group1_consumer2-0"), + ("3-0", "group1_consumer4-0"), + ("3-1", "group1_consumer4-0")) + + assertTrue( + TestUtils.waitUntilTrue(() => { + val actual_1 = ZkUtils.getZKChildrenValues(zkClient, groupTopic1.consumerOwnerDir) + val actual_2 = ZkUtils.getZKChildrenValues(zkClient, groupTopic2.consumerOwnerDir) + + actual_1.toSet == expected_1.toSet && actual_2.toSet == expected_2.toSet + }, 5000) + ) + + // Shut down a consumer and a server at the same time and check the rebalance result + //------------------------------------------------------------------------------// + server3.shutdown() + consumer1.shutdown() + + // Check the rebalance logic + // + // Group1: + // Topic1: + // Consumer1-1 : {Broker1-0, Broker1-1} + // Consumer2-1 : {Broker2-0, Broker2-1} + // + // Topic2: + // Consumer1-1 : {Broker1-0, Broker1-1} + // Consumer2-1 : {Broker2-0, Broker2-1} + + expected_1 = List( ("1-0", "group1_consumer2-0"), + ("1-1", "group1_consumer2-0"), + ("2-0", "group1_consumer4-0"), + ("2-1", "group1_consumer4-0")) + + expected_2 = List( ("1-0", "group1_consumer2-0"), + ("1-1", "group1_consumer2-0"), + ("2-0", "group1_consumer4-0"), + ("2-1", "group1_consumer4-0")) + + assertTrue( + TestUtils.waitUntilTrue(() => { + val actual_1 = ZkUtils.getZKChildrenValues(zkClient, groupTopic1.consumerOwnerDir) + val actual_2 = ZkUtils.getZKChildrenValues(zkClient, groupTopic2.consumerOwnerDir) + + actual_1.toSet == expected_1.toSet && actual_2.toSet == expected_2.toSet + }, 5000) + ) + + + oldCood = curCood + curCoordId = zkClient.readData(ZkUtils.ConsumerCoordinatorPath).toString.toInt + curCood = (if (curCoordId == 1) server1 else if (curCoordId == 2) server2 else server3) + + // Restart the shutdown server and check if the rebalance can catch up + //------------------------------------------------------------------------------// + server3.startup() + + expected_1 = List( ("1-0", "group1_consumer2-0"), + ("1-1", "group1_consumer2-0"), + ("2-0", "group1_consumer2-0"), + ("2-1", "group1_consumer4-0"), + ("3-0", "group1_consumer4-0"), + ("3-1", "group1_consumer4-0")) + + expected_2 = List( ("1-0", "group1_consumer2-0"), + ("1-1", "group1_consumer2-0"), + ("2-0", "group1_consumer2-0"), + ("2-1", "group1_consumer4-0"), + ("3-0", "group1_consumer4-0"), + ("3-1", "group1_consumer4-0")) + + assertTrue( + TestUtils.waitUntilTrue(() => { + val actual_1 = ZkUtils.getZKChildrenValues(zkClient, groupTopic1.consumerOwnerDir) + val actual_2 = ZkUtils.getZKChildrenValues(zkClient, groupTopic2.consumerOwnerDir) + + actual_1.toSet == expected_1.toSet && actual_2.toSet == expected_2.toSet + }, 5000) + ) + + //consumer1.shutdown() + consumer2.shutdown() + consumer4.shutdown() + + producer1.close() + producer2.close() + producer3.close() + + server1.shutdown() + server2.shutdown() + server3.shutdown() + } + +} Index: core/src/test/scala/unit/kafka/network/SocketServerTest.scala =================================================================== --- core/src/test/scala/unit/kafka/network/SocketServerTest.scala (revision 1371523) +++ core/src/test/scala/unit/kafka/network/SocketServerTest.scala (working copy) @@ -24,6 +24,7 @@ import kafka.utils.TestUtils import java.util.Random import org.apache.log4j._ +import kafka.server.KafkaRequestHandlers class SocketServerTest extends JUnitSuite { @@ -33,11 +34,17 @@ val id = receive.buffer.getShort Some(new BoundedByteBufferSend(receive.buffer.slice)) } + + class EchoHandler extends KafkaRequestHandlers(null, null) { + override def handlerFor(requestId: Short, receive: Receive): Handler.Handler = echo + } + + val echoHandler = new EchoHandler val server = new SocketServer(port = TestUtils.choosePort, numProcessorThreads = 1, monitoringPeriodSecs = 30, - handlerFactory = (requestId: Short, receive: Receive) => echo, + handlerFactory = echoHandler, sendBufferSize = 300000, receiveBufferSize = 300000, maxRequestSize = 50) Index: core/src/test/scala/unit/kafka/consumer/SimpleConsumerConnectorTest.scala =================================================================== --- core/src/test/scala/unit/kafka/consumer/SimpleConsumerConnectorTest.scala (revision 0) +++ core/src/test/scala/unit/kafka/consumer/SimpleConsumerConnectorTest.scala (revision 0) @@ -0,0 +1,497 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.consumer + +import org.scalatest.junit.{JUnit3Suite, JUnitSuite} +import kafka.utils._ +import junit.framework.Assert._ +import org.junit.Test +import java.util.Properties +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.producer.{SyncProducerConfig, SyncProducer} +import kafka.message.{Message, NoCompressionCodec, ByteBufferMessageSet} +import kafka.zk.ZooKeeperTestHarness + +class SimpleConsumerConnectorTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { + + // Global parameters + val zkConnect = TestZKUtils.zookeeperConnect + val zkSessionTimeoutMs = 1000 + val zkConnectionTimeoutMs = 1000 + + // Broker parameters + val brokerId1 = 1 + val brokerId2 = 2 + val brokerId3 = 3 + + val topic1 = "topic1" + val topic2 = "topic2" + + val brokerPort1 = TestUtils.choosePort() + val brokerPort2 = TestUtils.choosePort() + val brokerPort3 = TestUtils.choosePort() + + val brokerConfigProps1 = TestUtils.createBrokerConfig(brokerId1, brokerPort1) + val brokerConfigProps2 = TestUtils.createBrokerConfig(brokerId2, brokerPort2) + val brokerConfigProps3 = TestUtils.createBrokerConfig(brokerId3, brokerPort3) + + // Consumer parameters + val group = "group1" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val consumer3 = "consumer3" + val consumer4 = "consumer4" + + val consumerConfig1 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer1)) { + override val coordinatorSessionTimeoutMs = 2000 + override val consumerTimeoutMs = 1000 + override val fetchSize = 80 + override val autoCommit = true + override val initBrokerList = "{ \"1\" : \"localhost," + brokerPort1.toString + "\"," + + "\"2\" : \"localhost," + brokerPort2.toString + "\"," + + "\"3\" : \"localhost," + brokerPort3.toString + "\"}" + } + val consumerConfig2 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { + override val coordinatorSessionTimeoutMs = 2000 + override val consumerTimeoutMs = 1000 + override val fetchSize = 80 + override val autoCommit = true + override val initBrokerList = "{ \"1\" : \"localhost," + brokerPort1.toString + "\"," + + "\"2\" : \"localhost," + brokerPort2.toString + "\"," + + "\"3\" : \"localhost," + brokerPort3.toString + "\"}" + } + val consumerConfig3 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer3)) { + override val coordinatorSessionTimeoutMs = 2000 + override val consumerTimeoutMs = 1000 + override val fetchSize = 80 + override val autoCommit = true + override val initBrokerList = "{ \"1\" : \"localhost," + brokerPort1.toString + "\"," + + "\"2\" : \"localhost," + brokerPort2.toString + "\"," + + "\"3\" : \"localhost," + brokerPort3.toString + "\"}" + } + val consumerConfig4 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer4)) { + override val coordinatorSessionTimeoutMs = 2000 + override val consumerTimeoutMs = 1000 + override val fetchSize = 80 + override val autoCommit = true + override val initBrokerList = "{ \"1\" : \"localhost," + brokerPort1.toString + "\"," + + "\"2\" : \"localhost," + brokerPort2.toString + "\"," + + "\"3\" : \"localhost," + brokerPort3.toString + "\"}" + } + + // Producer parameters + val producerProps1 = new Properties() + producerProps1.put("host", "localhost") + producerProps1.put("port", brokerPort1.toString) + + val producerProps2 = new Properties() + producerProps2.put("host", "localhost") + producerProps2.put("port", brokerPort2.toString) + + val producerProps3 = new Properties() + producerProps3.put("host", "localhost") + producerProps3.put("port", brokerPort3.toString) + + @Test + def testConsumerDiscovery() { + // Create and start servers + var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] + + val server1 = TestUtils.createServer(new KafkaConfig(brokerConfigProps1) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + val server2 = TestUtils.createServer(new KafkaConfig(brokerConfigProps2) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + val server3 = TestUtils.createServer(new KafkaConfig(brokerConfigProps3) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + + servers ++= List(server1, server2, server3) + + val curCoordId = zkClient.readData(ZkUtils.ConsumerCoordinatorPath).toString.toInt + val curCood = (if (curCoordId == 1) server1 else if (curCoordId == 2) server2 else server3) + + // Create consumers + val consumer1 = new SimpleConsumerConnector(consumerConfig1, true) + val consumer2 = new SimpleConsumerConnector(consumerConfig2, true) + val consumer3 = new SimpleConsumerConnector(consumerConfig3, true) + + consumer1.createMessageStreams(Predef.Map()) + consumer2.createMessageStreams(Predef.Map()) + consumer3.createMessageStreams(Predef.Map()) + + assertEquals("consumer1 and consumer2 should receive the same coordinator ID from the metadata" + , consumer1.coordinatorId, consumer2.coordinatorId) + assertEquals("consumer1 and consumer3 should receive the same coordinator ID from the metadata" + , consumer1.coordinatorId, consumer3.coordinatorId) + + assertEquals("consumer1 and consumer2 should receive the same cluster info from the metadata" + , consumer1.kafkaCluster.toString, consumer2.kafkaCluster.toString) + assertEquals("consumer1 and consumer3 should receive the same cluster info from the metadata" + , consumer1.kafkaCluster.toString, consumer3.kafkaCluster.toString) + + Thread.sleep(1000) + + assertEquals("Coordinator should have registered all three consumers", + curCood.consumerCoordinator.readConsumerRegistriesSnapshot(group).size, 3) + + consumer1.shutdown() + consumer2.shutdown() + consumer3.shutdown() + + server1.shutdown() + server2.shutdown() + server3.shutdown() + } + + + @Test + def testConsumerFailureDetection() { + // Create and start servers + var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] + + val server1 = TestUtils.createServer(new KafkaConfig(brokerConfigProps1) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + val server2 = TestUtils.createServer(new KafkaConfig(brokerConfigProps2) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + val server3 = TestUtils.createServer(new KafkaConfig(brokerConfigProps3) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + + servers ++= List(server1, server2, server3) + + val curCoordId = zkClient.readData(ZkUtils.ConsumerCoordinatorPath).toString.toInt + val curCood = (if (curCoordId == 1) server1 else if (curCoordId == 2) server2 else server3) + + // Create consumers + val consumer1 = new SimpleConsumerConnector(consumerConfig1, true) + val consumer2 = new SimpleConsumerConnector(consumerConfig2, true) + val consumer3 = new SimpleConsumerConnector(consumerConfig3, true) + + consumer1.createMessageStreams(Predef.Map(topic1 -> 1, topic2 -> 1)) + consumer2.createMessageStreams(Predef.Map(topic1 -> 1, topic2 -> 1)) + consumer3.createMessageStreams(Predef.Map(topic1 -> 1, topic2 -> 1)) + + Thread.sleep(500) + + // Physically fail one consumer + consumer1.shutdown() + + assertTrue(TestUtils.waitUntilTrue(() => curCood.consumerCoordinator.isGroupRegistered(group) + && curCood.consumerCoordinator.readConsumerRegistriesSnapshot(group).size == 2, 5000)) + + // Create another consumer + val consumer4 = new SimpleConsumerConnector(consumerConfig4, true) + consumer4.createMessageStreams(Predef.Map()) + + assertTrue(TestUtils.waitUntilTrue(() => curCood.consumerCoordinator.readConsumerRegistriesSnapshot(group).size == 3, 5000)) + + // Virtually fail one consumer by suspending its coordinator coordinator + consumer4.blockConnector() + + assertTrue(TestUtils.waitUntilTrue(() => curCood.consumerCoordinator.readConsumerRegistriesSnapshot(group).size == 2, 20000)) + + // Resume the consumer and lets it to re-connect to the coordinator + consumer4.unblockConnector() + + assertTrue(TestUtils.waitUntilTrue(() => curCood.consumerCoordinator.readConsumerRegistriesSnapshot(group).size == 3, 5000)) + + consumer4.shutdown() + consumer2.shutdown() + consumer3.shutdown() + + server1.shutdown() + server2.shutdown() + server3.shutdown() + + } + + + @Test + def testCoordinatorFailureDetection() { + // Create and start servers + var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] + + val server1 = TestUtils.createServer(new KafkaConfig(brokerConfigProps1) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + val server2 = TestUtils.createServer(new KafkaConfig(brokerConfigProps2) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + val server3 = TestUtils.createServer(new KafkaConfig(brokerConfigProps3) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + + servers ++= List(server1, server2, server3) + + var curCoordId = zkClient.readData(ZkUtils.ConsumerCoordinatorPath).toString.toInt + var curCood = (if (curCoordId == 1) server1 else if (curCoordId == 2) server2 else server3) + + // Create consumers + val consumer1 = new SimpleConsumerConnector(consumerConfig1, true) + val consumer2 = new SimpleConsumerConnector(consumerConfig2, true) + val consumer3 = new SimpleConsumerConnector(consumerConfig3, true) + + consumer1.createMessageStreams(Predef.Map()) + consumer2.createMessageStreams(Predef.Map()) + consumer3.createMessageStreams(Predef.Map()) + + Thread.sleep(500) + + // Virtually blocking the current coordinator + curCood.consumerCoordinator.blockPingCommands() + + Thread.sleep(3000) + + curCood.consumerCoordinator.unblockPingCommands() + + assertTrue(TestUtils.waitUntilTrue(() => curCood.consumerCoordinator.isGroupRegistered(group) + && curCood.consumerCoordinator.readConsumerRegistriesSnapshot(group).size == 3, 2000)) + + // Physically shutdown the current coordinator + curCood.shutdown() + + assertTrue(TestUtils.waitUntilTrue(() => zkClient.readData(ZkUtils.ConsumerCoordinatorPath).toString.toInt != curCoordId, 1000)) + + curCoordId = zkClient.readData(ZkUtils.ConsumerCoordinatorPath).toString.toInt + curCood = (if (curCoordId == 1) server1 else if (curCoordId == 2) server2 else server3) + + assertTrue(TestUtils.waitUntilTrue(() => curCood.consumerCoordinator.isGroupRegistered(group) + && curCood.consumerCoordinator.readConsumerRegistriesSnapshot(group).size == 3, 4000)) + + consumer1.shutdown() + consumer2.shutdown() + consumer3.shutdown() + + server1.shutdown() + server2.shutdown() + server3.shutdown() + } + + @Test + def testOffsetCommit() { + // Create and start servers + var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] + + val server1 = TestUtils.createServer(new KafkaConfig(brokerConfigProps1) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + val server2 = TestUtils.createServer(new KafkaConfig(brokerConfigProps2) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + val server3 = TestUtils.createServer(new KafkaConfig(brokerConfigProps3) { + override val numPartitions = 2 + override val autoCommitIntervalMs = 2000 + }) + + servers ++= List(server1, server2, server3) + + // Create one producer per broker and send/create topics: + // three brokers, with each broker maintaining 2 partitions for each topic + // + // Broker1: topic1: {partition0, partition1} + // topic2: {partition0, partition1} + // + // Broker2: topic1: {partition0, partition1} + // topic2: {partition0, partition1} + // + // Broker3: topic1: {partition0, partition1} + // topic2: {partition0, partition1} + + val producer1 = new SyncProducer(new SyncProducerConfig(producerProps1)) + val producer2 = new SyncProducer(new SyncProducerConfig(producerProps2)) + val producer3 = new SyncProducer(new SyncProducerConfig(producerProps3)) + + for (i <- 0 until 100) { + producer1.send(topic1, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker1:topic1-partition0".getBytes()))) + producer1.send(topic1, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker1:topic2-partition1".getBytes()))) + producer1.send(topic2, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker1:topic1-partition0".getBytes()))) + producer1.send(topic2, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker1:topic2-partition1".getBytes()))) + + producer2.send(topic1, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker2:topic1-partition0".getBytes()))) + producer2.send(topic1, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker2:topic2-partition1".getBytes()))) + producer2.send(topic2, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker2:topic1-partition0".getBytes()))) + producer2.send(topic2, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker2:topic2-partition1".getBytes()))) + + producer3.send(topic1, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker3:topic1-partition0".getBytes()))) + producer3.send(topic1, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker3:topic2-partition1".getBytes()))) + producer3.send(topic2, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker3:topic1-partition0".getBytes()))) + producer3.send(topic2, 1, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + messages = new Message("broker3:topic2-partition1".getBytes()))) + } + + // Create consumers + val consumerConnector1 = new SimpleConsumerConnector(consumerConfig1, true) + val consumerConnector2 = new SimpleConsumerConnector(consumerConfig2, true) + val consumerConnector3 = new SimpleConsumerConnector(consumerConfig3, true) + + val topicMessageStreams1 = consumerConnector1.createMessageStreams(Predef.Map(topic1 -> 1, topic2 -> 1)) + val topicMessageStreams2 = consumerConnector2.createMessageStreams(Predef.Map(topic1 -> 1, topic2 -> 1)) + val topicMessageStreams3 = consumerConnector3.createMessageStreams(Predef.Map(topic1 -> 1, topic2 -> 1)) + + val topic1Stream1 = topicMessageStreams1(topic1) + val topic2Stream1 = topicMessageStreams1(topic2) + val topic1Stream2 = topicMessageStreams2(topic1) + val topic2Stream2 = topicMessageStreams2(topic2) + val topic1Stream3 = topicMessageStreams3(topic1) + val topic2Stream3 = topicMessageStreams3(topic2) + + var done = false + while (!done) { + done = true + + for (stream <- topic1Stream1) { + val iter = stream.iterator() + + try { + while(iter.hasNext()) { + done = false + iter.next() + } + } catch { + case e : ConsumerTimeoutException => + case e1 => throw e1 + } + } + for (stream <- topic2Stream1) { + val iter = stream.iterator() + + try { + while(iter.hasNext()) { + done = false + iter.next() + } + } catch { + case e : ConsumerTimeoutException => + case e1 => throw e1 + } + } + for (stream <- topic1Stream2) { + val iter = stream.iterator() + + try { + while(iter.hasNext()) { + done = false + iter.next() + } + } catch { + case e : ConsumerTimeoutException => + case e1 => throw e1 + } + } + for (stream <- topic2Stream2) { + val iter = stream.iterator() + + try { + while(iter.hasNext()) { + done = false + iter.next() + } + } catch { + case e : ConsumerTimeoutException => + case e1 => throw e1 + } + } + for (stream <- topic1Stream3) { + val iter = stream.iterator() + + try { + while(iter.hasNext()) { + done = false + iter.next() + } + } catch { + case e : ConsumerTimeoutException => + case e1 => throw e1 + } + } + for (stream <- topic2Stream3) { + val iter = stream.iterator() + + try { + while(iter.hasNext()) { + done = false + iter.next() + } + } catch { + case e : ConsumerTimeoutException => + case e1 => throw e1 + } + } + } + + val expected_1 = List( ("1-0", "3500"), + ("1-1", "3500"), + ("2-0", "3500"), + ("2-1", "3500"), + ("3-0", "3500"), + ("3-1", "3500")) + + val expected_2 = List( ("1-0", "3500"), + ("1-1", "3500"), + ("2-0", "3500"), + ("2-1", "3500"), + ("3-0", "3500"), + ("3-1", "3500")) + + val groupTopic1 = new ZKGroupTopicDirs(group, topic1) + val groupTopic2 = new ZKGroupTopicDirs(group, topic2) + + assertTrue(TestUtils.waitUntilTrue(() => + expected_1.toSet == ZkUtils.getZKChildrenValues(zkClient, groupTopic1.consumerOffsetDir).toSet + && expected_2.toSet == ZkUtils.getZKChildrenValues(zkClient, groupTopic2.consumerOffsetDir).toSet + , 5000)) + + consumerConnector1.shutdown() + consumerConnector2.shutdown() + consumerConnector3.shutdown() + + server1.shutdown() + server2.shutdown() + server3.shutdown() + } +} Index: core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala =================================================================== --- core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala (revision 1371523) +++ core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala (working copy) @@ -1,288 +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.consumer - -import junit.framework.Assert._ -import kafka.zk.ZooKeeperTestHarness -import kafka.integration.KafkaServerTestHarness -import kafka.server.KafkaConfig -import scala.collection._ -import kafka.utils.{Utils, Logging} -import kafka.utils.{TestZKUtils, TestUtils} -import org.scalatest.junit.JUnit3Suite -import org.apache.log4j.{Level, Logger} -import kafka.message._ -import kafka.serializer.StringDecoder - -class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { - - val zookeeperConnect = TestZKUtils.zookeeperConnect - val zkConnect = zookeeperConnect - val numNodes = 2 - val numParts = 2 - val topic = "topic1" - val configs = - for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) { - override val enableZookeeper = true - override val numPartitions = numParts - override val zkConnect = zookeeperConnect - } - val group = "group1" - val consumer0 = "consumer0" - val consumer1 = "consumer1" - val consumer2 = "consumer2" - val consumer3 = "consumer3" - val nMessages = 2 - - def testBasic() { - val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) - requestHandlerLogger.setLevel(Level.FATAL) - - var actualMessages: List[Message] = Nil - - // test consumer timeout logic - val consumerConfig0 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer0)) { - override val consumerTimeoutMs = 200 - } - val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true) - val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) - - // no messages to consume, we should hit timeout; - // also the iterator should support re-entrant, so loop it twice - for (i <- 0 until 2) { - try { - getMessages(nMessages*2, topicMessageStreams0) - fail("should get an exception") - } - catch { - case e: ConsumerTimeoutException => // this is ok - case e => throw e - } - } - - zkConsumerConnector0.shutdown - - // send some messages to each broker - val sentMessages1 = sendMessages(nMessages, "batch1") - // create a consumer - val consumerConfig1 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) - val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) - assertEquals(sentMessages1, receivedMessages1) - // commit consumed offsets - zkConsumerConnector1.commitOffsets - - // create a consumer - val consumerConfig2 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer2)) - val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) - val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) - // send some messages to each broker - val sentMessages2 = sendMessages(nMessages, "batch2") - Thread.sleep(200) - val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1) - val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2) - val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum) - assertEquals(sentMessages2, receivedMessages2) - - // create a consumer with empty map - val consumerConfig3 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer3)) - val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) - val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) - // send some messages to each broker - Thread.sleep(200) - val sentMessages3 = sendMessages(nMessages, "batch3") - Thread.sleep(200) - val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1) - val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2) - val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum) - assertEquals(sentMessages3, receivedMessages3) - - zkConsumerConnector1.shutdown - zkConsumerConnector2.shutdown - zkConsumerConnector3.shutdown - info("all consumer connectors stopped") - requestHandlerLogger.setLevel(Level.ERROR) - } - - def testCompression() { - val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) - requestHandlerLogger.setLevel(Level.FATAL) - - println("Sending messages for 1st consumer") - // send some messages to each broker - val sentMessages1 = sendMessages(nMessages, "batch1", DefaultCompressionCodec) - // create a consumer - val consumerConfig1 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) - val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) - assertEquals(sentMessages1, receivedMessages1) - // commit consumed offsets - zkConsumerConnector1.commitOffsets - - println("Sending more messages for 2nd consumer") - // create a consumer - val consumerConfig2 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer2)) - val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) - val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) - // send some messages to each broker - val sentMessages2 = sendMessages(nMessages, "batch2", DefaultCompressionCodec) - Thread.sleep(200) - val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1) - val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2) - val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum) - assertEquals(sentMessages2, receivedMessages2) - - // create a consumer with empty map - println("Sending more messages for 3rd consumer") - val consumerConfig3 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer3)) - val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) - val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) - // send some messages to each broker - Thread.sleep(200) - val sentMessages3 = sendMessages(nMessages, "batch3", DefaultCompressionCodec) - Thread.sleep(200) - val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1) - val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2) - val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum) - assertEquals(sentMessages3, receivedMessages3) - - zkConsumerConnector1.shutdown - zkConsumerConnector2.shutdown - zkConsumerConnector3.shutdown - info("all consumer connectors stopped") - requestHandlerLogger.setLevel(Level.ERROR) - } - - def testCompressionSetConsumption() { - val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) - requestHandlerLogger.setLevel(Level.FATAL) - - var actualMessages: List[Message] = Nil - - // shutdown one server - servers.last.shutdown - Thread.sleep(500) - - // send some messages to each broker - val sentMessages = sendMessages(configs.head, 200, "batch1", DefaultCompressionCodec) - // test consumer timeout logic - val consumerConfig0 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer0)) { - override val consumerTimeoutMs = 5000 - } - val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true) - val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> 1)) - getMessages(100, topicMessageStreams0) - zkConsumerConnector0.shutdown - // at this point, only some part of the message set was consumed. So consumed offset should still be 0 - // also fetched offset should be 0 - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig0, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1)) - val receivedMessages = getMessages(400, topicMessageStreams1) - val sortedReceivedMessages = receivedMessages.sortWith((s,t) => s.checksum < t.checksum) - val sortedSentMessages = sentMessages.sortWith((s,t) => s.checksum < t.checksum) - assertEquals(sortedSentMessages, sortedReceivedMessages) - zkConsumerConnector1.shutdown - - requestHandlerLogger.setLevel(Level.ERROR) - } - - def testConsumerDecoder() { - val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) - requestHandlerLogger.setLevel(Level.FATAL) - - val sentMessages = sendMessages(nMessages, "batch1", NoCompressionCodec). - map(m => Utils.toString(m.payload, "UTF-8")). - sortWith((s, t) => s.compare(t) == -1) - val consumerConfig = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - - val zkConsumerConnector = - new ZookeeperConsumerConnector(consumerConfig, true) - val topicMessageStreams = - zkConsumerConnector.createMessageStreams( - Predef.Map(topic -> numNodes*numParts/2), new StringDecoder) - - var receivedMessages: List[String] = Nil - for ((topic, messageStreams) <- topicMessageStreams) { - for (messageStream <- messageStreams) { - val iterator = messageStream.iterator - for (i <- 0 until nMessages * 2) { - assertTrue(iterator.hasNext()) - val message = iterator.next().message - receivedMessages ::= message - debug("received message: " + message) - } - } - } - receivedMessages = receivedMessages.sortWith((s, t) => s.compare(t) == -1) - assertEquals(sentMessages, receivedMessages) - - zkConsumerConnector.shutdown() - requestHandlerLogger.setLevel(Level.ERROR) - } - - def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compression: CompressionCodec): List[Message]= { - var messages: List[Message] = Nil - val producer = TestUtils.createProducer("localhost", conf.port) - for (partition <- 0 until numParts) { - val ms = 0.until(messagesPerNode).map(x => - new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray - val mSet = new ByteBufferMessageSet(compressionCodec = compression, messages = ms: _*) - for (message <- ms) - messages ::= message - producer.send(topic, partition, mSet) - } - producer.close() - messages - } - - def sendMessages(messagesPerNode: Int, header: String, compression: CompressionCodec = NoCompressionCodec): List[Message]= { - var messages: List[Message] = Nil - for(conf <- configs) { - messages ++= sendMessages(conf, messagesPerNode, header, compression) - } - messages.sortWith((s,t) => s.checksum < t.checksum) - } - - def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaStream[Message]]]): List[Message]= { - var messages: List[Message] = Nil - for ((topic, messageStreams) <- topicMessageStreams) { - for (messageStream <- messageStreams) { - val iterator = messageStream.iterator - for (i <- 0 until nMessagesPerThread) { - assertTrue(iterator.hasNext) - val message = iterator.next.message - messages ::= message - debug("received message: " + Utils.toString(message.payload, "UTF-8")) - } - } - } - messages.sortWith((s,t) => s.checksum < t.checksum) - } -} Index: core/src/test/scala/unit/kafka/consumer/ConsumerTest.scala =================================================================== --- core/src/test/scala/unit/kafka/consumer/ConsumerTest.scala (revision 0) +++ core/src/test/scala/unit/kafka/consumer/ConsumerTest.scala (revision 0) @@ -0,0 +1,323 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.consumer + +import junit.framework.Assert._ +import kafka.zk.ZooKeeperTestHarness +import kafka.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import scala.collection._ +import kafka.utils._ +import org.scalatest.junit.JUnit3Suite +import org.apache.log4j.{Level, Logger} +import kafka.message._ +import kafka.serializer.StringDecoder + +class ConsumerTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { + + val zookeeperConnect = TestZKUtils.zookeeperConnect + val zkConnect = zookeeperConnect + val numNodes = 2 + val numParts = 2 + val topic = "topic1" + val configs = + for(props <- TestUtils.createBrokerConfigs(numNodes)) + yield new KafkaConfig(props) { + override val enableZookeeper = true + override val numPartitions = numParts + override val zkConnect = zookeeperConnect + } + + val group = "group1" + val consumer0 = "consumer0" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val consumer3 = "consumer3" + val nMessages = 2 + + val groupTopicDir = new ZKGroupTopicDirs(group, topic) + + def testBasic() { + val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) + requestHandlerLogger.setLevel(Level.FATAL) + + var actualMessages: List[Message] = Nil + + // test consumer timeout logic + val consumerConfig0 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer0)) { + override val consumerTimeoutMs = 200 + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector0 = new SimpleConsumerConnector(consumerConfig0, true) + val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) + + // no messages to consume, we should hit timeout; + // also the iterator should support re-entrant, so loop it twice + for (i <- 0 until 2) { + try { + getMessages(nMessages*2, topicMessageStreams0) + fail("should get an exception") + } + catch { + case e: ConsumerTimeoutException => // this is ok + case e => throw e + } + } + + zkConsumerConnector0.shutdown + + // send some messages to each broker + val sentMessages1 = sendMessages(nMessages, "batch1") + // create a consumer + val consumerConfig1 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer1)) { + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector1 = new SimpleConsumerConnector(consumerConfig1, true) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) + val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) + assertEquals(sentMessages1, receivedMessages1) + // commit consumed offsets + //zkConsumerConnector1.commitOffsets + + TestUtils.waitUntilTrue(() => ZkUtils.getZKChildrenValues(zkClient, groupTopicDir.consumerOffsetDir).toSet + == zkConsumerConnector1.getOffsetsForTopic(topic).toSet + , 15 * 1000) + + // TODO: current design does not support active commitOffsets call, which need to be incorporated after 0.8 along with modified SocketServer + + // create a consumer + val consumerConfig2 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector2 = new SimpleConsumerConnector(consumerConfig2, true) + val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) + // send some messages to each broker + val sentMessages2 = sendMessages(nMessages, "batch2") + Thread.sleep(200) + val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1) + val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2) + val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum) + assertEquals(sentMessages2, receivedMessages2) + + // create a consumer with empty map + val consumerConfig3 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer3)) { + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector3 = new SimpleConsumerConnector(consumerConfig3, true) + val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) + // send some messages to each broker + Thread.sleep(200) + val sentMessages3 = sendMessages(nMessages, "batch3") + Thread.sleep(200) + val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1) + val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2) + val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum) + assertEquals(sentMessages3, receivedMessages3) + + zkConsumerConnector1.shutdown + zkConsumerConnector2.shutdown + zkConsumerConnector3.shutdown + info("all consumer connectors stopped") + requestHandlerLogger.setLevel(Level.ERROR) + } + + def testCompression() { + val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) + requestHandlerLogger.setLevel(Level.FATAL) + + println("Sending messages for 1st consumer") + // send some messages to each broker + val sentMessages1 = sendMessages(nMessages, "batch1", DefaultCompressionCodec) + // create a consumer + val consumerConfig1 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer1)) { + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector1 = new SimpleConsumerConnector(consumerConfig1, true) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) + val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) + assertEquals(sentMessages1, receivedMessages1) + // commit consumed offsets + //zkConsumerConnector1.commitOffsets + + TestUtils.waitUntilTrue(() => ZkUtils.getZKChildrenValues(zkClient, groupTopicDir.consumerOffsetDir).toSet + == zkConsumerConnector1.getOffsetsForTopic(topic).toSet + , 15 * 1000) + + // TODO: current design does not support active commitOffsets call, which need to be incorporated after 0.8 along with modified SocketServer + + println("Sending more messages for 2nd consumer") + // create a consumer + val consumerConfig2 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector2 = new SimpleConsumerConnector(consumerConfig2, true) + val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) + // send some messages to each broker + val sentMessages2 = sendMessages(nMessages, "batch2", DefaultCompressionCodec) + Thread.sleep(200) + val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1) + val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2) + val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum) + assertEquals(sentMessages2, receivedMessages2) + + // create a consumer with empty map + println("Sending more messages for 3rd consumer") + val consumerConfig3 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer3)) { + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector3 = new SimpleConsumerConnector(consumerConfig3, true) + val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) + // send some messages to each broker + Thread.sleep(200) + val sentMessages3 = sendMessages(nMessages, "batch3", DefaultCompressionCodec) + Thread.sleep(200) + val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1) + val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2) + val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum) + assertEquals(sentMessages3, receivedMessages3) + + zkConsumerConnector1.shutdown + zkConsumerConnector2.shutdown + zkConsumerConnector3.shutdown + info("all consumer connectors stopped") + requestHandlerLogger.setLevel(Level.ERROR) + } + + def testCompressionSetConsumption() { + val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) + requestHandlerLogger.setLevel(Level.FATAL) + + var actualMessages: List[Message] = Nil + + // shutdown one server + servers.last.shutdown + Thread.sleep(500) + + // send some messages to each broker + val sentMessages = sendMessages(configs.head, 200, "batch1", DefaultCompressionCodec) + // test consumer timeout logic + val consumerConfig0 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer0)) { + override val consumerTimeoutMs = 5000 + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector0 = new SimpleConsumerConnector(consumerConfig0, true) + val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> 1)) + getMessages(100, topicMessageStreams0) + zkConsumerConnector0.shutdown + // at this point, only some part of the message set was consumed. So consumed offset should still be 0 + // also fetched offset should be 0 + val consumerConfig1 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer1)) { + override val consumerTimeoutMs = 5000 + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector1 = new SimpleConsumerConnector(consumerConfig1, true) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1)) + val receivedMessages = getMessages(400, topicMessageStreams1) + val sortedReceivedMessages = receivedMessages.sortWith((s,t) => s.checksum < t.checksum) + val sortedSentMessages = sentMessages.sortWith((s,t) => s.checksum < t.checksum) + assertEquals(sortedSentMessages, sortedReceivedMessages) + zkConsumerConnector1.shutdown + + requestHandlerLogger.setLevel(Level.ERROR) + } + + def testConsumerDecoder() { + val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) + requestHandlerLogger.setLevel(Level.FATAL) + + val sentMessages = sendMessages(nMessages, "batch1", NoCompressionCodec). + map(m => Utils.toString(m.payload, "UTF-8")). + sortWith((s, t) => s.compare(t) == -1) + val consumerConfig = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer1)) { + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + + val zkConsumerConnector = + new SimpleConsumerConnector(consumerConfig, true) + val topicMessageStreams = + zkConsumerConnector.createMessageStreams( + Predef.Map(topic -> numNodes*numParts/2), new StringDecoder) + + var receivedMessages: List[String] = Nil + for ((topic, messageStreams) <- topicMessageStreams) { + for (messageStream <- messageStreams) { + val iterator = messageStream.iterator + for (i <- 0 until nMessages * 2) { + assertTrue(iterator.hasNext()) + val message = iterator.next().message + receivedMessages ::= message + debug("received message: " + message) + } + } + } + receivedMessages = receivedMessages.sortWith((s, t) => s.compare(t) == -1) + assertEquals(sentMessages, receivedMessages) + + zkConsumerConnector.shutdown() + requestHandlerLogger.setLevel(Level.ERROR) + } + + def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compression: CompressionCodec): List[Message]= { + var messages: List[Message] = Nil + val producer = TestUtils.createProducer("localhost", conf.port) + for (partition <- 0 until numParts) { + val ms = 0.until(messagesPerNode).map(x => + new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray + val mSet = new ByteBufferMessageSet(compressionCodec = compression, messages = ms: _*) + for (message <- ms) + messages ::= message + producer.send(topic, partition, mSet) + } + producer.close() + messages + } + + def sendMessages(messagesPerNode: Int, header: String, compression: CompressionCodec = NoCompressionCodec): List[Message]= { + var messages: List[Message] = Nil + for(conf <- configs) { + messages ++= sendMessages(conf, messagesPerNode, header, compression) + } + messages.sortWith((s,t) => s.checksum < t.checksum) + } + + def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaStream[Message]]]): List[Message]= { + var messages: List[Message] = Nil + for ((topic, messageStreams) <- topicMessageStreams) { + for (messageStream <- messageStreams) { + val iterator = messageStream.iterator + for (i <- 0 until nMessagesPerThread) { + assertTrue(iterator.hasNext) + val message = iterator.next.message + messages ::= message + debug("received message: " + Utils.toString(message.payload, "UTF-8")) + } + } + } + messages.sortWith((s,t) => s.checksum < t.checksum) + } +} Index: core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala =================================================================== --- core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala (revision 1371523) +++ core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala (working copy) @@ -1,124 +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.javaapi.consumer - -import junit.framework.Assert._ -import kafka.zk.ZooKeeperTestHarness -import kafka.integration.KafkaServerTestHarness -import kafka.server.KafkaConfig -import kafka.utils.{Utils, Logging} -import kafka.utils.{TestZKUtils, TestUtils} -import org.scalatest.junit.JUnit3Suite -import scala.collection.JavaConversions._ -import kafka.javaapi.message.ByteBufferMessageSet -import org.apache.log4j.{Level, Logger} -import kafka.message.{NoCompressionCodec, CompressionCodec, Message} -import kafka.consumer.{KafkaStream, ConsumerConfig} - - -class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { - - val zookeeperConnect = TestZKUtils.zookeeperConnect - val zkConnect = zookeeperConnect - val numNodes = 2 - val numParts = 2 - val topic = "topic1" - val configs = - for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) { - override val enableZookeeper = true - override val numPartitions = numParts - override val zkConnect = zookeeperConnect - } - val group = "group1" - val consumer1 = "consumer1" - val nMessages = 2 - - def testBasic() { - val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) - requestHandlerLogger.setLevel(Level.FATAL) - var actualMessages: List[Message] = Nil - - // send some messages to each broker - val sentMessages1 = sendMessages(nMessages, "batch1") - // create a consumer - val consumerConfig1 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(toJavaMap(Predef.Map(topic -> numNodes*numParts/2))) - val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) - assertEquals(sentMessages1, receivedMessages1) - - zkConsumerConnector1.shutdown - info("all consumer connectors stopped") - requestHandlerLogger.setLevel(Level.ERROR) - } - - def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compressed: CompressionCodec): List[Message]= { - var messages: List[Message] = Nil - val producer = kafka.javaapi.Implicits.toJavaSyncProducer(TestUtils.createProducer("localhost", conf.port)) - for (partition <- 0 until numParts) { - val ms = 0.until(messagesPerNode).map(x => - new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray - val mSet = new ByteBufferMessageSet(compressionCodec = compressed, messages = getMessageList(ms: _*)) - for (message <- ms) - messages ::= message - producer.send(topic, partition, mSet) - } - producer.close() - messages - } - - def sendMessages(messagesPerNode: Int, header: String, compressed: CompressionCodec = NoCompressionCodec): List[Message]= { - var messages: List[Message] = Nil - for(conf <- configs) { - messages ++= sendMessages(conf, messagesPerNode, header, compressed) - } - messages.sortWith((s,t) => s.checksum < t.checksum) - } - - def getMessages(nMessagesPerThread: Int, jTopicMessageStreams: java.util.Map[String, java.util.List[KafkaStream[Message]]]) - : List[Message]= { - var messages: List[Message] = Nil - val topicMessageStreams = asMap(jTopicMessageStreams) - for ((topic, messageStreams) <- topicMessageStreams) { - for (messageStream <- messageStreams) { - val iterator = messageStream.iterator - for (i <- 0 until nMessagesPerThread) { - assertTrue(iterator.hasNext) - val message = iterator.next.message - messages ::= message - debug("received message: " + Utils.toString(message.payload, "UTF-8")) - } - } - } - messages.sortWith((s,t) => s.checksum < t.checksum) - } - - private def getMessageList(messages: Message*): java.util.List[Message] = { - val messageList = new java.util.ArrayList[Message]() - messages.foreach(m => messageList.add(m)) - messageList - } - - private def toJavaMap(scalaMap: Map[String, Int]): java.util.Map[String, java.lang.Integer] = { - val javaMap = new java.util.HashMap[String, java.lang.Integer]() - scalaMap.foreach(m => javaMap.put(m._1, m._2.asInstanceOf[java.lang.Integer])) - javaMap - } -} Index: core/src/test/scala/unit/kafka/javaapi/consumer/ConsumerTest.scala =================================================================== --- core/src/test/scala/unit/kafka/javaapi/consumer/ConsumerTest.scala (revision 0) +++ core/src/test/scala/unit/kafka/javaapi/consumer/ConsumerTest.scala (revision 0) @@ -0,0 +1,127 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.javaapi.consumer + +import junit.framework.Assert._ +import kafka.zk.ZooKeeperTestHarness +import kafka.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import kafka.utils.{Utils, Logging} +import kafka.utils.{TestZKUtils, TestUtils} +import org.scalatest.junit.JUnit3Suite +import scala.collection.JavaConversions._ +import kafka.javaapi.message.ByteBufferMessageSet +import org.apache.log4j.{Level, Logger} +import kafka.message.{NoCompressionCodec, CompressionCodec, Message} +import kafka.consumer.{KafkaStream, ConsumerConfig} + + +class ConsumerTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { + + val zookeeperConnect = TestZKUtils.zookeeperConnect + val zkConnect = zookeeperConnect + val numNodes = 2 + val numParts = 2 + val topic = "topic1" + val configs = + for(props <- TestUtils.createBrokerConfigs(numNodes)) + yield new KafkaConfig(props) { + override val enableZookeeper = true + override val numPartitions = numParts + override val zkConnect = zookeeperConnect + } + + val group = "group1" + val consumer1 = "consumer1" + val nMessages = 2 + + def testBasic() { + val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandlers]) + requestHandlerLogger.setLevel(Level.FATAL) + var actualMessages: List[Message] = Nil + + // send some messages to each broker + val sentMessages1 = sendMessages(nMessages, "batch1") + // create a consumer + val consumerConfig1 = new ConsumerConfig( + TestUtils.createConsumerProperties(zkConnect, group, consumer1)) { + override val initBrokerList = "{ \"1\" : \"localhost," + configs.head.port.toString + "\"}" + } + val zkConsumerConnector1 = new SimpleConsumerConnector(consumerConfig1, true) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(toJavaMap(Predef.Map(topic -> numNodes*numParts/2))) + val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) + assertEquals(sentMessages1, receivedMessages1) + + zkConsumerConnector1.shutdown + info("all consumer connectors stopped") + requestHandlerLogger.setLevel(Level.ERROR) + } + + def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compressed: CompressionCodec): List[Message]= { + var messages: List[Message] = Nil + val producer = kafka.javaapi.Implicits.toJavaSyncProducer(TestUtils.createProducer("localhost", conf.port)) + for (partition <- 0 until numParts) { + val ms = 0.until(messagesPerNode).map(x => + new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray + val mSet = new ByteBufferMessageSet(compressionCodec = compressed, messages = getMessageList(ms: _*)) + for (message <- ms) + messages ::= message + producer.send(topic, partition, mSet) + } + producer.close() + messages + } + + def sendMessages(messagesPerNode: Int, header: String, compressed: CompressionCodec = NoCompressionCodec): List[Message]= { + var messages: List[Message] = Nil + for(conf <- configs) { + messages ++= sendMessages(conf, messagesPerNode, header, compressed) + } + messages.sortWith((s,t) => s.checksum < t.checksum) + } + + def getMessages(nMessagesPerThread: Int, jTopicMessageStreams: java.util.Map[String, java.util.List[KafkaStream[Message]]]) + : List[Message]= { + var messages: List[Message] = Nil + val topicMessageStreams = asMap(jTopicMessageStreams) + for ((topic, messageStreams) <- topicMessageStreams) { + for (messageStream <- messageStreams) { + val iterator = messageStream.iterator + for (i <- 0 until nMessagesPerThread) { + assertTrue(iterator.hasNext) + val message = iterator.next.message + messages ::= message + debug("received message: " + Utils.toString(message.payload, "UTF-8")) + } + } + } + messages.sortWith((s,t) => s.checksum < t.checksum) + } + + private def getMessageList(messages: Message*): java.util.List[Message] = { + val messageList = new java.util.ArrayList[Message]() + messages.foreach(m => messageList.add(m)) + messageList + } + + private def toJavaMap(scalaMap: Map[String, Int]): java.util.Map[String, java.lang.Integer] = { + val javaMap = new java.util.HashMap[String, java.lang.Integer]() + scalaMap.foreach(m => javaMap.put(m._1, m._2.asInstanceOf[java.lang.Integer])) + javaMap + } +} Index: core/src/test/scala/other/kafka/TestZKElection.scala =================================================================== --- core/src/test/scala/other/kafka/TestZKElection.scala (revision 0) +++ core/src/test/scala/other/kafka/TestZKElection.scala (revision 0) @@ -0,0 +1,102 @@ +/** + * 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 + +import org.I0Itec.zkclient.ZkClient +import utils.{ZkUtils, TestUtils, ZkElection} + + +object TestZKElection { + + val zkConnect = "127.0.0.1:2181" + + def asLeader(name: String) = { + println(name + ": am the leader!") + } + + def main(args: Array[String]) { + if(args.length < 2) { + println("USAGE: " + TestZKElection.getClass.getName + " num_electors zk_path") + System.exit(1) + } + + val size = args(0).toInt + val path = args(1) + + val electorThreads : Array[ZkElection] = new Array[ZkElection](size) + + var curLeader = -1 + + val zkClient = new ZkClient(zkConnect, 1000, 1000) + + // Start all the electors, let them elect + for(i <- 0 until size) { + val electThread = new ZkElection(new ZkClient(zkConnect, 1000, 1000), path, asLeader, i.toString) + //electorThreads(i) = electThread + electorThreads(i) = electThread + } + + // Check who is the leader now + for(i <- 0 until electorThreads.size) { + if (electorThreads(i).amILeader) { + + curLeader = i + } + else { + println(i + ": am not the leader, I think the leader is " + (if ( electorThreads(i).leaderId == "" ) + "... sorry I do not know since I am a dead man" + else + electorThreads(i).leaderId + )) + } + } + + // Do this 10 times + for (t <- 0 until 10) { + + // Shoot down current leader + println(curLeader + ": ah! I'm dead...") + electorThreads(curLeader).resign + + TestUtils.waitUntilTrue(() => ZkUtils.readDataMaybeNull(zkClient, path) != null, 1500) + + // Check who is the leader now + var hasLeader = false + for(i <- 0 until electorThreads.size) { + if (electorThreads(i).amILeader) { + + curLeader = i + assert(!hasLeader, "Two leaders at the same time is not acceptable") + hasLeader = true + } + else { + println(i + ": am not the leader, I think the leader is " + (if ( electorThreads(i).leaderId == "" ) + "... sorry I do not know since I am a dead man" + else + electorThreads(i).leaderId + )) + } + } + + if (!hasLeader) { + println("No Leader elected: probably everyone is dead...") + return + } + } + } +} \ No newline at end of file Index: core/src/test/resources/log4j.properties =================================================================== --- core/src/test/resources/log4j.properties (revision 1371523) +++ core/src/test/resources/log4j.properties (working copy) @@ -12,13 +12,13 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -log4j.rootLogger=WARN, stdout +log4j.rootLogger=INFO, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=WARN +log4j.logger.kafka=INFO # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN Index: core/src/main/scala/kafka/tools/DumpLogSegments.scala =================================================================== --- core/src/main/scala/kafka/tools/DumpLogSegments.scala (revision 1371523) +++ core/src/main/scala/kafka/tools/DumpLogSegments.scala (working copy) @@ -45,7 +45,7 @@ println("payload:\t" + Utils.toString(messageAndOffset.message.payload, "UTF-8")) offset = messageAndOffset.offset } - println("tail of the log is at offset: " + (startOffset + offset)) + println("tail of the log is at offset: " + (startOffset + offset)) } } } Index: core/src/main/scala/kafka/tools/ExportZkOffsets.scala =================================================================== --- core/src/main/scala/kafka/tools/ExportZkOffsets.scala (revision 1371523) +++ core/src/main/scala/kafka/tools/ExportZkOffsets.scala (working copy) @@ -84,7 +84,7 @@ var consumerGroups: Seq[String] = null if (groups.size == 0) { - consumerGroups = ZkUtils.getChildren(zkClient, ZkUtils.ConsumersPath).toList + consumerGroups = ZkUtils.getChildren(zkClient, ZkUtils.ConsumerGroupsPath).toList } else { import scala.collection.JavaConversions._ Index: core/src/main/scala/kafka/tools/MirrorMaker.scala =================================================================== --- core/src/main/scala/kafka/tools/MirrorMaker.scala (revision 1371523) +++ core/src/main/scala/kafka/tools/MirrorMaker.scala (working copy) @@ -99,7 +99,7 @@ val threads = { val connectors = options.valuesOf(consumerConfigOpt).toList .map(cfg => new ConsumerConfig(Utils.loadProps(cfg.toString))) - .map(new ZookeeperConsumerConnector(_)) + .map(Consumer.create(_)) Runtime.getRuntime.addShutdownHook(new Thread() { override def run() { Index: core/src/main/scala/kafka/utils/Utils.scala =================================================================== --- core/src/main/scala/kafka/utils/Utils.scala (revision 1371523) +++ core/src/main/scala/kafka/utils/Utils.scala (working copy) @@ -109,6 +109,16 @@ } bytes } + + def writeBoolean(buffer: ByteBuffer, boolean: Boolean): Unit = { + if(boolean) buffer.put(1.asInstanceOf[Byte]) + else buffer.put(0.asInstanceOf[Byte]) + } + + def readBoolean(buffer: ByteBuffer): Boolean = { + if (buffer.get == 1.asInstanceOf[Byte]) true + else false + } /** * Read size prefixed string where the size is stored as a 2 byte short. @@ -140,6 +150,24 @@ buffer.put(string.getBytes(encoding)) } } + + /** + * Return size of a size prefixed string where the size is stored as a 2 byte short + * @param string The string to write + * @param encoding The encoding in which to write the string + */ + def shortStringLength(string: String, encoding: String = "UTF-8"): Int = { + if(string == null) { + 2 + } else { + val encodedString = string.getBytes(encoding) + if(encodedString.length > Short.MaxValue) { + throw new IllegalArgumentException("String exceeds the maximum size of " + Short.MaxValue + ".") + } else { + 2 + encodedString.length + } + } + } /** * Read a properties file from the given path @@ -669,6 +697,21 @@ } } + def stringMapToJsonString(jsonDataMap: Map[String, String]): String = { + val builder = new StringBuilder + builder.append("{ ") + var numElements = 0 + for ( (key, value) <- jsonDataMap) { + if (numElements > 0) + builder.append(",") + builder.append("\"" + key + "\": ") + builder.append("\"" + value + "\"") + numElements += 1 + } + builder.append(" }") + builder.toString + } + /** * Create a circular (looping) iterator over a collection. * @param coll An iterable over the underlying collection. Index: core/src/main/scala/kafka/utils/ZkUtils.scala =================================================================== --- core/src/main/scala/kafka/utils/ZkUtils.scala (revision 1371523) +++ core/src/main/scala/kafka/utils/ZkUtils.scala (working copy) @@ -21,15 +21,34 @@ import org.I0Itec.zkclient.serialize.ZkSerializer import kafka.cluster.{Broker, Cluster} import scala.collection._ -import java.util.Properties +import java.util.{Collections, Properties} +import mutable.HashMap import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, ZkMarshallingError} -import kafka.consumer.TopicCount +import kafka.server.KafkaServer +import kafka.consumer.{WildcardTopicCount, TopicCount} +import java.util object ZkUtils extends Logging { - val ConsumersPath = "/consumers" + val ConsumerGroupsPath = "/consumers/groups" + val ConsumerCoordinatorPath = "/consumers/coordinator" val BrokerIdsPath = "/brokers/ids" val BrokerTopicsPath = "/brokers/topics" + def getZKChildrenValues(zkClient: ZkClient, path : String) : Seq[Tuple2[String,String]] = { + import scala.collection.JavaConversions + + try { + val children = zkClient.getChildren(path) + Collections.sort(children) + val childrenAsSeq : Seq[java.lang.String] = JavaConversions.asBuffer(children) + childrenAsSeq.map(partition => + (partition, zkClient.readData(path + "/" + partition).asInstanceOf[String])) + } catch { + case e: ZkNoNodeException => List.empty[Tuple2[String,String]] + case e2 => throw e2 + } + } + /** * make sure a persistent path exists in ZK. Create the path if not exist. */ @@ -95,6 +114,25 @@ } /** + * Create an persistent node with the given path and data. Create parents if necessary. + */ + def createPersistentPath(client: ZkClient, path: String, data: String = ""): Unit = { + try { + client.createPersistent(path, data) + } + catch { + case e: ZkNoNodeException => { + createParentPath(client, path) + client.createPersistent(path, data) + } + } + } + + def createSequentialPersistentPath(client: ZkClient, path: String, data: String = ""): String = { + client.createPersistentSequential(path, data) + } + + /** * Update the value of a persistent node with the given path and data. * create parrent directory if necessary. Never throw NodeExistException. */ @@ -134,7 +172,7 @@ } } - def deletePath(client: ZkClient, path: String) { + def deletePath(client: ZkClient, path: String): Boolean = { try { client.delete(path) } @@ -142,6 +180,7 @@ case e: ZkNoNodeException => // this can happen during a connection loss event, return normally info(path + " deleted during connection loss; this is ok") + false case e2 => throw e2 } } @@ -207,6 +246,46 @@ cluster } + /** + * Get the interested topics of the group + * + * @param zkClient: Zookeeper client + * @param group: Name of the group + */ + def getTopicsForGroup(zkClient: ZkClient, group: String) : Set[String] = { + val consumersInGroup = getConsumersInGroup(zkClient, group) + var totalTopics: Set[String] = Set.empty[String] + for (consumer <- consumersInGroup) { + val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient) + val consumerTopics = topicCount.getConsumerThreadIdsPerTopic.map(_._1) + totalTopics ++= consumerTopics + } + + totalTopics + } + + /** + * Get the interested topics of the group. + * If one consumer in the group use a wildcard, then the whole group is marked as wildcard + * + * @param zkClient: Zookeeper client + * @param group: Name of the group + */ + + def getTopicsForGroupMaybeWildcard(zkClient: ZkClient, group: String) : (Set[String], Boolean) = { + var isWildcard : Boolean = false + val consumersInGroup = getConsumersInGroup(zkClient, group) + var totalTopics: Set[String] = Set.empty[String] + for (consumer <- consumersInGroup) { + val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient) + if (!isWildcard && topicCount.isInstanceOf[WildcardTopicCount]) isWildcard = true + val consumerTopics = topicCount.getConsumerThreadIdsPerTopic.map(_._1) + totalTopics ++= consumerTopics + } + + (totalTopics, isWildcard) + } + def getPartitionsForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, List[String]] = { val ret = new mutable.HashMap[String, List[String]]() for (topic <- topics) { @@ -286,9 +365,11 @@ } class ZKGroupDirs(val group: String) { - def consumerDir = ZkUtils.ConsumersPath + def consumerDir = ZkUtils.ConsumerGroupsPath def consumerGroupDir = consumerDir + "/" + group def consumerRegistryDir = consumerGroupDir + "/ids" + def consumerChannelDir = consumerGroupDir + "/channel" + def consumerOwnersDir = consumerGroupDir + "/owners" } class ZKGroupTopicDirs(group: String, topic: String) extends ZKGroupDirs(group) { Index: core/src/main/scala/kafka/utils/ZkElection.scala =================================================================== --- core/src/main/scala/kafka/utils/ZkElection.scala (revision 0) +++ core/src/main/scala/kafka/utils/ZkElection.scala (revision 0) @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.utils + +import kafka.utils.ZkUtils._ +import org.I0Itec.zkclient.exception.ZkNodeExistsException +import org.I0Itec.zkclient.{IZkDataListener, ZkClient} + + +class ZkElection(zkClient: ZkClient, electionPath: String, actAsLeader: String => Unit, serverId: String) extends Logging { + + var leaderId = "" + + // create the election path in ZK, if one does not exist + val index = electionPath.lastIndexOf("/") + if (index > 0) + makeSurePersistentPathExists(zkClient, electionPath.substring(0, index)) + + val leaderChangeListener = new LeaderChangeListener + + zkClient.subscribeDataChanges(electionPath, leaderChangeListener) + + // Try to become a leader + if (elect) actAsLeader(serverId) + + // The election module does not handle session expiration, instead it + // presumes the caller will handle it by probably try to re-elect again + + def amILeader : Boolean = leaderId == serverId + + def tryElectAndAct: Unit = { + // If I am already the leader, no need to elect and act + if (amILeader) true + else if (elect) actAsLeader(serverId) + } + + def elect: Boolean = { + + try { + createEphemeralPathExpectConflict(zkClient, electionPath, serverId) + debug(serverId + " successfully elected as leader") + + leaderId = serverId + } catch { + case e: ZkNodeExistsException => + // If someone else has written the path, then + debug("Someone else has elected as leader other than " + serverId) + + val data: String = zkClient.readData(electionPath, true) + if (data != null) leaderId = data + + case e2 => throw e2 + } + + amILeader + } + + def resign = { + + // Only the leader can resign + val curLeaderId = ZkUtils.readData(zkClient, electionPath) + + assert(curLeaderId == serverId, "Cannot resign a non-leader for " + serverId) + + debug(serverId + " decide to resign from leadership") + + zkClient.delete(electionPath) + + leaderId = "" + } + + def close = { + // First unsubscribe the listener + zkClient.unsubscribeDataChanges(electionPath, leaderChangeListener) + + // If I am the leader, resign + if (leaderId == serverId) resign + + // The caller of the election module should be + // in charge of closing the zkClient + } + + // We do not have session expiration listen in the ZkElection, + // but assuming the caller who uses this module will have its + // own session expiration listener and handler + class LeaderChangeListener extends IZkDataListener with Logging { + + /** + * Called when the leader information stored in zookeeper has changed. Record the new leader in memory + * + * @throws Exception + * On any error. + */ + @throws(classOf[Exception]) + def handleDataChange(dataPath: String, data: Object) { + debug("%s leader change listener fired for path %s to handle data changed: record the new leader %s in memory" + .format(serverId, dataPath, data)) + + val oldLeader = leaderId + leaderId = data.toString + + if (oldLeader != leaderId && leaderId == serverId) { + debug("Now " + serverId + " is the new leader") + actAsLeader(serverId) + } + + } + + /** + * Called when the leader information stored in zookeeper has been delete. Try to elect as the leader + * + * @throws Exception + * On any error. + */ + @throws(classOf[Exception]) + def handleDataDeleted(dataPath: String) { + debug("%s leader change listener fired for path %s to handle data deleted: trying to elect as a leader" + .format(serverId, dataPath)) + + // Try to elect and act as a leader, if I am possibly + // already the leader then I need to do nothing + tryElectAndAct + } + } +} Index: core/src/main/scala/kafka/utils/KafkaScheduler.scala =================================================================== --- core/src/main/scala/kafka/utils/KafkaScheduler.scala (revision 1371523) +++ core/src/main/scala/kafka/utils/KafkaScheduler.scala (working copy) @@ -36,9 +36,15 @@ executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false) executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false) - def scheduleWithRate(fun: () => Unit, delayMs: Long, periodMs: Long) = + def scheduleParameterizedWithRate(command: Runnable, delayMs: Long, periodMs: Long): ScheduledFuture[_] = + executor.scheduleAtFixedRate(command, delayMs, periodMs, TimeUnit.MILLISECONDS) + + def scheduleWithRate(fun: () => Unit, delayMs: Long, periodMs: Long): ScheduledFuture[_] = executor.scheduleAtFixedRate(Utils.loggedRunnable(fun), delayMs, periodMs, TimeUnit.MILLISECONDS) + def removeTask(task: ScheduledFuture[_]) = + task.cancel(false) + def shutdownNow() { executor.shutdownNow() info("force shutdown scheduler " + baseThreadName) Index: core/src/main/scala/kafka/utils/Logging.scala =================================================================== --- core/src/main/scala/kafka/utils/Logging.scala (revision 1371523) +++ core/src/main/scala/kafka/utils/Logging.scala (working copy) @@ -74,8 +74,13 @@ } def warn(msg: => String, e: => Throwable) = { logger.warn(msgWithLogIdent(msg),e) - } + } + def swallowWarn(action: => Unit) { + Utils.swallow(logger.warn, action) + } + def swallow(action: => Unit) = swallowWarn(action) + def error(msg: => String): Unit = { logger.error(msgWithLogIdent(msg)) } Index: core/src/main/scala/kafka/cluster/Cluster.scala =================================================================== --- core/src/main/scala/kafka/cluster/Cluster.scala (revision 1371523) +++ core/src/main/scala/kafka/cluster/Cluster.scala (working copy) @@ -17,7 +17,9 @@ package kafka.cluster -import scala.collection._ +import collection._ +import kafka.utils.Utils._ +import java.nio.ByteBuffer /** * The set of active brokers in the cluster @@ -32,6 +34,8 @@ brokers.put(broker.id, broker) } + def getBrokerIds = brokers.keySet + def getBroker(id: Int): Option[Broker] = brokers.get(id) def add(broker: Broker) = brokers.put(broker.id, broker) @@ -39,7 +43,13 @@ def remove(id: Int) = brokers.remove(id) def size = brokers.size - + override def toString(): String = - "Cluster(" + brokers.values.mkString(", ") + ")" + "Cluster(" + brokers.values.mkString(", ") + ")" + + def writeTo(buffer: ByteBuffer) { + for (broker <- brokers.values) broker.writeTo(buffer) + } + + def sizeInBytes: Int = brokers.values.map(broker => broker.sizeInBytes).sum } Index: core/src/main/scala/kafka/cluster/Broker.scala =================================================================== --- core/src/main/scala/kafka/cluster/Broker.scala (revision 1371523) +++ core/src/main/scala/kafka/cluster/Broker.scala (working copy) @@ -18,6 +18,8 @@ package kafka.cluster import kafka.utils._ +import java.nio.ByteBuffer +import kafka.utils.Utils._ /** * A Kafka broker @@ -27,6 +29,14 @@ val brokerInfo = brokerInfoString.split(":") new Broker(id, brokerInfo(0), brokerInfo(1), brokerInfo(2).toInt) } + + def readFrom(buffer: ByteBuffer): Broker = { + val id = buffer.getInt + val host = readShortString(buffer, "UTF-8") + val port = buffer.getInt + + new Broker(id, "unknown", host, port) + } } private[kafka] class Broker(val id: Int, val creatorId: String, val host: String, val port: Int) { @@ -34,7 +44,9 @@ override def toString(): String = new String("id:" + id + ",creatorId:" + creatorId + ",host:" + host + ",port:" + port) def getZKString(): String = new String(creatorId + ":" + host + ":" + port) - + + def getByteString(): String = new String(id + ":" + host + ":" + port) + override def equals(obj: Any): Boolean = { obj match { case null => false @@ -44,5 +56,15 @@ } override def hashCode(): Int = Utils.hashcode(id, host, port) - + + def writeTo(buffer: ByteBuffer) { + buffer.putInt(id) + writeShortString(buffer, host, "UTF-8") + buffer.putInt(port) + } + + def sizeInBytes: Int = { + shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ + } + } Index: core/src/main/scala/kafka/server/RegistryReplySend.scala =================================================================== --- core/src/main/scala/kafka/server/RegistryReplySend.scala (revision 0) +++ core/src/main/scala/kafka/server/RegistryReplySend.scala (revision 0) @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.nio._ +import java.nio.channels._ +import kafka.network._ +import kafka.utils._ +import kafka.api.RegisterConsumerRequest + +/** + * A zero-copy message response that writes the bytes needed directly from the file + * wholly in kernel space + */ +@nonthreadsafe +class RegistryReplySend(val registryRequest: RegisterConsumerRequest, val errorCode: Int) extends Send { + + private val header = ByteBuffer.allocate(6) + + header.putInt(2 /* error code */) + header.putShort(errorCode.asInstanceOf[Short]) + header.rewind() + + var complete: Boolean = false + + def writeTo(channel: GatheringByteChannel): Int = { + expectIncomplete() + val written = channel.write(header) + // if we are done, mark it off + if(!header.hasRemaining) + complete = true + written + } + +} Index: core/src/main/scala/kafka/server/ConsumerCoordinator.scala =================================================================== --- core/src/main/scala/kafka/server/ConsumerCoordinator.scala (revision 0) +++ core/src/main/scala/kafka/server/ConsumerCoordinator.scala (revision 0) @@ -0,0 +1,720 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server + + +import kafka.api._ +import kafka.utils._ +import kafka.common._ +import kafka.network.SocketServer +import kafka.consumer.{TopicCount, WildcardTopicCount} +import scala.collection.JavaConversions._ +import collection.mutable +import collection.mutable.{Set, Map} +import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} +import org.apache.zookeeper.Watcher.Event.KeeperState +import java.util.concurrent.atomic.{AtomicReference, AtomicLong, AtomicBoolean} +import java.nio.channels.{SelectionKey, CancelledKeyException} +import java.util.concurrent.{BlockingQueue, LinkedBlockingQueue} +import java.util.concurrent.locks.{Condition, ReentrantLock} + +/** + * Once a broker has been elected as the coordinator, its coordinator's startup procedure will be triggered, which will do the following: + * + * 1. Register the listeners for brokers and their partitions + * + * 2. Register session expiration listener + * + * 3. Initialize the in memory metadata such as lists of interested groups per topic + * + * 4. Initialize and start the rebalance request handlers + */ +class ConsumerCoordinator(val zkClient: ZkClient, val brokerId: Int, val config: KafkaConfig) extends Logging { + private val consumerRegistryLock = new Object + + private val scheduler = new KafkaScheduler(1, "Kafka-ping-request-", false) + + private val committer = new KafkaScheduler(1, "Kafka-offset-commit-", false) + + private val groupRegistries = Map[String, GroupRegistry]() + + private val consumerGroupsPerTopic = Map[String, Set[String]]() + + private val groupsWithWildcardTopics = Set[String]() + + private var rebalanceRequestQueues: Array[BlockingQueue[String]] = null + + private var rebalanceRequestHandlers: Array[RebalanceRequestHandler] = null + + private val numRebalanceRequests: AtomicLong = new AtomicLong(0) + + var socketServer: SocketServer = null + + var heartbeatRequestPurgatory: HeartbeatRequestPurgatory = null + + // TODO: these two purgatory can be combined to one with the typing in 0.8 + var stopFetcherRequestPurgatory : StopFetcherRequestPurgatory = null + + var startFetcherRequestPurgatory : StartFetcherRequestPurgatory = null + + /* For test use only */ + val isBlocked: AtomicBoolean = new AtomicBoolean(false) + + /** + * The elector must only be initialized after all the other variables, + * since its callback becomeCoordinator will update these variables + */ + private val elector = new ZkElection(zkClient, ZkUtils.ConsumerCoordinatorPath, becomeCoordinator, brokerId.toString) + + def amICurrentCoordinator = elector.amILeader + + def currentCoordinatorId = elector.leaderId.toInt + + def setSocketServer(server: SocketServer) { + socketServer = server + } + + /* The call back function passed to the Zookeeper based elector, will be called once elected as the coordinator */ + def becomeCoordinator(id: String) { + // Check it is really me who become the coordinator + if (id.toInt != brokerId) + throw new IllegalStateException("The coordinator startup logic is called for a different server") + + startup + } + + def startup = { + info("Broker %d started serving as the consumer co-ordinator".format(brokerId)) + + // Register session expiration listener + registerSessionExpirationListener + + // Register the listeners for brokers and their partitions + registerTopicChangeListener + registerTopicPartitionChangeListener + + heartbeatRequestPurgatory = new HeartbeatRequestPurgatory(this) + stopFetcherRequestPurgatory = new StopFetcherRequestPurgatory(this) + startFetcherRequestPurgatory = new StartFetcherRequestPurgatory(this) + + // Add the topic to the consumer groups that have subscribed to it + val topics = ZkUtils.getChildren(zkClient, ZkUtils.BrokerTopicsPath) + for (topic <- topics) consumerGroupsPerTopic += (topic -> Set.empty[String]) + + // Start processing rebalance requests + rebalanceRequestQueues = new Array[BlockingQueue[String]](config.numRebalanceThreads) + rebalanceRequestHandlers = new Array[RebalanceRequestHandler](config.numRebalanceThreads) + for(i <- 0 until config.numRebalanceThreads) { + rebalanceRequestQueues(i) = new LinkedBlockingQueue[String](config.rebalanceQueueSize) + rebalanceRequestHandlers(i) = new RebalanceRequestHandler(zkClient, this, rebalanceRequestQueues(i)) + rebalanceRequestHandlers(i).start + } + + committer.scheduleWithRate(autoCommit, config.autoCommitIntervalMs, config.autoCommitIntervalMs) + } + + def shutdown() { + // Close the elector to trigger re-election of the consumer co-ordinator + elector.close + + // Then unsubscribe all the watchers before close the zkClient + zkClient.unsubscribeAll() + + scheduler.shutdown() + + // If the requestHandler is once created, shut it down + if (rebalanceRequestHandlers != null) { + for(i <- 0 until config.numRebalanceThreads) { + if (rebalanceRequestHandlers(i) != null) { + rebalanceRequestHandlers(i).shutdown + rebalanceRequestHandlers(i) = null + } + + } + } + rebalanceRequestHandlers = null + + // Shutdown committer and commit the last offset + committer.shutdown() + autoCommit + + // Then reset the state by clearing the cache + reset + + // Catch and ignore interruption exception while closing zkClient + zkClient.close() + + info("Coordinator for broker %s shutdown completed".format(brokerId)) + } + + def reset = { + groupRegistries.clear + consumerGroupsPerTopic.clear + groupsWithWildcardTopics.clear + } + + /* Add the rebalance request of specified group to one of the rebalancer's queue */ + def addRebalanceRequest(group: String) { + val curr = groupRegistries(group).currRebalancer + if (curr == GroupRegistry.None) { + val index = numRebalanceRequests.getAndIncrement % config.numRebalanceThreads + groupRegistries(group).currRebalancer = index.asInstanceOf[Int] + rebalanceRequestQueues(index.asInstanceOf[Int]).put(group) + + debug("Add the rebalance request for group %s to the #.%d rebalancer".format(group, index)) + } + else { + rebalanceRequestQueues(curr).put(group) + + debug("Add the rebalance request for group %s to the #.%d rebalancer".format(group, curr)) + } + } + + /* commit function triggered periodically by the Zookeeper committer thread */ + def autoCommit() { + for ((group, registry) <- groupRegistries) { + info("auto committing offsets for group " + group) + registry.writeOffsetsToZK(zkClient) + } + } + + def commitOffset(group: String) { + if (!groupRegistries.contains(group)) + throw new IllegalStateException("Group " + group + " is not registered yet while trying to commit its consumed offsets") + + val registry = groupRegistries(group) + registry.writeOffsetsToZK(zkClient) + } + + /* heartbeat function triggered periodically by the scheduler thread for each consumer */ + def heartbeat(consumer: String) { + trace("heartbeating consumer " + consumer) + try { + while(isBlocked.get()) {} + + sendHeartbeatToConsumer(consumer) + } + catch { + case t: Throwable => + // log it and let it go + error("exception during heartbeating consumer " + consumer + ": ", t) + } + } + + def sendHeartbeatToConsumer(consumer: String) { + // Get the processor Id and the selectionkey for sending the heartbeat request + val processorIdKeyAndTimeout = getProcessorIdKeyAndTimeout(consumer) + + // If the previous request has not been responded by the consumer, + // do not send the request + if (!heartbeatRequestPurgatory.noPendingDelayedRequest(consumer)) { + debug("The previous heartbeat to consumer " + consumer + " has not been responded yet, do not send the request") + } + // The consumer could have failed and removed from the group registry + // In this case, do nothing + else if (!processorIdKeyAndTimeout._1) { + debug("Consumer " + consumer + " does not exist in the group registry any more, do not send the request") + } + else { + val request = new HeartbeatRequest(consumer) + + try { + debug("Sending heartbeat request to consumer %s".format(consumer)) + + socketServer.send(processorIdKeyAndTimeout._2, + processorIdKeyAndTimeout._3, request) + + // Set the watcher for the heartbeat request + val delayMs = processorIdKeyAndTimeout._4 * 2/3 + val delayedHeartbeat = new DelayedHeartbeat(List[String](consumer), request, delayMs) + heartbeatRequestPurgatory.watch(delayedHeartbeat) + + trace("Add the expire watcher for the heartbeat request to consumer %s with timeout %d" + .format(consumer, delayMs)) + } + catch { + /** + * The consumer channel closed event can only be detected at the next heartbeat request, + * we decide to not detect channel closure since otherwise it will require + * SocketServer to be aware of the coordinator's existence + */ + case e: CancelledKeyException => { + info("Channel cancelled while sending heartbeat request, mark consumer %s as failed".format(consumer)) + val groupId = consumer.split("_")(0) + handleConsumerFailure(consumer, groupId) + } + } + } + } + + /* Procedure triggered upon some consumer failing to response the heartbeat request in time */ + def handleHeartbeatRequestExpired(consumer: String) { + val groupId = consumer.split("_")(0) + var markConsumerFailed : Boolean = false + + consumerRegistryLock synchronized { + // This consumer could have already been failed thus removed from the group registry + // In this case, do nothing + if (!groupRegistries(groupId).memberRegistries.contains(consumer)) return + + val consumerRegistry = groupRegistries(groupId).memberRegistries(consumer) + + consumerRegistry.numFailedRequests += 1 + + debug("Heartbeat request for consumer %s expires for #.%d time".format(consumer, consumerRegistry.numFailedRequests)) + + if (consumerRegistry.numFailedRequests == config.maxNumRequestFailures) { + info("Consumer %s failed to reply requests with maximum times %d, marked as dead" + .format(consumer, config.maxNumRequestFailures)) + + markConsumerFailed = true + } + } + + if (markConsumerFailed) + handleConsumerFailure(consumer, groupId) + } + + /* Procedure triggered upon some consumer failing to response the stop/start-fetcher request in time */ + def handleStopOrStartFetcherRequestExpired(consumer: String) { + val groupId = consumer.split("_")(0) + + consumerRegistryLock synchronized { + info("Stop/Start fetcher request for consumer %s expires".format(consumer)) + + groupRegistries(groupId).rebalanceFailed = true + + val rebalanceLock = groupRegistries(groupId).rebalanceLock + val rebalanceCondition = groupRegistries(groupId).rebalanceCondition + + rebalanceLock.lock() + rebalanceCondition.signal() + rebalanceLock.unlock() + } + } + + /* procedure triggered upon all stop/start-fetcher responses for specified group has been received */ + def handleReceivedAckForStartOrStopFetcherRequest(group: String) { + consumerRegistryLock synchronized { + debug("All stop/start fetcher request for group %s have been replied".format(group)) + + val rebalanceLock = groupRegistries(group).rebalanceLock + val rebalanceCondition = groupRegistries(group).rebalanceCondition + + rebalanceLock.lock() + rebalanceCondition.signal() + rebalanceLock.unlock() + } + } + + /* procedure triggered when a new group has been detected while registering some consumers */ + def handleNewGroup(group: String) { + consumerRegistryLock synchronized { + // Concurrent other threads could have handled this new group + // In this case, do nothing + if (groupRegistries.contains(group)) return + + info("Handle new group " + group) + + groupRegistries.put(group, new GroupRegistry(group)) + } + + } + + /* procedure trigger when a new consumer has been registered to a known group */ + def handleNewGroupMember(consumerRegistry: ConsumerRegistry, group: String) { + consumerRegistryLock synchronized { + if (groupRegistries(group).memberRegistries.contains(consumerRegistry.consumerId)) + throw new IllegalStateException("New added consumer %s is already registered in group %s" + .format(consumerRegistry.consumerId, group)) + + debug("Handle added member " + consumerRegistry.consumerId + " in group " + group) + + // Add the heartbeat task to the scheduler + val heartbeatCommand = new Runnable() { + def run() = { + try { + heartbeat(consumerRegistry.consumerId) + } + catch { + case t => + // log any error and the stack trace + error("error in heartbeatCommand runnable", t) + } + } + } + val futureTask = scheduler.scheduleParameterizedWithRate(heartbeatCommand, + consumerRegistry.sessionTimeoutMs * 1/3, + consumerRegistry.sessionTimeoutMs * 1/3) + consumerRegistry.registerHeartbeatTask(futureTask) + + // Add the consumer to the group + groupRegistries(group).memberRegistries.put(consumerRegistry.consumerId, consumerRegistry) + + // Read all the topics this group is interested in, and for each topic: + // If the topic already exists in coordinator's memory, update its list by adding this group + // Otherwise add this topic with the new group as its only interested group + val topicCount = consumerRegistry.topicCount + + // If the interested topics are in the wildcard format, add this group to the list of wildcard-interested groups + if (topicCount.isInstanceOf[WildcardTopicCount]) { + topicCount.asInstanceOf[WildcardTopicCount].setTopicsFromZk(zkClient) + groupsWithWildcardTopics.add(group) + } + + val topics = topicCount.getConsumerThreadIdsPerTopic.map(_._1) + for (topic <- topics) { + if (!consumerGroupsPerTopic.contains(topic)) consumerGroupsPerTopic += (topic -> Set(group)) + else consumerGroupsPerTopic(topic).add(group) + } + + // Try to rebalance + if (groupRegistries(group).rebalanceInitiated.compareAndSet(false, true)) + addRebalanceRequest(group) + } + } + + /* procedure triggered upon a consumer failure */ + def handleConsumerFailure(consumer: String, group: String) { + consumerRegistryLock synchronized { + // Concurrent other threads could have handled this failed consumer + // In this case, do nothing + if (!groupRegistries(group).memberRegistries.contains(consumer)) return + + debug("Handle consumer " + consumer + " failure in group " + group) + + // Remove its heartbeat task from the scheduler + scheduler.removeTask(groupRegistries(group).memberRegistries(consumer).heartbeatTask) + + // Close its socket channel + socketServer.close(groupRegistries(group).memberRegistries(consumer).processorId, + groupRegistries(group).memberRegistries(consumer).selectionKey) + + // Remove consumer from its group + groupRegistries(group).memberRegistries.remove(consumer) + + // If there is an ongoing rebalancing, let it fail directly + if (groupRegistries(group).currRebalancer != GroupRegistry.None) { + groupRegistries(group).rebalanceFailed = true + + val rebalanceLock = groupRegistries(group).rebalanceLock + val rebalanceCondition = groupRegistries(group).rebalanceCondition + + rebalanceLock.lock() + rebalanceCondition.signal() + rebalanceLock.unlock() + } + // If the group still has some consumers, try to rebalance + else if (!groupRegistries(group).memberRegistries.isEmpty && groupRegistries(group).rebalanceInitiated.compareAndSet(false, true)) { + addRebalanceRequest(group) + } + } + } + + // Watch for partition changes on the brokers + private def registerTopicPartitionChangeListener() = { + val topics = zkClient.getChildren(ZkUtils.BrokerTopicsPath) + for (topic <- topics) { + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.BrokerTopicsPath + "/" + topic) + zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath + "/" + topic, new TopicPartitionChangeListener) + } + } + + // Watch for topic changes + private def registerTopicChangeListener() = { + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.BrokerTopicsPath) + zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, new TopicChangeListener) + } + + // Watch for session timeout + private def registerSessionExpirationListener() = { + zkClient.subscribeStateChanges(new ConsumerCoordinatorSessionExpireListener) + } + + + class ConsumerCoordinatorSessionExpireListener extends IZkStateListener { + + @throws(classOf[Exception]) + def handleStateChanged(state: KeeperState) { + // do nothing, since zkclient will do reconnect for us. + } + + /** + * Called after the zookeeper session has expired and a new session has been created. Try to elect as the leader + * + * @throws Exception + * On any error. + */ + @throws(classOf[Exception]) + def handleNewSession() { + /** + * When we get a SessionExpired event, we lost all ephemeral nodes and zkclient has reestablished a + * connection for us. + */ + consumerRegistryLock synchronized { + debug(brokerId + " ZK expired; check if I am the coordinator") + + // First reset myself, then try to re-elect + // Note that do not need to unsubscribe all listeners, since they are already lost with the new session + reset + + // The state listener in ZkClient does not re-register itself once fired, + // so we have to re-subscribe this listener manually + registerSessionExpirationListener + + if (elector.elect) startup + } + } + } + + class TopicChangeListener extends IZkChildListener with Logging { + + /** + * Called when a new topic group is registered; + * an existing topic should not be deleted since the group nodes are persistent + * + * @throws Exception + * On any error. + */ + @throws(classOf[Exception]) + def handleChildChange(parentPath : String, curChilds : java.util.List[String]) { + import scala.collection.JavaConversions._ + + consumerRegistryLock synchronized { + debug("%s TopicChange listener fired for path %s with children %s" + .format(brokerId, parentPath, curChilds)) + + // Update in-memory data and subscribe watchers with newly registered groups + val addedTopics = curChilds filterNot (consumerGroupsPerTopic.keySet contains) + + if (addedTopics.nonEmpty) + debug("Topic event: added topics = %s".format(addedTopics)) + + for (topic <- addedTopics) { + // Watch for topic partition changes + zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath + "/" + topic, new TopicPartitionChangeListener) + + // Get groups with static interests that contain this topic + var staticGroups = Set[String]() + + if (!consumerGroupsPerTopic.contains(topic)) consumerGroupsPerTopic += (topic -> Set.empty[String]) + else staticGroups = consumerGroupsPerTopic(topic) + + // Get groups with wildcard interests that match this topic + var wildcardGroups = Set[String]() + + groupsWithWildcardTopics.foreach ( + group => + if (getTopicsForConsumers(readConsumerRegistriesSnapshot(group)) contains topic) wildcardGroups += group + ) + + for (group <- (staticGroups | wildcardGroups)) { + if (groupRegistries(group).rebalanceInitiated.compareAndSet(false, true)) { + addRebalanceRequest(group) + trace("Put the rebalance request for group %s due to added topic %s of interest".format(group, topic)) + } + } + } + + // Since topics are persistent path, they should never be deleted + val deletedTopics = consumerGroupsPerTopic.keySet filterNot (curChilds contains) + + if (deletedTopics.nonEmpty) + throw new IllegalStateException("Topics %s are deleted, which should not happen" + .format(deletedTopics)) + } + } + } + + class TopicPartitionChangeListener extends IZkChildListener with Logging { + + /** + * Called when the topic's partition information stored in zookeeper has changed. + * Try to request a rebalance for each group interested in the changed topic + * by setting the bit in groupsBeingRebalanced + * + * @throws Exception + * On any error. + */ + @throws(classOf[Exception]) + def handleChildChange(parentPath : String, curChilds : java.util.List[String]) { + consumerRegistryLock synchronized { + debug("%s TopicPartitionChange listener fired for path %s with children %s". + format(brokerId, parentPath, curChilds)) + + val topic = parentPath.split("/").last + + // Get groups with static interests that contain this topic + val staticGroups = consumerGroupsPerTopic(topic) + + // Get groups with wildcard interests that match this topic + var wildcardGroups = Set[String]() + + groupsWithWildcardTopics.foreach ( + group => + if (getTopicsForConsumers(readConsumerRegistriesSnapshot(group)) contains topic) wildcardGroups += group + ) + + for (group <- (staticGroups | wildcardGroups)) { + if (groupRegistries(group).rebalanceInitiated.compareAndSet(false, true)) { + addRebalanceRequest(group) + trace("Put the rebalance request for group %s due to topic %s partition change".format(group, topic)) + } + } + } + } + } + + + /* -------------------------- Util Functions -------------------------- */ + + /** + * Get the interested topics of the group + */ + def getTopicsForConsumers(consumerRegistries: collection.Map[String, ConsumerRegistry]) : Set[String] = { + var totalTopics: Set[String] = Set.empty[String] + for ((consumer, registry) <- consumerRegistries) { + val topicCount = registry.topicCount + // If it is a wildcard topicCount, needs to read the current topics from ZK + if (topicCount.isInstanceOf[WildcardTopicCount]) + topicCount.asInstanceOf[WildcardTopicCount].setTopicsFromZk(zkClient) + val consumerTopics = topicCount.getConsumerThreadIdsPerTopic.map(_._1) + totalTopics ++= consumerTopics + } + + totalTopics + } + + /** + * Get the interested consumers list per each topic + */ + def getConsumersPerTopic(consumerRegistries: collection.Map[String, ConsumerRegistry]) : Map[String, List[String]] = { + val consumersPerTopicMap = new mutable.HashMap[String, List[String]] + for ((consumer, registry) <- consumerRegistries) { + val topicCount = registry.topicCount + // If it is a wildcard topicCount, needs to read the current topics from ZK + if (topicCount.isInstanceOf[WildcardTopicCount]) + topicCount.asInstanceOf[WildcardTopicCount].setTopicsFromZk(zkClient) + for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic) { + for (consumerThreadId <- consumerThreadIdSet) + consumersPerTopicMap.get(topic) match { + case Some(curConsumers) => consumersPerTopicMap.put(topic, consumerThreadId :: curConsumers) + case _ => consumersPerTopicMap.put(topic, List(consumerThreadId)) + } + } + } + for ( (topic, consumerList) <- consumersPerTopicMap ) + consumersPerTopicMap.put(topic, consumerList.sortWith((s,t) => s < t)) + consumersPerTopicMap + } + + /** + * Check if the given consumer is already registered + */ + def isConsumerRegistered(consumer: String) : Boolean = { + val group = consumer.split("_")(0) + groupRegistries(group).memberRegistries.contains(consumer) + } + + /** + * Check if the given server is already registered + */ + def isGroupRegistered(group: String) : Boolean = groupRegistries.contains(group) + + /** + * Update the group offset information if possible + */ + def tryUpdateConsumedOffsets(consumer: String, parts: List[PartitionAndOffset]) : Unit = { + var autoOffsetCommit = false + val group = consumer.split("_")(0) + consumerRegistryLock synchronized { + // If the consumer is already deleted from the group, do nothing + if (!groupRegistries(group).memberRegistries.contains(consumer)) + return + + autoOffsetCommit = groupRegistries(group).memberRegistries(consumer).autoOffsetCommit + } + + if (autoOffsetCommit) { + groupRegistries(group).updateOffsets(parts) + } + } + + /** + * Set/reset/query the group's under rebalance bit + */ + def setGroupUnderReblance(group: String) : Boolean = + groupRegistries(group).rebalanceInitiated.compareAndSet(false, true) + + def resetRebalanceInitiated(group: String) : Boolean = + groupRegistries(group).rebalanceInitiated.compareAndSet(true, false) + + def isGroupUnderRebalance(group: String) : Boolean = groupRegistries(group).rebalanceInitiated.get + + /** + * Get the socket server processor id and the key and the session timeout for the consumer + */ + def getProcessorIdKeyAndTimeout(consumer: String) : Tuple4[Boolean, Int, SelectionKey, Int] = { + val group = consumer.split("_")(0) + + consumerRegistryLock synchronized { + if (!groupRegistries(group).memberRegistries.contains(consumer)) + (false, -1, null, -1) + else + (true, groupRegistries(group).memberRegistries(consumer).processorId, + groupRegistries(group).memberRegistries(consumer).selectionKey, + groupRegistries(group).memberRegistries(consumer).sessionTimeoutMs) + } + } + + /** + * Get the RPC waiting lock and condition for the group + */ + def getConsumerRegistryLock(group: String) : Tuple2[ReentrantLock, Condition] = + (groupRegistries(group).rebalanceLock, + groupRegistries(group).rebalanceCondition) + + + /** + * Get group members snapshot + */ + def readConsumerRegistriesSnapshot(group: String) : Map[String, ConsumerRegistry] = { + val snapshot: AtomicReference[Map[String, ConsumerRegistry]] = new AtomicReference(groupRegistries(group).memberRegistries) + snapshot.get + } + + /** + * Get the rebalance failure bit + */ + def rebalanceNotFailed(group: String) : Boolean = !groupRegistries(group).rebalanceFailed + + /** + * Reset current handling rebalancer for the group + */ + def resetCurrRebalancer(group: String) = groupRegistries(group).currRebalancer = GroupRegistry.None + + /** + * Reset rebalancing failure bit + */ + def resetRebalanceFailed(group: String) = groupRegistries(group).rebalanceFailed = false + + /* -------------------------- For Test Only -------------------------- */ + + def blockPingCommands() = isBlocked.set(true) + + def unblockPingCommands() = isBlocked.set(false) +} Index: core/src/main/scala/kafka/server/ConfirmationSend.scala =================================================================== --- core/src/main/scala/kafka/server/ConfirmationSend.scala (revision 0) +++ core/src/main/scala/kafka/server/ConfirmationSend.scala (revision 0) @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.nio._ +import java.nio.channels._ +import kafka.network.Send +import kafka.common.ErrorMapping +import kafka.utils.nonthreadsafe +import scala.Array + +/** + * A zero-copy message response that writes the bytes needed directly from the file + * wholly in kernel space + */ +@nonthreadsafe +class ConfirmationSend(val errorCode: Int) extends Send { + + private val header = ByteBuffer.allocate(6) + + header.putInt(2 /* error code */) + header.putShort(errorCode.asInstanceOf[Short]) + header.rewind() + + var complete: Boolean = false + + def this() = this(ErrorMapping.NoError) + + def writeTo(channel: GatheringByteChannel): Int = { + expectIncomplete() + val written = channel.write(header) + // if we are done, mark it off + if (!header.hasRemaining) + complete = true + written + } + +} Index: core/src/main/scala/kafka/server/KafkaConfig.scala =================================================================== --- core/src/main/scala/kafka/server/KafkaConfig.scala (revision 1371523) +++ core/src/main/scala/kafka/server/KafkaConfig.scala (working copy) @@ -24,7 +24,7 @@ /** * Configuration settings for the kafka server */ -class KafkaConfig(props: Properties) extends ZKConfig(props) { +class KafkaConfig(props: Properties) extends CoordinatorConfig(props) { /* the port to listen and accept connections on */ val port: Int = Utils.getInt(props, "port", 6667) Index: core/src/main/scala/kafka/server/KafkaServer.scala =================================================================== --- core/src/main/scala/kafka/server/KafkaServer.scala (revision 1371523) +++ core/src/main/scala/kafka/server/KafkaServer.scala (working copy) @@ -20,9 +20,10 @@ import kafka.log.LogManager import java.util.concurrent.CountDownLatch import java.util.concurrent.atomic.AtomicBoolean -import kafka.utils.{Mx4jLoader, Utils, SystemTime, KafkaScheduler, Logging} import kafka.network.{SocketServerStats, SocketServer} import java.io.File +import org.I0Itec.zkclient.ZkClient +import kafka.utils._ /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required @@ -37,10 +38,12 @@ var socketServer: SocketServer = null - val scheduler = new KafkaScheduler(1, "kafka-logcleaner-", false) + var scheduler: KafkaScheduler = null private var logManager: LogManager = null + var consumerCoordinator: ConsumerCoordinator = null + /** * Start up API for bringing up a single instance of the Kafka server. * Instantiates the LogManager, the SocketServer and the request handlers - KafkaRequestHandlers @@ -55,29 +58,37 @@ needRecovery = false cleanShutDownFile.delete } + scheduler = new KafkaScheduler(1, "kafka-logcleaner-", false) logManager = new LogManager(config, scheduler, SystemTime, 1000L * 60 * config.logCleanupIntervalMinutes, 1000L * 60 * 60 * config.logRetentionHours, needRecovery) - - val handlers = new KafkaRequestHandlers(logManager) + + if (config.enableZookeeper) { + val coordinatorZkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, + config.zkConnectionTimeoutMs, ZKStringSerializer) + + consumerCoordinator = new ConsumerCoordinator(coordinatorZkClient, config.brokerId, config) + } + + val handlers = new KafkaRequestHandlers(logManager, consumerCoordinator) socketServer = new SocketServer(config.port, config.numThreads, config.monitoringPeriodSecs, - handlers.handlerFor, + handlers, config.socketSendBuffer, config.socketReceiveBuffer, config.maxSocketRequestSize) Utils.registerMBean(socketServer.stats, statsMBeanName) socketServer.startup() Mx4jLoader.maybeLoad - /** - * Registers this broker in ZK. After this, consumers can connect to broker. - * So this should happen after socket server start. - */ + + if (consumerCoordinator != null) consumerCoordinator.setSocketServer(socketServer) + logManager.startup() + info("Kafka server started.") } @@ -93,9 +104,15 @@ if (socketServer != null) socketServer.shutdown() Utils.unregisterMBean(statsMBeanName) + + // We need to first close the coordinator's zkClient then the broker's zkClient + if (consumerCoordinator != null) { + consumerCoordinator.shutdown + } if (logManager != null) logManager.close() + val cleanShutDownFile = new File(new File(config.logDir), CLEAN_SHUTDOWN_FILE) cleanShutDownFile.createNewFile Index: core/src/main/scala/kafka/server/RequestPurgatory.scala =================================================================== --- core/src/main/scala/kafka/server/RequestPurgatory.scala (revision 0) +++ core/src/main/scala/kafka/server/RequestPurgatory.scala (revision 0) @@ -0,0 +1,301 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import scala.collection._ +import java.util.LinkedList +import java.util.concurrent._ +import java.util.concurrent.atomic._ +import kafka.network._ +import kafka.utils._ + +/** + * A request whose processing needs to be delayed for at most the given delayMs + * The associated keys are used for bookeeping, and represent the "trigger" that causes this request to check if it is satisfied, + * for example a key could be a (topic, partition) pair. + */ +class DelayedRequest(val keys: Seq[Any], val request: Request, delayMs: Long) extends DelayedItem[Request](request, delayMs) { + val satisfied = new AtomicBoolean(false) +} + +/** + * A helper class for dealing with asynchronous requests with a timeout. A DelayedRequest has a request to delay + * and also a list of keys that can trigger the action. Implementations can add customized logic to control what it means for a given + * request to be satisfied. For example it could be that we are waiting for user-specified number of acks on a given (topic, partition) + * to be able to respond to a request or it could be that we are waiting for a given number of bytes to accumulate on a given request + * to be able to respond to that request (in the simple case we might wait for at least one byte to avoid busy waiting). + * + * For us the key is generally a (topic, partition) pair. + * By calling + * watch(delayedRequest) + * we will add triggers for each of the given keys. It is up to the user to then call + * val satisfied = update(key, request) + * when a request relevant to the given key occurs. This triggers bookeeping logic and returns back any requests satisfied by this + * new request. + * + * An implementation provides extends two helper functions + * def checkSatisfied(request: R, delayed: T): Boolean + * this function returns true if the given request (in combination with whatever previous requests have happened) satisfies the delayed + * request delayed. This method will likely also need to do whatever bookkeeping is necessary. + * + * The second function is + * def expire(delayed: T) + * this function handles delayed requests that have hit their time limit without being satisfied. + * + */ +abstract class RequestPurgatory[T <: DelayedRequest, R] { + + /* a list of requests watching each key */ + private val watchersForKey = new ConcurrentHashMap[Any, Watchers] + + /* background thread expiring requests that have been waiting too long */ + private val expiredRequestReaper = new ExpiredRequestReaper + private val expirationThread = Utils.daemonThread("request-expiration-task", expiredRequestReaper) + expirationThread.start() + + /** + * Add a new delayed request watching the contained keys + */ + def watch(delayedRequest: T) { + for(key <- delayedRequest.keys) { + var lst = watchersFor(key) + lst.add(delayedRequest) + } + expiredRequestReaper.enqueue(delayedRequest) + } + + /** + * Update any watchers and return a list of newly satisfied requests. + */ + def update(key: Any, request: R): Seq[T] = { + val w = watchersForKey.get(key) + if(w == null) + Seq.empty + else + w.collectSatisfiedRequests(request) + } + + /** + * Check if there are pending request of the key + */ + def noPendingDelayedRequest(key: Any) : Boolean = watchersFor(key).isEmpty + + def numPendingDelayedRequest(key: Any) : Int = watchersFor(key).size + + private def watchersFor(key: Any): Watchers = { + var lst = watchersForKey.get(key) + if(lst == null) { + watchersForKey.putIfAbsent(key, new Watchers) + lst = watchersForKey.get(key) + } + lst + } + + /** + * Check if this request satisfied this delayed request + */ + protected def checkSatisfied(request: R, delayed: T): Boolean + + /** + * Handle an expired delayed request + */ + protected def expire(delayed: T) + + /** + * Shutdown the expirey thread + */ + def shutdown() { + expiredRequestReaper.shutdown() + } + + /** + * A linked list of DelayedRequests watching some key with some associated bookeeping logic + */ + private class Watchers { + + /* a few magic parameters to help do cleanup to avoid accumulating old watchers */ + private val CleanupThresholdSize = 100 + private val CleanupThresholdPrct = 0.5 + + private val requests = new LinkedList[T] + + /* you can only change this if you have added something or marked something satisfied */ + var liveCount = 0.0 + + def isEmpty = { + synchronized { + purge + requests.isEmpty + } + } + + def size = { + synchronized { + purge + requests.size + } + } + + def add(t: T) { + synchronized { + requests.add(t) + liveCount += 1 + maybePurge() + } + } + + private def maybePurge() { + if(requests.size > CleanupThresholdSize && liveCount / requests.size < CleanupThresholdPrct) + purge + } + + private def purge() { + val iter = requests.iterator() + while(iter.hasNext) { + val curr = iter.next + if(curr.satisfied.get()) + iter.remove() + } + } + + def decLiveCount() { + synchronized { + liveCount -= 1 + } + } + + def collectSatisfiedRequests(request: R): Seq[T] = { + val response = new mutable.ArrayBuffer[T] + synchronized { + val iter = requests.iterator() + while(iter.hasNext) { + val curr = iter.next + if(curr.satisfied.get) { + // another thread has satisfied this request, remove it + iter.remove() + } else { + if(checkSatisfied(request, curr)) { + iter.remove() + val updated = curr.satisfied.compareAndSet(false, true) + if(updated == true) { + response += curr + liveCount -= 1 + expiredRequestReaper.satisfyRequest() + } + } + } + } + } + response + } + } + + /** + * Runnable to expire requests that have sat unfullfilled past their deadline + */ + private class ExpiredRequestReaper extends Runnable with Logging { + + /* a few magic parameters to help do cleanup to avoid accumulating old watchers */ + private val CleanupThresholdSize = 100 + private val CleanupThresholdPrct = 0.5 + + private val delayed = new DelayQueue[T] + private val running = new AtomicBoolean(true) + private val shutdownLatch = new CountDownLatch(1) + private val needsPurge = new AtomicBoolean(false) + /* The count of elements in the delay queue that are unsatisfied */ + private val unsatisfied = new AtomicInteger(0) + + /** Main loop for the expiry thread */ + def run() { + while(running.get) { + try { + val curr = pollExpired() + expire(curr) + } catch { + case ie: InterruptedException => + if(needsPurge.getAndSet(false)) { + val purged = purgeSatisfied() + debug("Forced purge of " + purged + " requests from delay queue.") + } + case e: Exception => + error("Error in long poll expiry thread: ", e) + } + } + shutdownLatch.countDown() + } + + /** Add a request to be expired */ + def enqueue(t: T) { + delayed.add(t) + unsatisfied.incrementAndGet() + if(unsatisfied.get > CleanupThresholdSize && unsatisfied.get / delayed.size.toDouble < CleanupThresholdPrct) + forcePurge() + } + + private def forcePurge() { + needsPurge.set(true) + expirationThread.interrupt() + } + + /** Shutdown the expiry thread*/ + def shutdown() { + debug("Shutting down request expiry thread") + running.set(false) + expirationThread.interrupt() + shutdownLatch.await() + } + + /** Record the fact that we satisfied a request in the stats for the expiry queue */ + def satisfyRequest(): Unit = unsatisfied.getAndDecrement() + + /** + * Get the next expired event + */ + private def pollExpired(): T = { + while(true) { + val curr = delayed.take() + val updated = curr.satisfied.compareAndSet(false, true) + if(updated) { + unsatisfied.getAndDecrement() + for(key <- curr.keys) + watchersFor(key).decLiveCount() + return curr + } + } + throw new RuntimeException("This should not happen") + } + + /** + * Delete all expired events from the delay queue + */ + private def purgeSatisfied(): Int = { + var purged = 0 + val iter = delayed.iterator() + while(iter.hasNext) { + val curr = iter.next() + if(curr.satisfied.get) { + iter.remove() + purged += 1 + } + } + purged + } + } + +} \ No newline at end of file Index: core/src/main/scala/kafka/server/KafkaRequestHandlers.scala =================================================================== --- core/src/main/scala/kafka/server/KafkaRequestHandlers.scala (revision 1371523) +++ core/src/main/scala/kafka/server/KafkaRequestHandlers.scala (working copy) @@ -25,11 +25,16 @@ import kafka.common.ErrorMapping import java.util.concurrent.atomic.AtomicLong import kafka.utils._ +import kafka.cluster.Cluster +import java.nio.channels.SelectionKey +import collection.mutable.Set + /** * Logic to handle the various Kafka requests */ -private[kafka] class KafkaRequestHandlers(val logManager: LogManager) extends Logging { +private[kafka] class KafkaRequestHandlers(val logManager: LogManager, + val consumerCoordinator: ConsumerCoordinator) extends Logging { private val requestLogger = Logger.getLogger("kafka.request.logger") @@ -40,9 +45,118 @@ case RequestKeys.MultiFetch => handleMultiFetchRequest _ case RequestKeys.MultiProduce => handleMultiProducerRequest _ case RequestKeys.Offsets => handleOffsetRequest _ + case RequestKeys.ClusterMetadata => handlerClusterMetadataRequest _ case _ => throw new IllegalStateException("No mapping found for handler id " + requestTypeId) } } + + def coordinatorHandleFor(requestTypeId: Short, request: Receive): Handler.CoordinatorHandler = { + if (!consumerCoordinator.amICurrentCoordinator) + throw new IllegalStateException("A coordinator request with id " + requestTypeId + " is received by a non-coordinator") + + requestTypeId match { + case RequestKeys.RegisterConsumer => handleRegisterConsumerRequest _ + case RequestKeys.Heartbeat => handleHeartbeatResponse _ + case RequestKeys.StopFetcher => handleStopFetcherResponse _ + case RequestKeys.StartFetcher => handleStartFetcherResponse _ + case _ => throw new IllegalStateException("No mapping found for coordinator handler id " + requestTypeId) + } + } + + def handleRegisterConsumerRequest(receive: Receive, processor: Processor, key: SelectionKey): Option[Send] = { + val request = RegisterConsumerRequest.readFrom(receive.buffer) + + trace("Handle register consumer request from consumer " + request.consumerId) + + // If the session timeout is less than the minimum viable value, reject it + if (consumerCoordinator.config.minCoordinatorSessionTimeoutMs > request.sessionTimeoutMs) { + warn("Consumer session timeout " + request.sessionTimeoutMs + " is smaller than the allowed minimum value " + consumerCoordinator.config.minCoordinatorSessionTimeoutMs + ", reject the request") + return Some(new RegistryReplySend(request, ErrorMapping.SessionTimeoutTooShortCode)) + } + + // Handle the newly added group if it is not registered + val groupId = request.consumerId.split("_")(0) + consumerCoordinator.handleNewGroup(groupId) + + // If the consumer is already registered, reject it + if (consumerCoordinator.isConsumerRegistered(request.consumerId)) { + warn("Some consumer with the same id " + request.consumerId + " has already registered, reject the request") + return Some(new RegistryReplySend(request, ErrorMapping.ConsumerAlreadyRegisteredCode)) + } + + // Otherwise accept it + val consumerRegistry = new ConsumerRegistry(request, processor.id, key) + + // For the newly added consumer to a known group, handle the added consumer + consumerCoordinator.handleNewGroupMember(consumerRegistry, groupId) + + info("Accepted the registry request from consumer " + request.consumerId) + + Some(new RegistryReplySend(request, ErrorMapping.NoError)) + } + + def handleHeartbeatResponse(receive: Receive, processor: Processor, key: SelectionKey): Option[Send] = { + val response = HeartbeatResponse.readFrom(receive.buffer) + + trace("Handle heartbeat response from consumer " + response.consumer) + + // Clean the watcher for the heartbeat request + consumerCoordinator.heartbeatRequestPurgatory.update(response.consumer, response) + + // If auto_commit is enabled, read the offset info + consumerCoordinator.tryUpdateConsumedOffsets(response.consumer, response.parts) + + None + } + + def handleStopFetcherResponse(receive: Receive, processor: Processor, key: SelectionKey): Option[Send] = { + val response = StopFetcherResponse.readFrom(receive.buffer) + + trace("Handle stop fetcher response from consumer " + response.consumer) + + // Clean the watcher for the stop fetcher request + val groupId = response.consumer.split("_")(0) + consumerCoordinator.stopFetcherRequestPurgatory.update(groupId, response) + + // If all the stop fetcher responses have been received, notify the rebalancer + if (consumerCoordinator.stopFetcherRequestPurgatory.noPendingDelayedRequest(groupId)) { + consumerCoordinator.handleReceivedAckForStartOrStopFetcherRequest(groupId) + } + + // If auto_commit is enabled, update the offset info + consumerCoordinator.tryUpdateConsumedOffsets(response.consumer, response.parts) + + None + } + + def handleStartFetcherResponse(receive: Receive, processor: Processor, key: SelectionKey): Option[Send] = { + val response = StartFetcherResponse.readFrom(receive.buffer) + + trace("Handle start fetcher response from consumer " + response.consumer) + + // Clean the watcher for the stop fetcher request + val groupId = response.consumer.split("_")(0) + consumerCoordinator.startFetcherRequestPurgatory.update(groupId, response) + + // If all the stop fetcher responses have been received, notify the rebalancer + if (consumerCoordinator.startFetcherRequestPurgatory.noPendingDelayedRequest(groupId)) { + consumerCoordinator.handleReceivedAckForStartOrStopFetcherRequest(groupId) + } + + None + } + + def handlerClusterMetadataRequest(receive: Receive): Option[Send] = { + // If the server is not completely started so the coordinator does not exist yet, + // return the server_not_ready error code + if (consumerCoordinator == null) + return Some(new ClusterInfoSend(new Cluster(), -1, ErrorMapping.CoordinatorNotAvailableCode)) + + val cluster = ZkUtils.getCluster(consumerCoordinator.zkClient) + val coordinatorId = consumerCoordinator.currentCoordinatorId + + Some(new ClusterInfoSend(cluster, coordinatorId)) + } def handleProducerRequest(receive: Receive): Option[Send] = { val sTime = SystemTime.milliseconds Index: core/src/main/scala/kafka/server/ClusterInfoSend.scala =================================================================== --- core/src/main/scala/kafka/server/ClusterInfoSend.scala (revision 0) +++ core/src/main/scala/kafka/server/ClusterInfoSend.scala (revision 0) @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.nio._ +import java.nio.channels._ +import kafka.network._ +import kafka.utils._ +import kafka.common.ErrorMapping +import kafka.cluster.Cluster +import scala.Array + +/** + * A zero-copy message response that writes the bytes needed directly from the file + * wholly in kernel space + */ +@nonthreadsafe +private[server] class ClusterInfoSend(val cluster: Cluster, val coordinatorId: Int, val errorCode: Int) extends Send { + + private val size: Long = 4 /* coordinator id */ + 2 /* number of brokers */ + cluster.sizeInBytes + private val header = ByteBuffer.allocate(6) + private val content = ByteBuffer.allocate(size.asInstanceOf[Int]) + + header.putInt(size.asInstanceOf[Int] + 2 /* error code */) + header.putShort(errorCode.asInstanceOf[Short]) + header.rewind() + + content.putInt(coordinatorId) + content.putShort(cluster.size.asInstanceOf[Short]) + cluster.writeTo(content) + content.rewind() + + var complete: Boolean = false + + def this(cluster: Cluster, coordinatorId: Int) = this(cluster, coordinatorId, ErrorMapping.NoError) + + def writeTo(channel: GatheringByteChannel): Int = { + expectIncomplete() + val written = channel.write(Array(header, content)) + // if we are done, mark it off + if(!content.hasRemaining) + complete = true + written.asInstanceOf[Int] + } + +} Index: core/src/main/scala/kafka/server/CoordinatorConfig.scala =================================================================== --- core/src/main/scala/kafka/server/CoordinatorConfig.scala (revision 0) +++ core/src/main/scala/kafka/server/CoordinatorConfig.scala (revision 0) @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util.Properties +import kafka.utils.{Utils, ZKConfig} +import kafka.message.Message + +/** + * Configuration settings for the consumer coordinator + */ +class CoordinatorConfig(props: Properties) extends ZKConfig(props) { + /* max number of retries during consumer rebalance */ + val maxRebalanceRetries = Utils.getInt(props, "rebalance.retries.max", 4) + + /* backoff time between retries during rebalance, by default always retries rebalance immediately */ + val rebalanceBackoffMs = Utils.getInt(props, "rebalance.backoff.ms", 2000) + + /* rebalancing threadpool size */ + val numRebalanceThreads = Utils.getIntInRange(props, "num.rebalance.threads", 1, (1, Int.MaxValue)) + + /* minimum viable value for consumer session timeout */ + val minCoordinatorSessionTimeoutMs = Utils.getInt(props, "min.coordinator.session.timeout.ms", 2000) + + /* maximum number of allowed failed responses of heartbeats for consumers */ + val maxNumRequestFailures = Utils.getInt(props, "max.num.request.failures", 1) + + /* size of the rebalance request queue */ + val rebalanceQueueSize = Utils.getInt(props, "rebalance.queue.size", 1000) + + /* the frequency in ms that the consumer offsets are committed to zookeeper */ + val autoCommitIntervalMs = Utils.getInt(props, "autocommit.interval.ms", 10 * 1000) +} Index: core/src/main/scala/kafka/server/StopFetcherRequestPurgatory.scala =================================================================== --- core/src/main/scala/kafka/server/StopFetcherRequestPurgatory.scala (revision 0) +++ core/src/main/scala/kafka/server/StopFetcherRequestPurgatory.scala (revision 0) @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import scala.collection._ +import kafka.api._ + + +/** + * A delayed stop-fetcher request, whose watcher is based on groups + */ +class DelayedStopFetcher(keys: Seq[Any], request: StopFetcherRequest, delayMs: Long) extends DelayedRequest(keys, request, delayMs) + +class StopFetcherRequestPurgatory(coordinator : ConsumerCoordinator) + extends RequestPurgatory[DelayedStopFetcher, StopFetcherResponse] { + + /** + * A stop-fetcher request is satisfied when the corresponding response with the same consumer id is received + */ + def checkSatisfied(response: StopFetcherResponse, delayedRequest: DelayedStopFetcher): Boolean = { + response.consumer == delayedRequest.request.asInstanceOf[StopFetcherRequest].consumerId + } + + /** + * When a request expires just let the coordinator know + */ + def expire(delayedRequest: DelayedStopFetcher) { + coordinator.handleStopOrStartFetcherRequestExpired(delayedRequest.request.asInstanceOf[StopFetcherRequest].consumerId) + } +} Index: core/src/main/scala/kafka/server/RebalanceRequestHandler.scala =================================================================== --- core/src/main/scala/kafka/server/RebalanceRequestHandler.scala (revision 0) +++ core/src/main/scala/kafka/server/RebalanceRequestHandler.scala (revision 0) @@ -0,0 +1,543 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server + +import kafka.api._ +import kafka.common._ +import kafka.cluster._ +import kafka.utils._ +import kafka.utils.ZkUtils._ +import scala.collection._ +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.{BlockingQueue, TimeUnit, CountDownLatch} +import java.nio.channels.CancelledKeyException +import org.I0Itec.zkclient.ZkClient +import org.I0Itec.zkclient.exception.ZkInterruptedException +import kafka.consumer.SimpleConsumer +import kafka.consumer.ConsumerConfig + +class RebalanceRequestHandler(val zkClient: ZkClient, + val coordinator: ConsumerCoordinator, + val requestQueue: BlockingQueue[String]) extends Thread with Logging { + + val isRunning: AtomicBoolean = new AtomicBoolean(true) + + private val shutdownLatch = new CountDownLatch(1) + + override def run() { + try { + while(isRunning.get()) { + // get outstanding state change requests for this broker + val group = requestQueue.take() + + if (group != null){ + info(("Rebalancer starts to rebalance for group %s").format(group)) + + if (!coordinator.resetRebalanceInitiated(group)) + throw new IllegalStateException("An unexpected rebalance request is received from the request queue") + + val startTime = System.currentTimeMillis + + var done = false + var i = 1 + while (i <= coordinator.config.maxRebalanceRetries && !done) { + debug("Begin rebalancing for group " + group + " try #" + i) + + /* Get the snapshot of the current group registry info, and reset the rebalance failure bit */ + val consumerRegistries = coordinator.readConsumerRegistriesSnapshot(group) + coordinator.resetRebalanceFailed(group) + + done = rebalance(group, consumerRegistries) + + if (done) { + debug("End rebalancing for group " + group + " at attempt #" + i) + + val endTime = System.currentTimeMillis + val duration = endTime - startTime + info("Rebalancer completes for group " + group + " with " + duration + " miliseconds at attempt #" + i) + + if (!coordinator.isGroupUnderRebalance(group)) + coordinator.resetCurrRebalancer(group) + }else { + warn("Rebalancing attempt #" + i + " failed") + + Thread.sleep(coordinator.config.rebalanceBackoffMs) + + // One possibility for failure is that the handler is interrupted during server shutdown; + // in this case, the handler thread should terminate + if (!isRunning.get()) done = true + } + i += 1 + } + + if (i > coordinator.config.maxRebalanceRetries) { + info("Rebalancer cannot complete for group " + group) + throw new ConsumerRebalanceFailedException(group + " can't rebalance after " + coordinator.config.maxRebalanceRetries + " retries") + } + } + } + }catch { + case e: ZkInterruptedException => info("ZkClient interrupted. Shutting down") + case e: InterruptedException => info("RebalanceRequestHandler thread interrupted. Shutting down") + case e1 => error("Error in rebalance request handler. Shutting down due to ", e1) + } + + shutdownComplete() + } + + def shutdown() { + isRunning.set(false) + // Interrupt the thread within the semaphore to make sure zkClient is not working + interrupt() + shutdownLatch.await() + info("Rebalancer shutdown completed") + } + + private def shutdownComplete() = shutdownLatch.countDown + + private def sendStopFetcherRequest(group: String, + consumerRegistries: Map[String, ConsumerRegistry]) : Boolean = { + for ((consumer, registry) <- consumerRegistries) { + val request = new StopFetcherRequest(consumer) + + try { + debug("Sending stop fetcher request to consumer %s".format(consumer)) + + coordinator.socketServer.send(registry.processorId, registry.selectionKey, request) + + // Set the watcher for the stop fetcher request + val delayMs = registry.sessionTimeoutMs * 2/3 + val delayedStopFetcher = new DelayedStopFetcher(List[String](group), request, delayMs) + coordinator.stopFetcherRequestPurgatory.watch(delayedStopFetcher) + + debug("Add the expire watcher for the stop fetcher request to consumer %s with timeout %d" + .format(consumer, delayMs)) + + trace("Purgatory now has %d expire watchers for group %s" + .format(coordinator.stopFetcherRequestPurgatory.numPendingDelayedRequest(group), group)) + } + catch { + case e: CancelledKeyException => { + info("Channel cancelled while sending stop fetcher request, mark consumer %s as failed and retry rebalance" + .format(consumer)) + coordinator.handleConsumerFailure(consumer, group) + + return false + } + case e1 => throw e1 + } + } + + true + } + + private def sendStartFetcherRequest(group: String, + consumerRegistries: Map[String, ConsumerRegistry], + partitionOwnership: PartitionAssignmentStrategy) : Boolean = { + val commandPerConsumer = partitionOwnership.toOwnershipPerConsumer + + /* + * NOTE: add the current cluster metadata to the start-fetcher request in case it has changed since + * the consumers consulted brokers for it; this is used in order to avoid unnecessary rebalance retrials + */ + val cluster = ZkUtils.getCluster(coordinator.zkClient) + for ((consumer, registry) <- consumerRegistries) { + var request: StartFetcherRequest = null + + // It is possible that some consumers do not get any partitions, + // if so just send an empty start-fetcher request + if (commandPerConsumer.contains(consumer)) { + if (registry.autoOffsetCommit) { + /* + * NOTE: do not validate partition offset, since this is a rare case that + * the offset would be out-of-range; if that do happens, just trigger the rebalance + */ + getPartitionOffset(group, registry, commandPerConsumer(consumer)) + //getAndValidatePartitionOffset(group, registry, commandPerConsumer(consumer)) + } + + request = new StartFetcherRequest(consumer, commandPerConsumer(consumer), cluster) + } + else { + request = new StartFetcherRequest(consumer, List.empty[ConsumerPartitionOwnershipInfo], cluster) + } + + try { + debug("Sending start fetcher request to consumer %s".format(consumer)) + + coordinator.socketServer.send(registry.processorId, registry.selectionKey, request) + + // Set the watcher for the stop fetcher request + val delayMs = registry.sessionTimeoutMs * 2/3 + val delayedStartFetcher = new DelayedStartFetcher(List[String](group), request, delayMs) + coordinator.startFetcherRequestPurgatory.watch(delayedStartFetcher) + + debug("Add the expire watcher for the start fetcher request to consumer %s with timeout %d" + .format(consumer, delayMs)) + + trace("Purgatory now has %d expire watchers for group %s" + .format(coordinator.startFetcherRequestPurgatory.numPendingDelayedRequest(group), group)) + } + catch { + case e: CancelledKeyException => { + info("Channel cancelled while sending start fetcher request, mark consumer %s as failed and retry rebalance" + .format(consumer)) + coordinator.handleConsumerFailure(consumer, group) + + return false + } + case e1 => throw e1 + } + } + + true + } + + /** + * The whole rebalance process can only be based on a consistent view of the consumer/group registry + * This view will then not change even if the underlying map has changed + */ + private def rebalance(group: String, consumerRegistries: Map[String, ConsumerRegistry]): Boolean = { + // If the group is zero consumers, just clear the ZK ownership and then return + if (consumerRegistries.isEmpty) { + info("The group is already empty, no need to rebalance but just clear the ownership in ZK") + val groupDirs = new ZKGroupDirs(group) + deletePathRecursive(coordinator.zkClient, groupDirs.consumerOwnersDir) + return true + } + + // Compute all the topic ownership assignments for the group + val topics = coordinator.getTopicsForConsumers(consumerRegistries) + val partitionOwnership = computeOwnership(group, consumerRegistries, topics) + + // Check if rebalance is necessary by comparing the old ownership info in ZK + if (isRebalanceNecessary(partitionOwnership)) { + + // Get the maximum session timeout from the consumers of the group + var maxSesstionTimeoutMs: Int = 0 + for ((consumer, registry) <- consumerRegistries) { + if (registry.sessionTimeoutMs > maxSesstionTimeoutMs) + maxSesstionTimeoutMs = registry.sessionTimeoutMs + } + + // Send out stop-fetcher command to every consumer in the group + if (!sendStopFetcherRequest(group, consumerRegistries)) { + debug("Some consumers in group %s are no longer reachable while trying to send stop fetcher request, retry rebalance".format(group)) + return false + } + else { + debug("Finished sending stop fetcher request to everyone in group " + group) + } + + // Wait for all the stop fetcher request to be responded + if (coordinator.stopFetcherRequestPurgatory.noPendingDelayedRequest(group)) { + debug("All stop fetcher request timeout watchers have been cleared for group " + group + ", no need to wait") + } + else if (waitForAllResponses(group, maxSesstionTimeoutMs)) { + debug("All consumers in group %s have replied the stop fetcher request".format(group)) + } + else { + debug("Not every consumer in group %s have replied the stop fetcher request, retry rebalance".format(group)) + return false + } + + // Enforce a write of offsets to ZK + coordinator.commitOffset(group) + + // Send out start-fetcher command to every consumer in the group + if (!sendStartFetcherRequest(group, consumerRegistries, partitionOwnership)) { + debug("Some consumers in group %s are no longer reachable while trying to send start fetcher request, retry rebalance".format(group)) + return false + } + else { + debug("Finished sending start fetcher request to everyone in group " + group) + } + + // Wait for all the start fetcher request to be responded + if (coordinator.startFetcherRequestPurgatory.noPendingDelayedRequest(group)) { + debug("All start fetcher request timeout watchers have been cleared for group " + group + ", no need to wait") + } + else if (waitForAllResponses(group, maxSesstionTimeoutMs)) { + debug("All consumers in group %s have replied the start fetcher request".format(group)) + } + else { + debug("Not every consumer in group %s have replied the start fetcher request, retry rebalance".format(group)) + return false + } + + // If rebalance succeeds, reflect the new ownership in ZK + updatePartitionOwnershipInZK(group, partitionOwnership.getPartitionOwnership) + + true + } + else { + debug("Rebalance is not necessary since the ownership recorded in ZK matches the computed result") + true + } + } + + private def isRebalanceNecessary(partitionOwnership: PartitionAssignmentStrategy): Boolean = { + val topics = partitionOwnership.getTopics + val group = partitionOwnership.getGroup + + for (topic <- topics) { + val groupTopicDir = new ZKGroupTopicDirs(group, topic) + val allOwnedPartitionsForTopic = getChildrenParentMayNotExist(zkClient, groupTopicDir.consumerOwnerDir) + + // If there is no ownership info yet, rebalance is necessary + if (allOwnedPartitionsForTopic == Nil) { + debug("Rebalance for group %s is necessary since there is no ownership info in ZK yet".format(group)) + return true + } + + val curPartitions: List[String] = partitionOwnership.getPartitionsPerTopicMap(coordinator).get(topic).get + + // If some partitions are not owned by any consumers, rebalance is necessary + val notOwnedParitions = curPartitions filterNot (allOwnedPartitionsForTopic contains) + if (!notOwnedParitions.isEmpty) { + debug("Rebalance for group %s regarding topic %s is necessary since some partitions %s are not owned by any consumers" + .format(group, topic, notOwnedParitions)) + return true + } + + // If some partitions that are claimed do not exist any more, their hosting + // brokers may already be gone, rebalance is necessary + val notExistPartitions = allOwnedPartitionsForTopic filterNot (curPartitions contains) + if (!notExistPartitions.isEmpty) { + debug("Rebalance for group %s regarding topic %s is necessary since some claimed partitions %s no longer exist on any brokers" + .format(group, topic, notExistPartitions)) + return true + } + + // If ownership does not match, rebalance is necessary + val ownedPartitionsPerConsumer = new mutable.HashMap[String, List[String]]() + + allOwnedPartitionsForTopic.foreach { p => + val consumerThreadId = readData(zkClient, groupTopicDir.consumerOwnerDir + "/" + p) + + ownedPartitionsPerConsumer.get(consumerThreadId) match { + case None => ownedPartitionsPerConsumer.put(consumerThreadId, List(p)) + case Some(lst) => ownedPartitionsPerConsumer.put(consumerThreadId, p :: lst) + } + } + + val computedOwnership = partitionOwnership.getPartitionOwnership + val consumersComputed = computedOwnership(topic).keySet + val consumersFromZK = ownedPartitionsPerConsumer.keySet + + if (!((consumersComputed diff consumersFromZK) union (consumersFromZK diff consumersComputed)).isEmpty) { + debug("Rebalance for group %s regarding topic %s is necessary since the lists of consumers that own some partitions do not match" + .format(group, topic)) + return true + } + + for (consumer <- consumersComputed) { + val partitionsComputed = computedOwnership(topic)(consumer) + val partitionsFromZK = ownedPartitionsPerConsumer(consumer) + + if (!((partitionsComputed diff partitionsFromZK) union (partitionsFromZK diff partitionsComputed)).isEmpty) { + debug("Rebalance for group %s regarding topic %s is necessary since the owned partitions for consumer %s do not match" + .format(group, topic, consumer)) + return true + } + } + } + + false + } + + private def computeOwnership(group: String, + consumerRegistries: Map[String, ConsumerRegistry], + topics: Set[String]): PartitionAssignmentStrategy = + new RangePartitionAssignmentStrategy(coordinator, group, consumerRegistries, topics) + + private def waitForAllResponses(group: String, timeoutMs: Long): Boolean = { + val lockAndCond = coordinator.getConsumerRegistryLock(group) + + lockAndCond._1.lock() + try { + lockAndCond._2.await(timeoutMs, TimeUnit.MILLISECONDS) + coordinator.rebalanceNotFailed(group) + } catch { + case e: InterruptedException => { + error("Rebalance request handler thread is interrupted while waiting for rebalancing consumers to finish") + false + } + case e1 => throw e1 + } finally { + lockAndCond._1.unlock() + } + } + + private def updatePartitionOwnershipInZK(group: String, ownership: Map[String, Map[String, List[String]]]) = { + debug("Update new partition ownership in ZK") + + // First delete all old ownership entries if there is any + val groupDirs = new ZKGroupDirs(group) + deletePathRecursive(coordinator.zkClient, groupDirs.consumerOwnersDir) + + // Then reflect new ownership + for (topic <- ownership.keySet) { + val topicDirs = new ZKGroupTopicDirs(group, topic) + for (threadId <- ownership(topic).keySet) { + for (partition <- ownership(topic)(threadId)) { + val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition + createPersistentPath(coordinator.zkClient, partitionOwnerPath, threadId) + debug(threadId + " successfully owned partition " + partition + " for topic " + topic) + } + } + } + } + + private def getPartitionOffset(group: String, + consumerRegistry: ConsumerRegistry, + threadParts: List[ConsumerPartitionOwnershipInfo]) : Boolean = { + val cluster = ZkUtils.getCluster(coordinator.zkClient) + val offsetRequests = new mutable.HashMap[Int, List[(String, Int, PartitionAndOffset)]] + + for (threadPart <- threadParts) { + for (part <- threadPart.parts) { + val partInfo: Partition = Partition.parse(part.partition) + + cluster.getBroker(partInfo.brokerId) match { + case Some(broker) => { + offsetRequests.get(partInfo.brokerId) match { + // Prepare the offset requests for each broker + case Some(curRequests) => offsetRequests.put(partInfo.brokerId, (part.topic, partInfo.partId, part) :: curRequests) + case _ => offsetRequests.put(partInfo.brokerId, List((part.topic, partInfo.partId, part))) + } + } + // If the broker's host does not exist anymore, directly fail the attempt + case _ => return false + } + } + } + + for ((brokerId, requests) <- offsetRequests) { + for (request <- requests) { + // Try to get the offset from ZK + val topicDirs = new ZKGroupTopicDirs(group, request._1) + val znode = topicDirs.consumerOffsetDir + "/" + brokerId + "-" + request._2 + val offsetString = readDataMaybeNull(coordinator.zkClient, znode) + + var offset: Long = -1 + if (offsetString == null) { + // If no value stored in ZK, directly consult the broker + val broker = cluster.getBroker(brokerId).get + val simpleConsumer = new SimpleConsumer(broker.host, + broker.port, ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize) + + offset = consumerRegistry.autoOffsetReset match { + case OffsetRequest.SmallestTimeString => { + val offsets = simpleConsumer.getOffsetsBefore(request._1, request._2, OffsetRequest.EarliestTime, 1) + offsets(0) + } + case OffsetRequest.LargestTimeString => { + val offsets = simpleConsumer.getOffsetsBefore(request._1, request._2, OffsetRequest.LatestTime, 1) + offsets(0) + } + case _ => + throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig") + } + + simpleConsumer.close + } + else { + // Otherwise directly read the offset + offset = offsetString.toLong + } + + request._3.offset = offset + } + } + + true + } + + private def getAndValidatePartitionOffset(group: String, + consumerRegistry: ConsumerRegistry, + threadParts: List[ConsumerPartitionOwnershipInfo]) : Boolean = { + val cluster = ZkUtils.getCluster(coordinator.zkClient) + val offsetRequests = new mutable.HashMap[Int, List[(String, Int, PartitionAndOffset)]] + + for (threadPart <- threadParts) { + for (part <- threadPart.parts) { + val partInfo: Partition = Partition.parse(part.partition) + + cluster.getBroker(partInfo.brokerId) match { + case Some(broker) => { + offsetRequests.get(partInfo.brokerId) match { + // Prepare the offset requests for each broker + case Some(curRequests) => offsetRequests.put(partInfo.brokerId, (part.topic, partInfo.partId, part) :: curRequests) + case _ => offsetRequests.put(partInfo.brokerId, List((part.topic, partInfo.partId, part))) + } + } + // If the broker's host does not exist anymore, directly fail the attempt + case _ => return false + } + } + } + + for ((brokerId, requests) <- offsetRequests) { + val broker = cluster.getBroker(brokerId).get + val simpleConsumer = new SimpleConsumer(broker.host, + broker.port, ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize) + + for (request <- requests) { + // Try to get the offset from ZK + val topicDirs = new ZKGroupTopicDirs(group, request._1) + val znode = topicDirs.consumerOffsetDir + "/" + brokerId + "-" + request._2 + val offsetString = readDataMaybeNull(coordinator.zkClient, znode) + + var offset: Long = -1 + if (offsetString == null) { + // If no value stored in ZK, directly consult the broker + offset = consumerRegistry.autoOffsetReset match { + case OffsetRequest.SmallestTimeString => { + val offsets = simpleConsumer.getOffsetsBefore(request._1, request._2, OffsetRequest.EarliestTime, 1) + offsets(0) + } + case OffsetRequest.LargestTimeString => { + val offsets = simpleConsumer.getOffsetsBefore(request._1, request._2, OffsetRequest.LatestTime, 1) + offsets(0) + } + case _ => + throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig") + } + + trace("Get offset " + offset + " from host broker for partition " + request._1 + ":" + brokerId + "-" + request._2) + } + else { + // Otherwise validate by the earliest offset from the broker + val offsetFromZK = offsetString.toLong + val offsetFromBroker = { + val offsets = simpleConsumer.getOffsetsBefore(request._1, request._2, OffsetRequest.EarliestTime, 1) + offsets(0) + } + offset = offsetFromZK.max(offsetFromBroker) + + trace("Get offset " + offsetFromBroker + " from host broker and " + offsetFromZK + " from ZK for partition " + request._1 + ":" + brokerId + "-" + request._2) + } + + request._3.offset = offset + } + simpleConsumer.close + } + + true + } +} Index: core/src/main/scala/kafka/server/StartFetcherRequestPurgatory.scala =================================================================== --- core/src/main/scala/kafka/server/StartFetcherRequestPurgatory.scala (revision 0) +++ core/src/main/scala/kafka/server/StartFetcherRequestPurgatory.scala (revision 0) @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import scala.collection._ +import kafka.api._ + + +/** + * A delayed start-fetcher request, whose watcher is based on groups + */ +class DelayedStartFetcher(keys: Seq[Any], request: StartFetcherRequest, delayMs: Long) extends DelayedRequest(keys, request, delayMs) + +class StartFetcherRequestPurgatory(coordinator : ConsumerCoordinator) + extends RequestPurgatory[DelayedStartFetcher, StartFetcherResponse] { + + /** + * A start-fetcher request is satisfied when the corresponding response with the same consumer id is received + */ + def checkSatisfied(response: StartFetcherResponse, delayedRequest: DelayedStartFetcher): Boolean = { + response.consumer == delayedRequest.request.asInstanceOf[StartFetcherRequest].consumerStreamId + } + + /** + * When a request expires just let the coordinator know + */ + def expire(delayedRequest: DelayedStartFetcher) { + coordinator.handleStopOrStartFetcherRequestExpired(delayedRequest.request.asInstanceOf[StartFetcherRequest].consumerStreamId) + } +} + Index: core/src/main/scala/kafka/server/PartitionAssignmentStrategy.scala =================================================================== --- core/src/main/scala/kafka/server/PartitionAssignmentStrategy.scala (revision 0) +++ core/src/main/scala/kafka/server/PartitionAssignmentStrategy.scala (revision 0) @@ -0,0 +1,153 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.utils.ZkUtils._ +import kafka.utils.Logging +import kafka.api.{ConsumerRegistry, PartitionAndOffset, ConsumerPartitionOwnershipInfo} +import scala._ +import scala.collection.mutable.{Map, HashMap} + +object PartitionAssignmentStrategy { + def parseTopicPartitions(ownershipString: String): List[(String, String)] = { + val ownershipPartitions = ownershipString.split(", ").toList + val ownershipTuples = ownershipPartitions.map { p => + val topicPartition = p.split(":") + if(topicPartition.length != 2) + throw new IllegalArgumentException("Expected name in the form x:y.") + (topicPartition(0), topicPartition(1)) + } + ownershipTuples + } +} + +abstract class PartitionAssignmentStrategy(group: String, topics: collection.Set[String]) { + var consumersPerTopicMap: Map[String, List[String]] = null + var partitionsPerTopicMap: Map[String, List[String]] = null + var partitionOwnershipCommand: Map[String, List[ConsumerPartitionOwnershipInfo]] = null + var partitionOwnershipDecision: Map[String, Map[String, List[String]]] = null + + def getGroup = group + + def getTopics = topics + + def getPartitionOwnership: Map[String, Map[String, List[String]]] + + def getConsumersPerTopicMap(coordinator: ConsumerCoordinator, + consumerRegistries: collection.Map[String, ConsumerRegistry]) = { + if (consumersPerTopicMap == null) + consumersPerTopicMap = coordinator.getConsumersPerTopic(consumerRegistries) + consumersPerTopicMap + } + + def getPartitionsPerTopicMap(coordinator: ConsumerCoordinator) = { + if (partitionsPerTopicMap == null) + partitionsPerTopicMap = getPartitionsForTopics(coordinator.zkClient, topics.iterator) + partitionsPerTopicMap + } + + def toOwnershipPerConsumer: Map[String, List[ConsumerPartitionOwnershipInfo]] = { + partitionOwnershipDecision = getPartitionOwnership + + if (partitionOwnershipCommand == null) { + val partitionOwnershipPerThread = new HashMap[String, List[PartitionAndOffset]]() + + for (topic <- topics) { + val consumerThreads = partitionOwnershipDecision(topic).keySet + for (consumerThreadId <- consumerThreads) { + val consumerOwnedPartitions = partitionOwnershipDecision(topic)(consumerThreadId) + for (partition <- consumerOwnedPartitions) { + partitionOwnershipPerThread.get(consumerThreadId) match { + case Some(curPartOffsets) => partitionOwnershipPerThread.put(consumerThreadId, new PartitionAndOffset(topic, partition, -1) :: curPartOffsets) + case _ => partitionOwnershipPerThread.put(consumerThreadId, List(new PartitionAndOffset(topic, partition, -1))) + } + } + } + } + + partitionOwnershipCommand = new HashMap[String, List[ConsumerPartitionOwnershipInfo]]() + + for ( (thread, ownedParts) <- partitionOwnershipPerThread ) { + val consumerId = thread.substring(0, thread.lastIndexOf("-")) + partitionOwnershipCommand.get(consumerId) match { + case None => partitionOwnershipCommand.put(consumerId, List[ConsumerPartitionOwnershipInfo](new ConsumerPartitionOwnershipInfo(thread, ownedParts))) + case Some(lst) => partitionOwnershipCommand.put(consumerId, new ConsumerPartitionOwnershipInfo(thread, ownedParts) :: lst) + } + } + } + + partitionOwnershipCommand + } +} + +class RangePartitionAssignmentStrategy(coordinator: ConsumerCoordinator, + group: String, + consumerRegistries: collection.Map[String, ConsumerRegistry], + topics: collection.Set[String]) + extends PartitionAssignmentStrategy(group, topics) with Logging { + + def getPartitionOwnership: Map[String, Map[String, List[String]]] = { + partitionsPerTopicMap = getPartitionsPerTopicMap(coordinator) + consumersPerTopicMap = getConsumersPerTopicMap(coordinator, consumerRegistries) + + if (partitionOwnershipDecision == null) { + partitionOwnershipDecision = new HashMap[String, Map[String, List[String]]]() + for (topic <- topics) { + partitionOwnershipDecision += (topic -> HashMap[String, List[String]]()) + + val curConsumers = consumersPerTopicMap.get(topic).getOrElse(List[String]()) + val curPartitions: List[String] = partitionsPerTopicMap.get(topic).getOrElse(List[String]()) + + if (curConsumers.nonEmpty && curPartitions.nonEmpty) { + val nPartsPerConsumer = curPartitions.size / curConsumers.size + val nConsumersWithExtraPart = curPartitions.size % curConsumers.size + + trace("Coordinator rebalancing for group " + group + " and topic " + topic + + " with the following partitions: " + curPartitions + " and consumers: " + curConsumers) + + for (consumerThreadId <- curConsumers) { + val myConsumerPosition = curConsumers.findIndexOf(_ == consumerThreadId) + assert(myConsumerPosition >= 0) + val startPart = nPartsPerConsumer*myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) + val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1) + + /** + * Range-partition the sorted partitions to consumers for better locality. + * The first few consumers pick up an extra partition, if any. + */ + if (nParts <= 0) + warn("No broker partitions consumed by consumer " + group + ":" + consumerThreadId + " for topic " + topic) + else { + for (i <- startPart until startPart + nParts) { + val partition = curPartitions(i) + trace(topic + ":" + partition + " is assigned to consumer " + group + ":" + consumerThreadId) + + partitionOwnershipDecision(topic).get(consumerThreadId) match { + case None => partitionOwnershipDecision(topic).put(consumerThreadId, List(partition)) + case Some(lst) => partitionOwnershipDecision(topic).put(consumerThreadId, partition :: lst) + } + } + } + } + } + } + } + + partitionOwnershipDecision + } +} Index: core/src/main/scala/kafka/server/HeartbeatRequestPurgatory.scala =================================================================== --- core/src/main/scala/kafka/server/HeartbeatRequestPurgatory.scala (revision 0) +++ core/src/main/scala/kafka/server/HeartbeatRequestPurgatory.scala (revision 0) @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import scala.collection._ +import kafka.api._ + + +/** + * A delayed heartbeat request + */ +class DelayedHeartbeat(keys: Seq[Any], request: HeartbeatRequest, delayMs: Long) extends DelayedRequest(keys, request, delayMs) + +class HeartbeatRequestPurgatory(coordinator : ConsumerCoordinator) extends RequestPurgatory[DelayedHeartbeat, HeartbeatResponse] { + + /** + * A heartbeat request is satisfied when the corresponding heartbeat response is received + */ + def checkSatisfied(response: HeartbeatResponse, delayedHeartbeat: DelayedHeartbeat): Boolean = { + true + } + + /** + * When a request expires just let the coordinator know + */ + def expire(delayedRequest: DelayedHeartbeat) { + coordinator.handleHeartbeatRequestExpired(delayedRequest.request.asInstanceOf[HeartbeatRequest].consumerId) + } +} Index: core/src/main/scala/kafka/common/ErrorMapping.scala =================================================================== --- core/src/main/scala/kafka/common/ErrorMapping.scala (revision 1371523) +++ core/src/main/scala/kafka/common/ErrorMapping.scala (working copy) @@ -33,6 +33,10 @@ val InvalidMessageCode = 2 val WrongPartitionCode = 3 val InvalidFetchSizeCode = 4 + val ServerNotReadyCode = 5 + val SessionTimeoutTooShortCode = 6 + val ConsumerAlreadyRegisteredCode = 7 + val CoordinatorNotAvailableCode = 8 private val exceptionToCode = Map[Class[Throwable], Int]( Index: core/src/main/scala/kafka/api/StopFetcherRequest.scala =================================================================== --- core/src/main/scala/kafka/api/StopFetcherRequest.scala (revision 0) +++ core/src/main/scala/kafka/api/StopFetcherRequest.scala (revision 0) @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio._ +import kafka.network.Request +import kafka.utils._ + +object StopFetcherRequest { + + def readFrom(buffer: ByteBuffer): StopFetcherRequest = { + val consumerId = Utils.readShortString(buffer, "UTF-8") + + new StopFetcherRequest(consumerId) + } +} + +class StopFetcherRequest(val consumerId: String) extends Request(RequestKeys.StopFetcher) { + + def writeTo(buffer: ByteBuffer) = { + Utils.writeShortString(buffer, consumerId, "UTF-8") + } + + def sizeInBytes(): Int = 2 + consumerId.size /* Consumer ID */ + + override def toString(): String= "StopFetcherRequest(consumer:" + consumerId + ")" +} + Index: core/src/main/scala/kafka/api/ClusterMetadataResponse.scala =================================================================== --- core/src/main/scala/kafka/api/ClusterMetadataResponse.scala (revision 0) +++ core/src/main/scala/kafka/api/ClusterMetadataResponse.scala (revision 0) @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer +import kafka.cluster.{Cluster, Broker} + + +object ClusterMetadataResponse { + def readFrom(buffer: ByteBuffer): ClusterMetadataResponse = { + val coordinatorId = buffer.getInt + val numBrokers = buffer.getShort + + var brokers: List[Broker] = Nil + for (i <- 1 to numBrokers) brokers ::= Broker.readFrom(buffer) + + new ClusterMetadataResponse(coordinatorId, new Cluster(brokers)) + } +} + +class ClusterMetadataResponse(val coordinatorId : Int, val cluster : Cluster) Index: core/src/main/scala/kafka/api/StopFetcherResponse.scala =================================================================== --- core/src/main/scala/kafka/api/StopFetcherResponse.scala (revision 0) +++ core/src/main/scala/kafka/api/StopFetcherResponse.scala (revision 0) @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer +import kafka.utils.Utils + +object StopFetcherResponse { + def readFrom(buffer: ByteBuffer): StopFetcherResponse = { + val errorCode: Int = buffer.getShort + val consumer: String = Utils.readShortString(buffer, "UTF-8") + val numPartitions = buffer.getShort + var parts: List[PartitionAndOffset] = Nil + for (i <- 0 until numPartitions) parts ::= PartitionAndOffset.readFrom(buffer) + + new StopFetcherResponse(consumer, numPartitions, parts, errorCode) + } +} + +class StopFetcherResponse(val consumer: String, + val numPartitions: Short, + val parts: List[PartitionAndOffset], + val errorCode: Int) + Index: core/src/main/scala/kafka/api/StartFetcherRequest.scala =================================================================== --- core/src/main/scala/kafka/api/StartFetcherRequest.scala (revision 0) +++ core/src/main/scala/kafka/api/StartFetcherRequest.scala (revision 0) @@ -0,0 +1,148 @@ +/** + * 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 java.nio._ +import kafka.network.Request +import kafka.utils._ +import kafka.cluster.{Broker, Cluster} + +object StartFetcherRequest { + + def readFrom(buffer: ByteBuffer): StartFetcherRequest = { + val consumerStreamId = Utils.readShortString(buffer, "UTF-8") + val numThreads = buffer.getShort + var threads : List[ConsumerPartitionOwnershipInfo] = Nil + + for (i <- 0 until numThreads) { + threads ::= ConsumerPartitionOwnershipInfo.readFrom(buffer) + } + + val numBrokers = buffer.getShort + + var brokers: List[Broker] = Nil + for (i <- 1 to numBrokers) brokers ::= Broker.readFrom(buffer) + + new StartFetcherRequest(consumerStreamId, threads, new Cluster(brokers)) + } +} + +class StartFetcherRequest(val consumerStreamId: String, + val ownership : List[ConsumerPartitionOwnershipInfo], + val cluster : Cluster) + extends Request(RequestKeys.StartFetcher) { + + def writeTo(buffer: ByteBuffer) = { + Utils.writeShortString(buffer, consumerStreamId, "UTF-8") + buffer.putShort(ownership.size.asInstanceOf[Short]) + + for (i <- 0 until ownership.size) + ownership(i).writeTo(buffer) + + buffer.putShort(cluster.size.asInstanceOf[Short]) + cluster.writeTo(buffer) + } + + def sizeInBytes(): Int = { + var size: Int = 2 + consumerStreamId.size /* Consumer ID */ + + 2 /* Number of Partitions */ + + // Ownership info + for (i <- 0 until ownership.size) + size += ownership(i).sizeInBytes + // Cluster metadata + size += 2 /* Number of Brokers*/ + size += cluster.sizeInBytes + + size + } + + override def toString(): String = { + val startFetcherString = new StringBuilder + startFetcherString.append("StartFetcherRequest(consumerId: " + consumerStreamId) + startFetcherString.append(", numThreads: " + ownership.size) + startFetcherString.append(", Ownership: (") + + for (i <- 0 until ownership.size) { + if (i > 0) startFetcherString.append(", ") + startFetcherString.append(ownership(i).toString) + } + + startFetcherString.append("), ") + startFetcherString.append(cluster.toString) + startFetcherString.append(")") + + startFetcherString.toString() + } +} + + +object ConsumerPartitionOwnershipInfo { + def readFrom(buffer: ByteBuffer): ConsumerPartitionOwnershipInfo = { + val threadId = Utils.readShortString(buffer, "UTF-8") + val numPartitions = buffer.getShort + + var parts : List[PartitionAndOffset] = Nil + + for (i <- 0 until numPartitions) { + parts ::= PartitionAndOffset.readFrom(buffer) + } + + new ConsumerPartitionOwnershipInfo(threadId, parts) + } +} + +/* + * A list of PartitionAndOffset that is assigned to one consumer thread + */ +class ConsumerPartitionOwnershipInfo(val threadId: String, + val parts: List[PartitionAndOffset]) { + + def writeTo(buffer: ByteBuffer) = { + Utils.writeShortString(buffer, threadId, "UTF-8") + buffer.putShort(parts.size.asInstanceOf[Short]) + + for (i <- 0 until parts.size) + parts(i).writeTo(buffer) + } + + def sizeInBytes: Int = { + var size: Int = 2 + threadId.size /* Thread ID */ + + 2 /* Number of Partitions */ + for (i <- 0 until parts.size) + size += parts(i).sizeInBytes + + size + } + + override def toString(): String = { + val startFetcherString = new StringBuilder + startFetcherString.append("ConsumerPartitionOwnershipInfo(thread:" + threadId) + startFetcherString.append(", partitionOffsets: (") + + for (i <- 0 until parts.size) { + if (i > 0) startFetcherString.append(", ") + startFetcherString.append(parts(i).toString) + } + + startFetcherString.append("))") + + startFetcherString.toString() + } +} + Index: core/src/main/scala/kafka/api/HeartbeatRequest.scala =================================================================== --- core/src/main/scala/kafka/api/HeartbeatRequest.scala (revision 0) +++ core/src/main/scala/kafka/api/HeartbeatRequest.scala (revision 0) @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio._ +import kafka.network._ +import kafka.utils._ + +object HeartbeatRequest { + + def readFrom(buffer: ByteBuffer): HeartbeatRequest = { + val consumerId = Utils.readShortString(buffer, "UTF-8") + + new HeartbeatRequest(consumerId) + } +} + +/* + * This request is periodically sent by the coordinator to each consumers + * and an response is expected for checking the consumer's liveness + */ +class HeartbeatRequest(val consumerId: String) extends Request(RequestKeys.Heartbeat) { + + def writeTo(buffer: ByteBuffer) = { + Utils.writeShortString(buffer, consumerId, "UTF-8") + } + + def sizeInBytes(): Int = 2 + consumerId.size /* Consumer ID */ + + override def toString(): String= "HeartbeatRequest(consumer:" + consumerId + ")" +} Index: core/src/main/scala/kafka/api/RegisterConsumerRequest.scala =================================================================== --- core/src/main/scala/kafka/api/RegisterConsumerRequest.scala (revision 0) +++ core/src/main/scala/kafka/api/RegisterConsumerRequest.scala (revision 0) @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio._ +import channels.SelectionKey +import kafka.network._ +import kafka.utils._ +import kafka.consumer.TopicCount +import java.util.concurrent.ScheduledFuture +import kafka.utils.ZkUtils._ +import org.I0Itec.zkclient.ZkClient +import collection.mutable +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.locks.ReentrantLock + +object RegisterConsumerRequest { + + def readFrom(buffer: ByteBuffer): RegisterConsumerRequest = { + val consumerId = Utils.readShortString(buffer, "UTF-8") + val topicCountString = Utils.readShortString(buffer, "UTF-8") + val autoOffsetReset = Utils.readShortString(buffer, "UTF-8") + val autoOffsetCommit = Utils.readBoolean(buffer) + val sessionTimeoutMs = buffer.getInt + + val topicCount = TopicCount.constructTopicCount(consumerId, topicCountString) + + new RegisterConsumerRequest(consumerId, topicCount, autoOffsetCommit, autoOffsetReset, sessionTimeoutMs) + } +} + +class RegisterConsumerRequest(val consumerId : String, + val topicCount : TopicCount, + val autoOffsetCommit : Boolean, + val autoOffsetReset : String, + val sessionTimeoutMs : Int) + extends Request(RequestKeys.RegisterConsumer) { + + val topicCountString = topicCount.dbString + + def writeTo(buffer: ByteBuffer) = { + Utils.writeShortString(buffer, consumerId, "UTF-8") + Utils.writeShortString(buffer, topicCountString, "UTF-8") + Utils.writeShortString(buffer, autoOffsetReset, "UTF-8") + + // Encode boolean for buffer writing + Utils.writeBoolean(buffer, autoOffsetCommit) + + buffer.putInt(sessionTimeoutMs) + } + + def sizeInBytes(): Int = { + + 2 + consumerId.size /* consumerId */ + + 2 + topicCountString.size /* topicCount */ + + 2 + autoOffsetReset.size /* autoOffsetReset */ + + 1 /* autoOffsetCommit */ + + 4 /* sessionTimeoutMs */ + } + + + override def toString(): String= "RegisterConsumerRequest(consumer:" + consumerId + + ", topicCount:" + topicCountString + + ", autoOffsetReset:" + autoOffsetReset + + ", autoOffsetCommit:" + autoOffsetCommit + + ", sessionTimeoutMs:" + sessionTimeoutMs + ")" +} + Index: core/src/main/scala/kafka/api/StartFetcherResponse.scala =================================================================== --- core/src/main/scala/kafka/api/StartFetcherResponse.scala (revision 0) +++ core/src/main/scala/kafka/api/StartFetcherResponse.scala (revision 0) @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer +import kafka.utils.Utils + + +object StartFetcherResponse { + def readFrom(buffer: ByteBuffer): StartFetcherResponse = { + val errorCode: Int = buffer.getShort + val consumer: String = Utils.readShortString(buffer, "UTF-8") + + if (buffer.getShort != 0.asInstanceOf[Short]) + throw new IllegalStateException("StartFetcherResponse have non-zero partition info") + + new StartFetcherResponse(consumer, errorCode) + } +} + + + +class StartFetcherResponse(val consumer: String, + val errorCode: Int) Index: core/src/main/scala/kafka/api/HeartbeatResponse.scala =================================================================== --- core/src/main/scala/kafka/api/HeartbeatResponse.scala (revision 0) +++ core/src/main/scala/kafka/api/HeartbeatResponse.scala (revision 0) @@ -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 java.nio.ByteBuffer +import kafka.utils.Utils + +/* + * One partition info (topic, partition, offset) + */ +object PartitionAndOffset { + def readFrom(buffer: ByteBuffer): PartitionAndOffset = { + val topic = Utils.readShortString(buffer, "UTF-8") + val partition = Utils.readShortString(buffer, "UTF-8") + val offset = buffer.getLong + + new PartitionAndOffset(topic, partition, offset) + } +} + +class PartitionAndOffset(val topic: String, + val partition: String, + var offset: Long) { + + def writeTo(buffer: ByteBuffer) = { + Utils.writeShortString(buffer, topic, "UTF-8") + Utils.writeShortString(buffer, partition, "UTF-8") + buffer.putLong(offset) + } + + def sizeInBytes: Int = { + 2 + topic.size /* topic */ + + 2 + partition.size /* partition */ + + 8 /* offset */ + } + + override def toString(): String = "PartitionAndOffset(topic:" + topic + + ", partition:" + partition + + ", offset:" + offset + ")" +} + +object HeartbeatResponse { + def readFrom(buffer: ByteBuffer): HeartbeatResponse = { + val errorCode: Int = buffer.getShort + val consumer: String = Utils.readShortString(buffer, "UTF-8") + val numPartitions = buffer.getShort + + var parts: List[PartitionAndOffset] = Nil + for (i <- 0 until numPartitions) parts ::= PartitionAndOffset.readFrom(buffer) + + new HeartbeatResponse(consumer, numPartitions, parts, errorCode) + } +} + +class HeartbeatResponse(val consumer: String, + val numPartitions: Short, + val parts: List[PartitionAndOffset], + val errorCode: Int) Index: core/src/main/scala/kafka/api/RequestKeys.scala =================================================================== --- core/src/main/scala/kafka/api/RequestKeys.scala (revision 1371523) +++ core/src/main/scala/kafka/api/RequestKeys.scala (working copy) @@ -23,4 +23,14 @@ val MultiFetch: Short = 2 val MultiProduce: Short = 3 val Offsets: Short = 4 + val ClusterMetadata: Short = 5 + + val RegisterConsumer: Short = 10 + val Heartbeat: Short = 11 + val StopFetcher: Short = 12 + val StartFetcher: Short = 13 + + def isCoordinatorRequest(key: Short): Boolean = { + key >= 10 + } } Index: core/src/main/scala/kafka/api/GroupRegistry.scala =================================================================== --- core/src/main/scala/kafka/api/GroupRegistry.scala (revision 0) +++ core/src/main/scala/kafka/api/GroupRegistry.scala (revision 0) @@ -0,0 +1,127 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import kafka.consumer.TopicCount +import java.nio.channels.SelectionKey +import java.util.concurrent.ScheduledFuture +import collection.mutable +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.locks.ReentrantLock +import org.I0Itec.zkclient.ZkClient +import kafka.utils.ZKGroupTopicDirs +import kafka.utils.ZkUtils._ + +/* Consumer registry contains metadata of a registered consumer in the coordinator, each group registry has a list of its member consumer registries: + + 1. The topic/count information for this consumer. + 2. A bit indicating whether auto offset commit is turned on by this consumer. + 3. A string indicating the offset reset policy specified by this consumer. + 4. The accepted session timeout value in ms for this consumer. + 5. The corresponding processor Id and selection key in the socket server for this consumer. + 6. The number of failed responses to heartbeat request by this consumer. + */ + +class ConsumerRegistry(val consumerId : String, + val topicCount : TopicCount, + val autoOffsetCommit : Boolean, + val autoOffsetReset : String, + val sessionTimeoutMs : Int, + val processorId : Int, + val selectionKey : SelectionKey) { + + var numFailedRequests: Int = 0 + + var heartbeatTask: ScheduledFuture[_] = null + + def this(request: RegisterConsumerRequest, processorId: Int, selectionKey: SelectionKey) = { + this(request.consumerId, + request.topicCount, + request.autoOffsetCommit, + request.autoOffsetReset, + request.sessionTimeoutMs, + processorId, + selectionKey) + } + + def registerHeartbeatTask(task: ScheduledFuture[_]) = { + heartbeatTask = task + } +} + +object GroupRegistry { + val None = -1 +} + +/* Group registry contains metadata of a registered group in the coordinator: + + 1. The current consumed offset for each topic/partition that is received from the heartbeat and stop-fetcher response. + 2. A bit indicating whether the current consumed offset has been updated since the last time it is committed to Zookeeper. + 3. A bit indicating if the group has been put into some rebalancer's work queues, it will be reset once it gets dequeued by the rebalancer. + 4. Id of the current rebalancer which is hanlding this group's reblance task, it will be reset once its rebalance has completed by the rebalancer. + If this Id is not "NoBody", when a new rebalance task is generated for the group, it must be put to the same rebalancer's queue to avoid concurrent processing of the same group. + 5. A list of the group's current member consumer registries + 6. The lock for waiting stop/start-fetcher responses during the rebalance procedure. + 7. A bit indicating if the current rebalance trials has already failed; it will be reset by the rebalancer at the start of the trial, and can be set by multiple threads upon consumer failure or request expiration. + */ +class GroupRegistry(val group: String) { + var offsetUpdated: Boolean = false + var rebalanceFailed: Boolean = false + var rebalanceInitiated: AtomicBoolean = new AtomicBoolean(false) + var currRebalancer: Int = GroupRegistry.None + val rebalanceLock = new ReentrantLock() + val rebalanceCondition = rebalanceLock.newCondition() + val memberRegistries = mutable.Map[String, ConsumerRegistry]() + val offsetPerTopic = mutable.HashMap[String, mutable.HashMap[String, Long]]() + + def updateOffsets(parts: List[PartitionAndOffset]) = { + synchronized { + for (part <- parts) { + if (!offsetPerTopic.contains(part.topic)) + offsetPerTopic.put(part.topic, mutable.HashMap[String, Long]()) + + offsetPerTopic(part.topic).put(part.partition, part.offset) + } + offsetUpdated = true + } + } + + def writeOffsetsToZK(zkClient: ZkClient) = { + synchronized { + if (offsetUpdated == true) { + for ((topic, parts) <- offsetPerTopic) { + val topicDirs = new ZKGroupTopicDirs(group, topic) + for ((part, offset) <- parts) { + try { + updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + part, + offset.toString) + } + catch { + case t: Throwable => + // log it and let it go + warn("exception during commitOffsets", t) + } + debug("Committed offset " + offset + " for topic " + topic + ":" + part) + } + } + offsetUpdated = false + } + } + } +} + Index: core/src/main/scala/kafka/api/ClusterMetadataRequest.scala =================================================================== --- core/src/main/scala/kafka/api/ClusterMetadataRequest.scala (revision 0) +++ core/src/main/scala/kafka/api/ClusterMetadataRequest.scala (revision 0) @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio._ +import kafka.network._ +import kafka.utils._ + +object ClusterMetadataRequest { + + def readFrom(buffer: ByteBuffer): ClusterMetadataRequest = { + new ClusterMetadataRequest() + } +} + +/* + * The first request a consumer sends to any of its known brokers upon startup + * to get the current server cluster metadata + */ +class ClusterMetadataRequest extends Request(RequestKeys.ClusterMetadata) { + + def writeTo(buffer: ByteBuffer) = {} // Do Nothing + + def sizeInBytes(): Int = 0 // No data + + override def toString(): String= "ClusterMetadataRequest" +} Index: core/src/main/scala/kafka/network/Handler.scala =================================================================== --- core/src/main/scala/kafka/network/Handler.scala (revision 1371523) +++ core/src/main/scala/kafka/network/Handler.scala (working copy) @@ -17,6 +17,8 @@ package kafka.network +import java.nio.channels.SelectionKey + private[kafka] object Handler { /** @@ -24,6 +26,11 @@ * transmission into an outgoing transmission */ type Handler = Receive => Option[Send] + + /** + * A coordinator request handler, which requires more metadata than Handler + */ + type CoordinatorHandler = (Receive, Processor, SelectionKey) => Option[Send] /** * A handler mapping finds the right Handler function for a given request Index: core/src/main/scala/kafka/network/SocketServer.scala =================================================================== --- core/src/main/scala/kafka/network/SocketServer.scala (revision 1371523) +++ core/src/main/scala/kafka/network/SocketServer.scala (working copy) @@ -27,6 +27,7 @@ import org.apache.log4j.Logger import kafka.api.RequestKeys +import kafka.server.{RegistryReplySend, KafkaRequestHandlers} /** * An NIO socket server. The thread model is @@ -36,14 +37,14 @@ class SocketServer(val port: Int, val numProcessorThreads: Int, monitoringPeriodSecs: Int, - private val handlerFactory: Handler.HandlerMapping, + private val handlerFactory: KafkaRequestHandlers, val sendBufferSize: Int, val receiveBufferSize: Int, val maxRequestSize: Int = Int.MaxValue) { private val time = SystemTime private val processors = new Array[Processor](numProcessorThreads) - private var acceptor: Acceptor = new Acceptor(port, processors, sendBufferSize, receiveBufferSize) + private val acceptor: Acceptor = new Acceptor(port, processors, sendBufferSize, receiveBufferSize) val stats: SocketServerStats = new SocketServerStats(1000L * 1000L * 1000L * monitoringPeriodSecs) /** @@ -51,7 +52,7 @@ */ def startup() { for(i <- 0 until numProcessorThreads) { - processors(i) = new Processor(handlerFactory, time, stats, maxRequestSize) + processors(i) = new Processor(i, handlerFactory, time, stats, maxRequestSize) Utils.newThread("kafka-processor-" + i, processors(i), false).start() } Utils.newThread("kafka-acceptor", acceptor, false).start() @@ -67,6 +68,18 @@ processor.shutdown } + /** + * Send a request through a socket server + */ + def send(id: Int, key: SelectionKey, request: Request) { + val send = new BoundedByteBufferSend(request) + processors(id).prepareWrite(key, send) + } + + /** + * Close a specified socket if it exists + */ + def close(id: Int, key: SelectionKey) = processors(id).close(key) } /** @@ -186,7 +199,8 @@ * Thread that processes all requests from a single connection. There are N of these running in parallel * each of which has its own selectors */ -private[kafka] class Processor(val handlerMapping: Handler.HandlerMapping, +private[kafka] class Processor(val id: Int, + val handlerMapping: KafkaRequestHandlers, val time: Time, val stats: SocketServerStats, val maxRequestSize: Int) extends AbstractServerThread { @@ -220,13 +234,14 @@ throw new IllegalStateException("Unrecognized key state for processor thread.") } catch { case e: EOFException => { - logger.info("Closing socket connection to %s.".format(channelFor(key).socket.getInetAddress)) + logger.trace("Closing socket connection to %s due to EOF error %s.".format(channelFor(key).socket.getInetAddress, e.getMessage)) close(key) - } - case e: InvalidRequestException => { - logger.info("Closing socket connection to %s due to invalid request: %s".format(channelFor(key).socket.getInetAddress, e.getMessage)) - close(key) - } case e: Throwable => { + } + case e: InvalidRequestException => { + logger.trace("Closing socket connection to %s due to invalid request: %s".format(channelFor(key).socket.getInetAddress, e.getMessage)) + close(key) + } + case e: Throwable => { logger.error("Closing socket for " + channelFor(key).socket.getInetAddress + " because of error", e) close(key) } @@ -239,7 +254,7 @@ shutdownComplete() } - private def close(key: SelectionKey) { + def close(key: SelectionKey) { val channel = key.channel.asInstanceOf[SocketChannel] if(logger.isDebugEnabled) logger.debug("Closing connection from " + channel.socket.getRemoteSocketAddress()) @@ -286,14 +301,42 @@ requestLogger.trace("Handling multi-produce request from " + channelFor(key).socket.getRemoteSocketAddress()) case RequestKeys.Offsets => requestLogger.trace("Handling offset request from " + channelFor(key).socket.getRemoteSocketAddress()) + case RequestKeys.ClusterMetadata => + requestLogger.trace("Handling cluster metadata request from " + channelFor(key).socket.getRemoteSocketAddress()) + case RequestKeys.RegisterConsumer => + requestLogger.trace("Handling register consumer request from " + channelFor(key).socket.getRemoteSocketAddress()) + case RequestKeys.Heartbeat => + requestLogger.trace("Handling heartbeat response from " + channelFor(key).socket.getRemoteSocketAddress()) + case RequestKeys.StopFetcher => + requestLogger.trace("Handling stop fetcher response from " + channelFor(key).socket.getRemoteSocketAddress()) + case RequestKeys.StartFetcher => + requestLogger.trace("Handling start fetcher response from " + channelFor(key).socket.getRemoteSocketAddress()) case _ => throw new InvalidRequestException("No mapping found for handler id " + requestTypeId) } } - val handler = handlerMapping(requestTypeId, request) - if(handler == null) - throw new InvalidRequestException("No handler found for request") - val start = time.nanoseconds - val maybeSend = handler(request) + + // Here we have to hack the socket server a little bit since the request + // does not have the key information for the coordinator to store the map + // + // In 0.8 socket server this is unnecessary since the key info is included + // in the request + var start: Long = -1 + var maybeSend: Option[Send] = null + if (RequestKeys.isCoordinatorRequest(requestTypeId)) { + val handler = handlerMapping.coordinatorHandleFor(requestTypeId, request) + if(handler == null) + throw new InvalidRequestException("No handler found for request") + start = time.nanoseconds + maybeSend = handler(request, this, key) + } + else { + val handler = handlerMapping.handlerFor(requestTypeId, request) + if(handler == null) + throw new InvalidRequestException("No handler found for request") + start = time.nanoseconds + maybeSend = handler(request) + } + stats.recordRequest(requestTypeId, time.nanoseconds - start) maybeSend } @@ -316,8 +359,13 @@ close(key) return } else if(request.complete) { + /** + * We first clear the attachment then try to handle the complete request since the handling could enable some other + * threads trying to write to it, for example consider the heartbeat scheduler; in the future we may need to add some synchronization + * primitives to handle the case of interleaving read and write + */ + key.attach(null) val maybeResponse = handle(key, request) - key.attach(null) // if there is a response, send it, otherwise do nothing if(maybeResponse.isDefined) { key.attach(maybeResponse.getOrElse(None)) @@ -349,6 +397,21 @@ } } + /* + * Prepare writes, and make the socket ready + */ + def prepareWrite(key: SelectionKey, request: Send) { + // TODO: If the channel already has some un-handled data while trying to write to it, + // simply do not write but let the coordinator to detect failures later. + // This would be fixed when extending SocketServer to allow concurrent read/write after 0.8 + if (key.attachment() == null) { + key.attach(request) + key.interestOps(SelectionKey.OP_WRITE) + selector.wakeup() + } +// throw new IllegalStateException("The channel has some un-handled data while trying to write to it") + } + private def channelFor(key: SelectionKey) = key.channel().asInstanceOf[SocketChannel] } Index: core/src/main/scala/kafka/network/BlockingChannel.scala =================================================================== --- core/src/main/scala/kafka/network/BlockingChannel.scala (revision 0) +++ core/src/main/scala/kafka/network/BlockingChannel.scala (revision 0) @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.network + +import java.net.InetSocketAddress +import java.nio.channels._ +import kafka.utils.{nonthreadsafe, Logging} + + +object BlockingChannel{ + val UseDefaultBufferSize = -1 +} + +/** + * A simple blocking channel with timeouts correctly enabled. + * + */ +@nonthreadsafe +class BlockingChannel( val host: String, + val port: Int, + val readBufferSize: Int, + val writeBufferSize: Int, + val readTimeoutMs: Int ) extends Logging { + private var connected = false + private var channel: SocketChannel = null + private var readChannel: ReadableByteChannel = null + private var writeChannel: GatheringByteChannel = null + private val lock = new Object() + + def connect() = lock synchronized { + if(!connected) { + channel = SocketChannel.open() + if(readBufferSize > 0) + channel.socket.setReceiveBufferSize(readBufferSize) + if(writeBufferSize > 0) + channel.socket.setSendBufferSize(writeBufferSize) + channel.configureBlocking(true) + channel.socket.setSoTimeout(readTimeoutMs) + channel.socket.setKeepAlive(true) + channel.socket.setTcpNoDelay(true) + channel.connect(new InetSocketAddress(host, port)) + + writeChannel = channel + readChannel = Channels.newChannel(channel.socket().getInputStream) + connected = true + } + } + + def disconnect() = lock synchronized { + if(connected || channel != null) { + // closing the main socket channel *should* close the read channel + // but let's do it to be sure. + swallow(channel.close()) + swallow(channel.socket.close()) + swallow(readChannel.close()) + channel = null; readChannel = null; writeChannel = null + connected = false + } + } + + def isConnected = connected + + // TODO: Better be RequestOrResponse to be compatible with 0.8 + def send(request: Request): Int = { + if(!connected) + throw new ClosedChannelException() + + val send = new BoundedByteBufferSend(request) + send.writeCompletely(writeChannel) + } + + def send(buffer: Send) : Int = buffer.writeCompletely(writeChannel) + + def receive(): Receive = { + if(!connected) + throw new ClosedChannelException() + + val response = new BoundedByteBufferReceive() + response.readCompletely(readChannel) + + response + } + +} \ No newline at end of file Index: core/src/main/scala/kafka/network/BoundedByteBufferSend.scala =================================================================== --- core/src/main/scala/kafka/network/BoundedByteBufferSend.scala (revision 1371523) +++ core/src/main/scala/kafka/network/BoundedByteBufferSend.scala (working copy) @@ -43,7 +43,7 @@ request.writeTo(buffer) buffer.rewind() } - + def writeTo(channel: GatheringByteChannel): Int = { expectIncomplete() var written = channel.write(Array(sizeBuffer, buffer)) Index: core/src/main/scala/kafka/network/NonBlockingChannel.scala =================================================================== --- core/src/main/scala/kafka/network/NonBlockingChannel.scala (revision 0) +++ core/src/main/scala/kafka/network/NonBlockingChannel.scala (revision 0) @@ -0,0 +1,160 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.network + +import java.net.InetSocketAddress +import java.nio.channels._ +import kafka.utils.{nonthreadsafe, Logging} +import java.io.EOFException + + +object NonBlockingChannel{ + val UseDefaultBufferSize = -1 +} + +/** + * A simple blocking channel with timeouts correctly enabled. + * + */ +@nonthreadsafe +class NonBlockingChannel( val host: String, + val port: Int, + val readBufferSize: Int, + val writeBufferSize: Int, + val maxRequestSize: Int, + val readTimeoutMs: Int ) extends Logging { + private var connected = false + private var channel: SocketChannel = null + private var selector: Selector = null + private var key: SelectionKey = null + private val lock = new Object() + + def connect() = lock synchronized { + if(!connected) { + channel = SocketChannel.open() + if(readBufferSize > 0) + channel.socket.setReceiveBufferSize(readBufferSize) + if(writeBufferSize > 0) + channel.socket.setSendBufferSize(writeBufferSize) + channel.configureBlocking(false) + channel.socket.setKeepAlive(true) + channel.socket.setTcpNoDelay(true) + channel.connect(new InetSocketAddress(host, port)) + + while(!channel.finishConnect()) {} + + selector = Selector.open() + key = channel.register(selector, SelectionKey.OP_READ) + + connected = true + } + } + + def disconnect() = lock synchronized { + if(connected || channel != null) { + // closing the main socket channel *should* close the read channel + // but let's do it to be sure. + swallow(channel.close()) + swallow(channel.socket.close()) + channel = null + connected = false + } + } + + def isConnected = connected + + def send(request: Request): Int = { + val buffer = new BoundedByteBufferSend(request) + send(buffer) + } + + // TODO: Better be RequestOrResponse to be compatible with 0.8 + def send(buffer: Send): Int = { + if(!connected) + throw new ClosedChannelException() + + // There is still incomplete read-in data, first complete the read then write + if (key.attachment != null) { + val request = key.attachment.asInstanceOf[Receive] + while(!request.complete) + request.readFrom(channel) + } + + var written: Int = 0 + while(!buffer.complete) { + written += buffer.writeTo(channel) + } + + written + } + + /** + * Immediately send by interrupting the reading selector + */ + def sendNow(buffer: Send): Int = { + selector.wakeup() + send(buffer) + } + + def receive(): Receive = { + if(!connected) + throw new ClosedChannelException() + + var request = key.attachment.asInstanceOf[Receive] + if (request != null && request.complete) { + key.interestOps(SelectionKey.OP_READ) + key.attach(null) + request + } + else { + val ready = selector.select(readTimeoutMs) + + if(ready == 1) { + val iter = selector.selectedKeys().iterator() + if (key != iter.next()) + throw new IllegalStateException("Non blocking channel has unrecognized key") + iter.remove() + + if(key.isReadable) { + if(request == null) { + request = new BoundedByteBufferReceive(maxRequestSize) + key.attach(request) + } + val read = request.readFrom(channel) + key.interestOps(SelectionKey.OP_READ) + + if(read < 0) { + throw new java.io.IOException("Reading request from non blocking channel failed") + } else if(request.complete) { + key.attach(null) + request + } else { + // more reading to be done + null + } + } + else { + throw new IllegalStateException("Non blocking channel's key is not readable") + } + } + else { + throw new java.net.SocketTimeoutException("Non blocking channel's selector has timeout expired") + } + } + } +} \ No newline at end of file Index: core/src/main/scala/kafka/consumer/ConsumerOffsetsSend.scala =================================================================== --- core/src/main/scala/kafka/consumer/ConsumerOffsetsSend.scala (revision 0) +++ core/src/main/scala/kafka/consumer/ConsumerOffsetsSend.scala (revision 0) @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.consumer + +import java.nio._ +import java.nio.channels._ +import kafka.network._ +import kafka.utils._ +import kafka.common.ErrorMapping +import kafka.cluster.{Partition, Cluster} +import scala.Array +import kafka.api.PartitionAndOffset + +/** + * A zero-copy message response that writes the bytes needed directly from the file + * wholly in kernel space + */ +@nonthreadsafe +class ConsumerOffsetsSend(val consumer: String, + val responseId: Short, + val topicRegistry: Pool[String, Pool[Partition, PartitionTopicInfo]], + val errorCode: Int) extends Send { + + private var offsets : List[PartitionAndOffset] = Nil + private var size: Int = 2 /* response type id */ + 2 /* error code */ + 2 + consumer.size /* consumer string */ + 2 /* number of partitions */ + + if (topicRegistry != null) { + for (parts <- topicRegistry.values) { + for (part <- parts) { + val partitionOffset = new PartitionAndOffset(part._2.topic, part._2.partition.toString, part._2.getConsumeOffset) + offsets ::= partitionOffset + size += partitionOffset.sizeInBytes + } + } + } + + private val header = ByteBuffer.allocate(4) + private val content = ByteBuffer.allocate(size) + + header.putInt(size) + header.rewind() + + content.putShort(responseId) + content.putShort(errorCode.asInstanceOf[Short]) + Utils.writeShortString(content, consumer, "UTF-8") + content.putShort(offsets.size.asInstanceOf[Short]) + + if (topicRegistry != null) { + for (partOffset <- offsets) partOffset.writeTo(content) + } + + content.rewind() + + var complete: Boolean = false + + def writeTo(channel: GatheringByteChannel): Int = { + expectIncomplete() + val written = channel.write(Array(header, content)) + // if we are done, mark it off + if(!content.hasRemaining) + complete = true + written.asInstanceOf[Int] + } + +} Index: core/src/main/scala/kafka/consumer/ConsumerIterator.scala.orig =================================================================== --- core/src/main/scala/kafka/consumer/ConsumerIterator.scala.orig (revision 0) +++ core/src/main/scala/kafka/consumer/ConsumerIterator.scala.orig (revision 0) @@ -0,0 +1,100 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.consumer + +import kafka.utils.{IteratorTemplate, Logging} +import java.util.concurrent.{TimeUnit, BlockingQueue} +import kafka.serializer.Decoder +import java.util.concurrent.atomic.AtomicReference +import kafka.message.{MessageAndOffset, MessageAndMetadata} + + +/** + * An iterator that blocks until a value can be read from the supplied queue. + * The iterator takes a shutdownCommand object which can be added to the queue to trigger a shutdown + * + */ +class ConsumerIterator[T](private val channel: BlockingQueue[FetchedDataChunk], + consumerTimeoutMs: Int, + private val decoder: Decoder[T], + val enableShallowIterator: Boolean) + extends IteratorTemplate[MessageAndMetadata[T]] with Logging { + + private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null) + private var currentTopicInfo:PartitionTopicInfo = null + private var consumedOffset: Long = -1L + + override def next(): MessageAndMetadata[T] = { + val item = super.next() + if(consumedOffset < 0) + throw new IllegalStateException("Offset returned by the message set is invalid %d".format(consumedOffset)) + currentTopicInfo.resetConsumeOffset(consumedOffset) + val topic = currentTopicInfo.topic + trace("Setting %s consumed offset to %d".format(topic, consumedOffset)) + ConsumerTopicStat.getConsumerTopicStat(topic).recordMessagesPerTopic(1) + ConsumerTopicStat.getConsumerAllTopicStat().recordMessagesPerTopic(1) + item + } + + protected def makeNext(): MessageAndMetadata[T] = { + var currentDataChunk: FetchedDataChunk = null + // if we don't have an iterator, get one + var localCurrent = current.get() + if(localCurrent == null || !localCurrent.hasNext) { + if (consumerTimeoutMs < 0) + currentDataChunk = channel.take + else { + currentDataChunk = channel.poll(consumerTimeoutMs, TimeUnit.MILLISECONDS) + if (currentDataChunk == null) { + // reset state to make the iterator re-iterable + resetState() + throw new ConsumerTimeoutException + } + } + if(currentDataChunk eq ZookeeperConsumerConnector.shutdownCommand) { + debug("Received the shutdown command") + channel.offer(currentDataChunk) + return allDone + } else { + currentTopicInfo = currentDataChunk.topicInfo + if (currentTopicInfo.getConsumeOffset != currentDataChunk.fetchOffset) { + error("consumed offset: %d doesn't match fetch offset: %d for %s;\n Consumer may lose data" + .format(currentTopicInfo.getConsumeOffset, currentDataChunk.fetchOffset, currentTopicInfo)) + currentTopicInfo.resetConsumeOffset(currentDataChunk.fetchOffset) + } + localCurrent = if (enableShallowIterator) currentDataChunk.messages.shallowIterator + else currentDataChunk.messages.iterator + current.set(localCurrent) + } + } + val item = localCurrent.next() + consumedOffset = item.offset + + new MessageAndMetadata(decoder.toEvent(item.message), currentTopicInfo.topic) + } + + def clearCurrentChunk() { + try { + info("Clearing the current data chunk for this consumer iterator") + current.set(null) + } + } +} + +class ConsumerTimeoutException() extends RuntimeException() + Index: core/src/main/scala/kafka/consumer/ConsumerIterator.scala =================================================================== --- core/src/main/scala/kafka/consumer/ConsumerIterator.scala (revision 1371523) +++ core/src/main/scala/kafka/consumer/ConsumerIterator.scala (working copy) @@ -66,7 +66,7 @@ throw new ConsumerTimeoutException } } - if(currentDataChunk eq ZookeeperConsumerConnector.shutdownCommand) { + if(currentDataChunk eq SimpleConsumerConnector.shutdownCommand) { debug("Received the shutdown command") channel.offer(currentDataChunk) return allDone Index: core/src/main/scala/kafka/consumer/ConsumerConnector.scala =================================================================== --- core/src/main/scala/kafka/consumer/ConsumerConnector.scala (revision 1371523) +++ core/src/main/scala/kafka/consumer/ConsumerConnector.scala (working copy) @@ -73,7 +73,7 @@ * connection string zk.connect. */ def create(config: ConsumerConfig): ConsumerConnector = { - val consumerConnect = new ZookeeperConsumerConnector(config) + val consumerConnect = new SimpleConsumerConnector(config) Utils.registerMBean(consumerConnect, consumerStatsMBeanName) consumerConnect } @@ -85,7 +85,7 @@ * connection string zk.connect. */ def createJavaConsumerConnector(config: ConsumerConfig): kafka.javaapi.consumer.ConsumerConnector = { - val consumerConnect = new kafka.javaapi.consumer.ZookeeperConsumerConnector(config) + val consumerConnect = new kafka.javaapi.consumer.SimpleConsumerConnector(config) Utils.registerMBean(consumerConnect.underlying, consumerStatsMBeanName) consumerConnect } Index: core/src/main/scala/kafka/consumer/TopicAndConsumerStreamId.scala =================================================================== --- core/src/main/scala/kafka/consumer/TopicAndConsumerStreamId.scala (revision 0) +++ core/src/main/scala/kafka/consumer/TopicAndConsumerStreamId.scala (revision 0) @@ -0,0 +1,20 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.consumer + +case class TopicAndConsumerStreamId(topic: String, ConsumerId: String) \ No newline at end of file Index: core/src/main/scala/kafka/consumer/FetcherRunnable.scala =================================================================== --- core/src/main/scala/kafka/consumer/FetcherRunnable.scala (revision 1371523) +++ core/src/main/scala/kafka/consumer/FetcherRunnable.scala (working copy) @@ -26,7 +26,6 @@ import java.io.IOException class FetcherRunnable(val name: String, - val zkClient : ZkClient, val config: ConsumerConfig, val broker: Broker, val partitionTopicInfos: List[PartitionTopicInfo]) @@ -37,6 +36,8 @@ @volatile private var stopped = false + def isShutdown: Boolean = stopped + def shutdown(): Unit = { stopped = true interrupt @@ -58,7 +59,7 @@ trace("fetch request: " + fetches.toString) val response = simpleConsumer.multifetch(fetches : _*) - trace("recevied response from fetch request: " + fetches.toString) + trace("received response of " + response.numSets + " message sets from fetch request") var read = 0L @@ -66,17 +67,15 @@ try { var done = false if(messages.getErrorCode == ErrorMapping.OffsetOutOfRangeCode) { - info("offset for " + infopti + " out of range") - // see if we can fix this error - val resetOffset = resetConsumerOffsets(infopti.topic, infopti.partition) - if(resetOffset >= 0) { - infopti.resetFetchOffset(resetOffset) - infopti.resetConsumeOffset(resetOffset) - done = true - } + info("offset for " + infopti + " out of range, will be terminating itself") + + // Stop itself to let the coordinator connector know, so that it will reconnect to the coordinator + stopped = true + done = true } if (!done) read += infopti.enqueue(messages, infopti.getFetchOffset) + } catch { case e1: IOException => @@ -118,6 +117,8 @@ */ private def shutdownComplete() = shutdownLatch.countDown + /** + private def resetConsumerOffsets(topic : String, partition: Partition) : Long = { var offset : Long = 0 @@ -133,9 +134,11 @@ // reset manually in zookeeper info("updating partition " + partition.name + " for topic " + topic + " with " + - (if(offset == OffsetRequest.EarliestTime) "earliest " else " latest ") + "offset " + offsets(0)) + (if(offset == OffsetRequest.EarliestTime) "earliest " else " latest ") + "offset " + offsets(0)) ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + partition.name, offsets(0).toString) offsets(0) } + + */ } Index: core/src/main/scala/kafka/consumer/ConsumerCoordinatorConnector.scala =================================================================== --- core/src/main/scala/kafka/consumer/ConsumerCoordinatorConnector.scala (revision 0) +++ core/src/main/scala/kafka/consumer/ConsumerCoordinatorConnector.scala (revision 0) @@ -0,0 +1,244 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.consumer + +import kafka.api._ +import kafka.utils._ +import kafka.network._ +import kafka.common.ErrorMapping +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.CountDownLatch + + +class ConsumerCoordinatorConnector(val name: String, + val consumerConnector: SimpleConsumerConnector) + extends Thread(name) with Logging { + + var fetcherStopped: Boolean = true + + var coordinatorChannel: NonBlockingChannel = null + + val coordinatorRequestHandler: CoordinatorRequestHandler = new CoordinatorRequestHandler(this, consumerConnector) + + /* If automatic offset commit is turned on, When the shutdown signal is initiated by the consumer (i.e. shutdownInitiaited == true), + the connector cannot shutdown immediately and has to wait for: + 1. The next heartbeat to be received and replied, after which it sets lastHeartbeatReplied = true + 2. The next heartbeat to be received (meaning that the previous heartbeat response has been replied), + after which it can then set canShutdown = true + Only when canShutdown == true then the connector can shutdown + */ + val shutdownInitiated: AtomicBoolean = new AtomicBoolean(false) + val lastHeartbeatReplied: AtomicBoolean = new AtomicBoolean(false) + val canShutdown: AtomicBoolean = new AtomicBoolean(false) + private val shutdownLatch = new CountDownLatch(1) + + // TODO: Used for unit test only, for simulating virtual failure such as long GC + // Will be removed eventually + val isBlocked: AtomicBoolean = new AtomicBoolean(false) + + def shutdown() { + shutdownInitiated.set(true) + //interrupt() + shutdownLatch.await() + info("Coordinator connector shutdown completed") + } + + /* Repeatedly trying to get the next request from the coordinator, handles it and sends by the response; + also handles situations when the coordinator is suspected to be failed + */ + override def run() { + // Try to connect the register with the coordinator + // If register fails, try to reconnect again + while (!shutdownInitiated.get() && !registerWithCoordinator) + handleLostConnection + + try { + while (!canShutdown.get()) { + // Try receiving the next response from the coordinator + try { + val receive = coordinatorChannel.receive + + if (receive != null) { + // Verify that there is no error; if yes, return the cluster metadata + val requestTypeId = receive.buffer.getShort + val response: Send = coordinatorRequestHandler.handle(requestTypeId, receive) + + coordinatorChannel.send(response) + } + } catch { + case e: java.net.SocketTimeoutException => { + info("Did not receive any request from coordinator %s:%d within session timeout, try to reconnect to the new coordinator" + .format(coordinatorChannel.host, coordinatorChannel.port)) + + reconnectToCoordinator + } + case e: java.io.IOException => { + info("Cannot reach coordinator %s:%d anymore probably due broken pipe, try to reconnect to the new coordinator" + .format(coordinatorChannel.host, coordinatorChannel.port)) + + reconnectToCoordinator + } + case e: IllegalStateException => { + info(e.getMessage) + + reconnectToCoordinator + } + case e => throw e + } + + // TODO: for unite test only, simluating long GC + while(isBlocked.get()) {} + } + } catch { + case e: InterruptedException => info("Coordinator connector interrupted. Shutting down") + case e1 => error("Error in coordinator connector. Shutting down due to ", e1) + } + + if(coordinatorChannel != null) { + coordinatorChannel.disconnect() + coordinatorChannel = null + } + + shutdownComplete() + } + + private def registerWithCoordinator(): Boolean = { + // First consult known brokers to get metadata about the server cluster + if (!discoverConsumerCoordinator) return false + + val coordinator = consumerConnector.kafkaCluster.getBroker(consumerConnector.coordinatorId).get + debug("Connecting to the current coordinator at " + coordinator.getByteString) + + coordinatorChannel = new NonBlockingChannel( + coordinator.host, + coordinator.port, + consumerConnector.config.socketBufferSize, + consumerConnector.config.socketBufferSize, + consumerConnector.config.maxCoordinatorRequestSize, + consumerConnector.config.coordinatorSessionTimeoutMs) + + val request = new RegisterConsumerRequest( + consumerConnector.consumerIdString, + consumerConnector.topicCount, + consumerConnector.config.autoCommit, + consumerConnector.config.autoOffsetReset, + consumerConnector.config.coordinatorSessionTimeoutMs) + + var response: Receive = null + + var succeed = false + try { + coordinatorChannel.connect() + coordinatorChannel.send(request) + + while(response == null) { response = coordinatorChannel.receive() } + + // Verify that there is no error; if yes, return the cluster metadata + val error: Short = response.buffer.getShort + if (error != ErrorMapping.NoError) { + info("Received error while trying to register with the coordinator " + error) + } else { + info("Received valid reply from the coordinator: register succeeds") + + succeed = true + } + } catch { + case e : java.io.IOException => + info("Cannot reach broker %s due to socket error: ", e) + case e: java.io.IOException => + info("Cannot reach coordinator after timeout: ", e) + } + + if (!succeed) { + coordinatorChannel.disconnect + coordinatorChannel = null + } + + return succeed + } + + private def reconnectToCoordinator() = { + coordinatorChannel.disconnect + handleLostConnection + + while(!canShutdown.get && !registerWithCoordinator) + handleLostConnection + } + + private def discoverConsumerCoordinator(): Boolean = { + debug("Consulting known brokers for cluster and current coordinator info") + + for (i <- consumerConnector.kafkaCluster.getBrokerIds) { + val broker = consumerConnector.kafkaCluster.getBroker(i).get + debug("Start consulting broker " + broker.toString) + + // Consult broker for cluster metadata + val channel = new BlockingChannel(broker.host, broker.port, + consumerConnector.config.socketBufferSize, + consumerConnector.config.socketBufferSize, + consumerConnector.config.socketTimeoutMs) + + var response: Receive = null + val request = new ClusterMetadataRequest() + + var succeed = false + try { + channel.connect() + channel.send(request) + response = channel.receive() + + // Verify that there is no error; if yes, return the cluster metadata + val error: Short = response.buffer.getShort + + succeed = error match { + case ErrorMapping.NoError => + val metadata = ClusterMetadataResponse.readFrom(response.buffer) + consumerConnector.coordinatorId = metadata.coordinatorId + consumerConnector.kafkaCluster = metadata.cluster + info("Received cluster metadata with coordinator %d" + .format(consumerConnector.coordinatorId)) + + true + case ErrorMapping.UnknownCode => + info("Received cluster metadata has unknown error") + + false + } + } catch { + case e : java.io.IOException => + info("Cannot reach broker " + i + " due to socket error: ", e) + case e => throw e + } + + if (succeed) return true + } + + warn("Cannot reach any of the known brokers in the config list") + return false + } + + private def handleLostConnection() { + // First close all fetchers and clean their corresponding queues + fetcherStopped = true + + consumerConnector.closeAllFetchers(consumerConnector.kafkaCluster, consumerConnector.messageStreamsPerTopic) + + Thread.sleep(consumerConnector.config.fetcherBackoffMs) + } + + private def shutdownComplete() = shutdownLatch.countDown +} Index: core/src/main/scala/kafka/consumer/ConsoleConsumer.scala =================================================================== --- core/src/main/scala/kafka/consumer/ConsoleConsumer.scala (revision 1371523) +++ core/src/main/scala/kafka/consumer/ConsoleConsumer.scala (working copy) @@ -96,6 +96,11 @@ .ofType(classOf[java.lang.Integer]) val skipMessageOnErrorOpt = parser.accepts("skip-message-on-error", "If there is an error when processing a message, " + "skip it instead of halt.") + val initBrokerListOpt = parser.accepts("init-broker-list", "Initial broker list for bootstraping the server cluster metadata.") + .withRequiredArg + .describedAs("json") + .ofType(classOf[String]) + .defaultsTo("") val options: OptionSet = tryParse(parser, args) Utils.checkRequiredArgs(parser, options, zkConnectOpt) @@ -121,6 +126,7 @@ props.put("autooffset.reset", if(options.has(resetBeginningOpt)) "smallest" else "largest") props.put("zk.connect", options.valueOf(zkConnectOpt)) props.put("consumer.timeout.ms", options.valueOf(consumerTimeoutMsOpt).toString) + props.put("init.broker.list", options.valueOf(initBrokerListOpt).toString) val config = new ConsumerConfig(props) val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false @@ -155,6 +161,7 @@ try { for(messageAndTopic <- iter) { try { + // TODO: comment it out for latency test formatter.writeTo(messageAndTopic.message, System.out) } catch { case e => Index: core/src/main/scala/kafka/consumer/Fetcher.scala =================================================================== --- core/src/main/scala/kafka/consumer/Fetcher.scala (revision 1371523) +++ core/src/main/scala/kafka/consumer/Fetcher.scala (working copy) @@ -19,22 +19,37 @@ import scala.collection._ import kafka.cluster._ -import org.I0Itec.zkclient.ZkClient import java.util.concurrent.BlockingQueue import kafka.utils._ import java.lang.IllegalStateException +import org.I0Itec.zkclient.ZkClient /** * The fetcher is a background thread that fetches data from a set of servers */ -private [consumer] class Fetcher(val config: ConsumerConfig, val zkClient : ZkClient) extends Logging { +private [consumer] class Fetcher(val config: ConsumerConfig) extends Logging { private val EMPTY_FETCHER_THREADS = new Array[FetcherRunnable](0) @volatile private var fetcherThreads : Array[FetcherRunnable] = EMPTY_FETCHER_THREADS /** - * shutdown all fetcher threads + * Check liveness of fetcher threads */ + def checkLiveness(): Boolean = { + fetcherThreads match { + case EMPTY_FETCHER_THREADS => false + case fetchers => { + if (fetchers.foldLeft(0)((sum, fetcher) => sum + (if(fetcher.isShutdown) 1 else 0)) > 0) + false + else + true + } + } + } + + /** + * Shutdown all fetcher threads + */ def stopConnectionsToAllBrokers = { // shutdown the old fetcher threads, if any for (fetcherThread <- fetcherThreads) @@ -84,7 +99,7 @@ fetcherThreads = new Array[FetcherRunnable](brokers.size) var i = 0 for(broker <- brokers) { - val fetcherThread = new FetcherRunnable("FetchRunnable-" + i, zkClient, config, broker, m.get(broker.id).get) + val fetcherThread = new FetcherRunnable("FetchRunnable-" + i, config, broker, m.get(broker.id).get) fetcherThreads(i) = fetcherThread fetcherThread.start i +=1 Index: core/src/main/scala/kafka/consumer/SimpleConsumerConnector.scala =================================================================== --- core/src/main/scala/kafka/consumer/SimpleConsumerConnector.scala (revision 0) +++ core/src/main/scala/kafka/consumer/SimpleConsumerConnector.scala (revision 0) @@ -0,0 +1,371 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.consumer + +import kafka.api._ +import kafka.utils._ +import kafka.cluster._ +import kafka.utils.ZkUtils._ +import kafka.serializer.Decoder +import scala.collection._ +import java.util.concurrent._ +import java.util.concurrent.atomic._ +import java.net.InetAddress +import java.util.UUID +import java.lang.IllegalStateException +import util.parsing.json.JSON + + +private[kafka] object SimpleConsumerConnector { + val shutdownCommand: FetchedDataChunk = new FetchedDataChunk(null, null, -1L) +} + +/** + * JMX interface for monitoring consumer + */ +trait SimpleConsumerConnectorMBean { + def getPartOwnerStats: String + def getConsumerGroup: String + // TODO: we need to migrate these metrics to the coordinator when incorporating with 0.8 + // def getOffsetLag(topic: String, brokerId: Int, partitionId: Int): Long + // def getConsumedOffset(topic: String, brokerId: Int, partitionId: Int): Long + // def getLatestOffset(topic: String, brokerId: Int, partitionId: Int): Long +} + +private[kafka] class SimpleConsumerConnector(val config: ConsumerConfig, + val enableFetcher: Boolean) // for testing only + extends ConsumerConnector with SimpleConsumerConnectorMBean with Logging { + + private val isShuttingDown = new AtomicBoolean(false) + + private val messageStreamCreated = new AtomicBoolean(false) + + private var coordinatorConnector: ConsumerCoordinatorConnector = null + + private var fetcher: Option[Fetcher] = None + + val topicThreadIdAndQueues = new Pool[TopicAndConsumerStreamId, BlockingQueue[FetchedDataChunk]] + + val messageStreamsPerTopic: mutable.Map[String,List[KafkaStream[_]]] = new mutable.HashMap[String,List[KafkaStream[_]]] + + var topicRegistry = new Pool[String, Pool[Partition, PartitionTopicInfo]] + + var wildcardStreamsHandler: WildcardStreamsHandler[_] = null + + var topicCount: TopicCount = null + + var kafkaCluster: Cluster = null + + var coordinatorId: Int = -1 + + // Construct its uid + val consumerIdString = { + var consumerUuid : String = null + config.consumerId match { + case Some(consumerId) // for testing only + => consumerUuid = consumerId + case None // generate unique consumerId automatically + => val uuid = UUID.randomUUID() + consumerUuid = "%s-%d-%s".format( + InetAddress.getLocalHost.getHostName, System.currentTimeMillis, + uuid.getMostSignificantBits().toHexString.substring(0,8)) + } + config.groupId + "_" + consumerUuid + } + this.logIdent = consumerIdString + " " + + // Construct initial server cluster info + kafkaCluster = new Cluster() + JSON.parseFull(config.initBrokerList) match { + case Some(m) => + val brokerMap = m.asInstanceOf[Map[String, String]] + + for ((brokerId, addrs) <- brokerMap) { + // Consult broker for cluster metadata + val brokerAddrInfo = addrs.split(",") + kafkaCluster.add(new Broker(brokerId.toInt, "unknown", brokerAddrInfo(0), brokerAddrInfo(1).toInt)) + } + case None => throw new RuntimeException("Error parsing the init broker list : " + config.initBrokerList) + } + info("Read in initial servers " + kafkaCluster.toString) + + createFetcher + + // TODO: To be supported after 0.8, with modified SocketServer + def commitOffsets() {} + + def getOffsetsForTopic(topic: String): Seq[Tuple2[String,String]] = { + var partitionAndOffsets = List.empty[Tuple2[String,String]] + + topicRegistry.get(topic) match { + case null => { + throw new IllegalStateException("The topic is not registeredin ZK for the group offsets") + } + case infos => { + for (info <- infos.values) { + val newOffset = info.getConsumeOffset + partitionAndOffsets ::= (info.partition.name, newOffset.toString) + } + } + } + + partitionAndOffsets + } + + def this(config: ConsumerConfig) = this(config, true) + + def createMessageStreams[T](topicCountMap: Map[String,Int], decoder: Decoder[T]) = { + if (messageStreamCreated.getAndSet(true)) + throw new RuntimeException(this.getClass.getSimpleName + + " can create message streams at most once") + + debug("entering consume ") + if (topicCountMap == null) + throw new RuntimeException("topicCountMap is null") + + topicCount = TopicCount.constructTopicCount(consumerIdString, topicCountMap) + + // Create the coordinator connector thread + coordinatorConnector = new ConsumerCoordinatorConnector( + "kafka-coordinator-connector-", this) + coordinatorConnector.start + + // make a list of (queue,stream) pairs, one pair for each ConsumerId + val topicThreadIds = topicCount.getConsumerThreadIdsPerTopic + val queuesAndStreams = topicThreadIds.values.map(threadIdSet => + threadIdSet.map(_ => { + val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks) + val stream = new KafkaStream[T]( + queue, config.consumerTimeoutMs, decoder, config.enableShallowIterator) + (queue, stream) + }) + ).flatten.toList + + // Construct the mapping between streams and topic-threads + constructStreams(topicCount, queuesAndStreams) + + messageStreamsPerTopic.asInstanceOf[Map[String, List[KafkaStream[T]]]] + } + + def createMessageStreamsByFilter[T](topicFilter: TopicFilter, numStreams: Int, decoder: Decoder[T]) = { + if (messageStreamCreated.getAndSet(true)) + throw new RuntimeException(this.getClass.getSimpleName + + " can create message streams by filter at most once.") + + debug("entering consume by filter ") + if (topicFilter == null) + throw new RuntimeException("topicFilter is null") + + topicCount = TopicCount.constructTopicCount(consumerIdString, topicFilter, numStreams) + + // Create the coordinator connector thread + coordinatorConnector = new ConsumerCoordinatorConnector( + "kafka-coordinator-connector-", this) + coordinatorConnector.start + + // Create the wildcard stream handler to remember the stream type + // Note that the mapping between streams and threads is not yet created since the topics are not known yet + wildcardStreamsHandler = new WildcardStreamsHandler[T](topicFilter, numStreams, decoder) + + wildcardStreamsHandler.streams.asInstanceOf[Seq[KafkaStream[T]]] + } + + private def createFetcher() { + if (enableFetcher) + fetcher = Some(new Fetcher(config)) + } + + def checkFetcherLiveness(): Boolean = { + fetcher match { + case Some(fetcher) => fetcher.checkLiveness() + case _ => false + } + } + + def shutdown() { + val canShutdown = isShuttingDown.compareAndSet(false, true) + if (canShutdown) { + info("ConsumerConnector shutting down") + + try { + fetcher match { + case Some(f) => f.stopConnectionsToAllBrokers + case None => + } + sendShutdownToAllQueues() + + if (coordinatorConnector != null) { + coordinatorConnector.shutdown() + coordinatorConnector = null + } + } + catch { + case e => + fatal("error during consumer connector shutdown", e) + } + info("ConsumerConnector shut down completed") + + } + } + + def resetState() { + topicRegistry.clear + } + + private def sendShutdownToAllQueues() = { + for (queue <- topicThreadIdAndQueues.values) { + debug("Clearing up queue") + queue.clear() + queue.put(SimpleConsumerConnector.shutdownCommand) + debug("Cleared queue and sent shutdown command") + } + } + + def constructStreams[T](topicCount: TopicCount, + queuesAndStreams: List[(LinkedBlockingQueue[FetchedDataChunk], KafkaStream[T])]) { + // map of {topic -> Set(thread-1, thread-2, ...)} + val consumerThreadIdsPerTopic: Map[String, Set[String]] = + topicCount.getConsumerThreadIdsPerTopic + + /* + * This usage of map flatten breaks up consumerThreadIdsPerTopic into + * a set of (topic, thread-id) pairs that we then use to construct + * the updated (topic, thread-id) -> queues map + */ + implicit def getTopicThreadIds(v: (String, Set[String])): Set[(String, String)] = v._2.map((v._1, _)) + + // iterator over (topic, thread-id) tuples + val topicThreadIds: Iterable[(String, String)] = consumerThreadIdsPerTopic.flatten + + // list of (pairs of pairs): e.g., ((topic, thread-id),(queue, stream)) + val threadQueueStreamPairs = topicCount match { + case wildTopicCount: WildcardTopicCount => + require(topicThreadIds.size % queuesAndStreams.size == 0, + "Multiplex mismatch between thread ID count (%d) and queue count (%d)".format( + topicThreadIds.size, queuesAndStreams.size)) + for (tt <- topicThreadIds; qs <- queuesAndStreams) yield (tt -> qs) + case statTopicCount: StaticTopicCount => { + require(topicThreadIds.size == queuesAndStreams.size, + "Mismatch between thread ID count (%d) and queue count (%d)".format( + topicThreadIds.size, queuesAndStreams.size)) + topicThreadIds.zip(queuesAndStreams) + } + } + + threadQueueStreamPairs.foreach(e => { + val topicThreadId = e._1 + val q = e._2._1 + topicThreadIdAndQueues.put(TopicAndConsumerStreamId(topicThreadId._1, topicThreadId._2), q) + }) + + val groupedByTopic = threadQueueStreamPairs.groupBy(_._1._1) + groupedByTopic.foreach(e => { + val topic = e._1 + val streams = e._2.map(_._2._2).toList + messageStreamsPerTopic += (topic -> streams) + debug("adding topic %s and %d streams to map.".format(topic, streams.size)) + }) + + } + + def closeFetchersForQueues(cluster: Cluster, + messageStreams: Map[String,List[KafkaStream[_]]], + queuesToBeCleared: Iterable[BlockingQueue[FetchedDataChunk]]) { + val allPartitionInfos = topicRegistry.values.map(p => p.values).flatten + fetcher match { + case Some(f) => f.stopConnectionsToAllBrokers + f.clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, messageStreams) + case None => + } + } + + def startAllFetchers(cluster: Cluster) { + // update partitions for fetcher + var allPartitionInfos : List[PartitionTopicInfo] = Nil + for (partitionInfos <- topicRegistry.values) + for (partition <- partitionInfos.values) + allPartitionInfos ::= partition + debug("Consumer " + consumerIdString + " selected partitions : " + + allPartitionInfos.sortWith((s,t) => s.partition < t.partition).map(_.toString).mkString(",")) + + fetcher match { + case Some(f) => + f.startConnections(allPartitionInfos, cluster) + case None => + } + } + + def closeAllFetchers(cluster: Cluster, messageStreams: Map[String,List[KafkaStream[_]]]) { + val queuesTobeCleared = topicThreadIdAndQueues.map(q => q._2) + closeFetchersForQueues(cluster, messageStreams, queuesTobeCleared) + } + + /** + * The following functions are all used for JMX metrics + * + */ + def getPartOwnerStats(): String = { + val builder = new StringBuilder + for ((topic, infos) <- topicRegistry) { + builder.append("\n" + topic + ": [") + val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) + for(partition <- infos.values) { + builder.append("\n {") + builder.append{partition.partition.name} + builder.append(",fetch offset:" + partition.getFetchOffset) + builder.append(",consumer offset:" + partition.getConsumeOffset) + builder.append("}") + } + builder.append("\n ]") + } + builder.toString + } + + def getConsumerGroup(): String = config.groupId + + /** + * This is only used to carry the type of the wildcard streams + */ + class WildcardStreamsHandler[T](topicFilter: TopicFilter, + numStreams: Int, + decoder: Decoder[T]) + { + val wildcardQueuesAndStreams = (1 to numStreams) + .map(e => { + val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks) + val stream = new KafkaStream[T]( + queue, config.consumerTimeoutMs, decoder, config.enableShallowIterator) + (queue, stream) + }).toList + + def recreateStreams(topicCount: TopicCount) { + constructStreams(topicCount, wildcardQueuesAndStreams) + } + + def streams: Seq[KafkaStream[T]] = + wildcardQueuesAndStreams.map(_._2) + } + + + /* -------------------------- For Test Only -------------------------- */ + + def blockConnector() = coordinatorConnector.isBlocked.set(true) + + def unblockConnector() = coordinatorConnector.isBlocked.set(false) +} + Index: core/src/main/scala/kafka/consumer/ConsumerConfig.scala =================================================================== --- core/src/main/scala/kafka/consumer/ConsumerConfig.scala (revision 1371523) +++ core/src/main/scala/kafka/consumer/ConsumerConfig.scala (working copy) @@ -28,6 +28,7 @@ val DefaultFetcherBackoffMs = 1000 val AutoCommit = true val AutoCommitInterval = 10 * 1000 + val SessionTimeoutMs = 5 * 1000 val MaxQueuedChunks = 10 val MaxRebalanceRetries = 4 val AutoOffsetReset = OffsetRequest.SmallestTimeString @@ -71,6 +72,9 @@ /** the frequency in ms that the consumer offsets are committed to zookeeper */ val autoCommitIntervalMs = Utils.getInt(props, "autocommit.interval.ms", AutoCommitInterval) + /* the maximum number of bytes in a coordinator request */ + val maxCoordinatorRequestSize: Int = Utils.getIntInRange(props, "max.coordinator.request.bytes", 100*1024*1024, (1, Int.MaxValue)) + /** max number of messages buffered for consumption */ val maxQueuedChunks = Utils.getInt(props, "queuedchunks.max", MaxQueuedChunks) @@ -94,5 +98,12 @@ * overhead of decompression. * */ val enableShallowIterator = Utils.getBoolean(props, "shallowiterator.enable", false) + + /** The initial broker list used for discovering the topology of the Kafka cluster. + Will only be used at consumer startup */ + val initBrokerList = Utils.getString(props, "init.broker.list", "") + + /* coordinator session timeout in milliseconds */ + val coordinatorSessionTimeoutMs = Utils.getInt(props, "coordinator.session.timeout.ms", SessionTimeoutMs) } Index: core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala =================================================================== --- core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala (revision 1371523) +++ core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala (working copy) @@ -1,105 +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.consumer - -import scala.collection.JavaConversions._ -import kafka.utils.{ZkUtils, ZKStringSerializer, Logging} -import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} -import org.apache.zookeeper.Watcher.Event.KeeperState - -class ZookeeperTopicEventWatcher(val config:ConsumerConfig, - val eventHandler: TopicEventHandler[String]) extends Logging { - - val lock = new Object() - - private var zkClient: ZkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, ZKStringSerializer) - - startWatchingTopicEvents() - - private def startWatchingTopicEvents() { - val topicEventListener = new ZkTopicEventListener() - ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.BrokerTopicsPath) - - zkClient.subscribeStateChanges( - new ZkSessionExpireListener(topicEventListener)) - - val topics = zkClient.subscribeChildChanges( - ZkUtils.BrokerTopicsPath, topicEventListener).toList - - // call to bootstrap topic list - topicEventListener.handleChildChange(ZkUtils.BrokerTopicsPath, topics) - } - - private def stopWatchingTopicEvents() { zkClient.unsubscribeAll() } - - def shutdown() { - lock.synchronized { - info("Shutting down topic event watcher.") - if (zkClient != null) { - stopWatchingTopicEvents() - zkClient.close() - zkClient = null - } - else - warn("Cannot shutdown already shutdown topic event watcher.") - } - } - - class ZkTopicEventListener extends IZkChildListener { - - @throws(classOf[Exception]) - def handleChildChange(parent: String, children: java.util.List[String]) { - lock.synchronized { - try { - if (zkClient != null) { - val latestTopics = zkClient.getChildren(ZkUtils.BrokerTopicsPath).toList - debug("all topics: %s".format(latestTopics)) - - eventHandler.handleTopicEvent(latestTopics) - } - } - catch { - case e => - error("error in handling child changes", e) - } - } - } - - } - - class ZkSessionExpireListener(val topicEventListener: ZkTopicEventListener) - extends IZkStateListener { - - @throws(classOf[Exception]) - def handleStateChanged(state: KeeperState) { } - - @throws(classOf[Exception]) - def handleNewSession() { - lock.synchronized { - if (zkClient != null) { - info( - "ZK expired: resubscribing topic event listener to topic registry") - zkClient.subscribeChildChanges( - ZkUtils.BrokerTopicsPath, topicEventListener) - } - } - } - } -} - Index: core/src/main/scala/kafka/consumer/TopicCount.scala =================================================================== --- core/src/main/scala/kafka/consumer/TopicCount.scala (revision 1371523) +++ core/src/main/scala/kafka/consumer/TopicCount.scala (working copy) @@ -63,18 +63,16 @@ val myConversionFunc = {input : String => input.toInt} JSON.globalNumberParser = myConversionFunc - def constructTopicCount(group: String, - consumerId: String, - zkClient: ZkClient) : TopicCount = { - val dirs = new ZKGroupDirs(group) - val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId) + def constructTopicCount(consumerId: String, + topicCountString: String) : TopicCount = { + val hasWhitelist = topicCountString.startsWith(WHITELIST_MARKER) val hasBlacklist = topicCountString.startsWith(BLACKLIST_MARKER) if (hasWhitelist || hasBlacklist) - info("Constructing topic count for %s from %s using %s as pattern." + debug("Constructing topic count for %s from %s using %s as pattern." .format(consumerId, topicCountString, - if (hasWhitelist) WHITELIST_PATTERN else BLACKLIST_PATTERN)) + if (hasWhitelist) WHITELIST_PATTERN else BLACKLIST_PATTERN)) if (hasWhitelist || hasBlacklist) { val matcher = if (hasWhitelist) @@ -89,7 +87,9 @@ else new Blacklist(regex) - new WildcardTopicCount(zkClient, consumerId, filter, numStreams) + val wildcardTopicCount = new WildcardTopicCount(consumerId, filter, numStreams) + + wildcardTopicCount } else { var topMap : Map[String,Int] = null @@ -109,15 +109,27 @@ } } + def constructTopicCount(group: String, + consumerId: String, + zkClient: ZkClient) : TopicCount = { + val dirs = new ZKGroupDirs(group) + val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId) + + val topicCount = constructTopicCount(consumerId, topicCountString) + + if (topicCount.isInstanceOf[WildcardTopicCount]) + topicCount.asInstanceOf[WildcardTopicCount].setTopicsFromZk(zkClient) + + topicCount + } + def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) = new StaticTopicCount(consumerIdString, topicCount) def constructTopicCount(consumerIdString: String, filter: TopicFilter, - numStreams: Int, - zkClient: ZkClient) = - new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams) - + numStreams: Int) = + new WildcardTopicCount(consumerIdString, filter, numStreams) } private[kafka] class StaticTopicCount(val consumerIdString: String, @@ -156,17 +168,27 @@ } } -private[kafka] class WildcardTopicCount(zkClient: ZkClient, - consumerIdString: String, +private[kafka] class WildcardTopicCount(consumerIdString: String, topicFilter: TopicFilter, numStreams: Int) extends TopicCount { - def getConsumerThreadIdsPerTopic = { - val wildcardTopics = ZkUtils.getChildrenParentMayNotExist( + var wildcardTopics: Seq[String] = Nil + + def setTopicsFromZk(zkClient: ZkClient) = { + wildcardTopics = ZkUtils.getChildrenParentMayNotExist( zkClient, ZkUtils.BrokerTopicsPath).filter(topicFilter.isTopicAllowed(_)) - makeConsumerThreadIdsPerTopic(consumerIdString, - Map(wildcardTopics.map((_, numStreams)): _*)) } + def setTopics(topics: Set[String]) = { + wildcardTopics = topics.toSeq + } + + def getConsumerThreadIdsPerTopic = { + if (wildcardTopics == Nil) + Map.empty[String, Set[String]] + else + makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*)) + } + def dbString = { val marker = topicFilter match { case wl: Whitelist => TopicCount.WHITELIST_MARKER @@ -175,6 +197,5 @@ "%s%d%s%s".format(marker, numStreams, marker, topicFilter.regex) } - } Index: core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala =================================================================== --- core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala (revision 1371523) +++ core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala (working copy) @@ -1,810 +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.consumer - -import java.util.concurrent._ -import java.util.concurrent.atomic._ -import locks.ReentrantLock -import scala.collection._ -import kafka.cluster._ -import kafka.utils._ -import org.I0Itec.zkclient.exception.ZkNodeExistsException -import java.net.InetAddress -import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} -import org.apache.zookeeper.Watcher.Event.KeeperState -import kafka.api.OffsetRequest -import java.util.UUID -import kafka.serializer.Decoder -import kafka.common.{ConsumerRebalanceFailedException, InvalidConfigException} -import java.lang.IllegalStateException -import kafka.utils.ZkUtils._ - - -/** - * This class handles the consumers interaction with zookeeper - * - * Directories: - * 1. Consumer id registry: - * /consumers/[group_id]/ids[consumer_id] -> topic1,...topicN - * A consumer has a unique consumer id within a consumer group. A consumer registers its id as an ephemeral znode - * and puts all topics that it subscribes to as the value of the znode. The znode is deleted when the client is gone. - * A consumer subscribes to event changes of the consumer id registry within its group. - * - * The consumer id is picked up from configuration, instead of the sequential id assigned by ZK. Generated sequential - * ids are hard to recover during temporary connection loss to ZK, since it's difficult for the client to figure out - * whether the creation of a sequential znode has succeeded or not. More details can be found at - * (http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling) - * - * 2. Broker node registry: - * /brokers/[0...N] --> { "host" : "host:port", - * "topics" : {"topic1": ["partition1" ... "partitionN"], ..., - * "topicN": ["partition1" ... "partitionN"] } } - * This is a list of all present broker brokers. A unique logical node id is configured on each broker node. A broker - * node registers itself on start-up and creates a znode with the logical node id under /brokers. The value of the znode - * is a JSON String that contains (1) the host name and the port the broker is listening to, (2) a list of topics that - * the broker serves, (3) a list of logical partitions assigned to each topic on the broker. - * A consumer subscribes to event changes of the broker node registry. - * - * 3. Partition owner registry: - * /consumers/[group_id]/owner/[topic]/[broker_id-partition_id] --> consumer_node_id - * This stores the mapping before broker partitions and consumers. Each partition is owned by a unique consumer - * within a consumer group. The mapping is reestablished after each rebalancing. - * - * 4. Consumer offset tracking: - * /consumers/[group_id]/offsets/[topic]/[broker_id-partition_id] --> offset_counter_value - * Each consumer tracks the offset of the latest message consumed for each partition. - * - */ -private[kafka] object ZookeeperConsumerConnector { - val shutdownCommand: FetchedDataChunk = new FetchedDataChunk(null, null, -1L) -} - -/** - * JMX interface for monitoring consumer - */ -trait ZookeeperConsumerConnectorMBean { - def getPartOwnerStats: String - def getConsumerGroup: String - def getOffsetLag(topic: String, brokerId: Int, partitionId: Int): Long - def getConsumedOffset(topic: String, brokerId: Int, partitionId: Int): Long - def getLatestOffset(topic: String, brokerId: Int, partitionId: Int): Long -} - -private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, - val enableFetcher: Boolean) // for testing only - extends ConsumerConnector with ZookeeperConsumerConnectorMBean - with Logging { - private val isShuttingDown = new AtomicBoolean(false) - private val rebalanceLock = new Object - private var fetcher: Option[Fetcher] = None - private var zkClient: ZkClient = null - private var topicRegistry = new Pool[String, Pool[Partition, PartitionTopicInfo]] - // topicThreadIdAndQueues : (topic,consumerThreadId) -> queue - private val topicThreadIdAndQueues = new Pool[Tuple2[String,String], BlockingQueue[FetchedDataChunk]] - private val scheduler = new KafkaScheduler(1, "Kafka-consumer-autocommit-", false) - private val messageStreamCreated = new AtomicBoolean(false) - - private var sessionExpirationListener: ZKSessionExpireListener = null - private var loadBalancerListener: ZKRebalancerListener = null - - private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null - - val consumerIdString = { - var consumerUuid : String = null - config.consumerId match { - case Some(consumerId) // for testing only - => consumerUuid = consumerId - case None // generate unique consumerId automatically - => val uuid = UUID.randomUUID() - consumerUuid = "%s-%d-%s".format( - InetAddress.getLocalHost.getHostName, System.currentTimeMillis, - uuid.getMostSignificantBits().toHexString.substring(0,8)) - } - config.groupId + "_" + consumerUuid - } - this.logIdent = consumerIdString + " " - - connectZk() - createFetcher() - if (config.autoCommit) { - info("starting auto committer every " + config.autoCommitIntervalMs + " ms") - scheduler.scheduleWithRate(autoCommit, config.autoCommitIntervalMs, config.autoCommitIntervalMs) - } - - def this(config: ConsumerConfig) = this(config, true) - - def createMessageStreams[T](topicCountMap: Map[String,Int], - decoder: Decoder[T]) - : Map[String,List[KafkaStream[T]]] = { - if (messageStreamCreated.getAndSet(true)) - throw new RuntimeException(this.getClass.getSimpleName + - " can create message streams at most once") - consume(topicCountMap, decoder) - } - - def createMessageStreamsByFilter[T](topicFilter: TopicFilter, numStreams: Int, decoder: Decoder[T]) = { - val wildcardStreamsHandler = new WildcardStreamsHandler[T](topicFilter, numStreams, decoder) - wildcardStreamsHandler.streams - } - - private def createFetcher() { - if (enableFetcher) - fetcher = Some(new Fetcher(config, zkClient)) - } - - private def connectZk() { - info("Connecting to zookeeper instance at " + config.zkConnect) - zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) - } - - def shutdown() { - val canShutdown = isShuttingDown.compareAndSet(false, true); - if (canShutdown) { - info("ZKConsumerConnector shutting down") - - if (wildcardTopicWatcher != null) - wildcardTopicWatcher.shutdown() - try { - scheduler.shutdownNow() - fetcher match { - case Some(f) => f.stopConnectionsToAllBrokers - case None => - } - sendShutdownToAllQueues() - if (config.autoCommit) - commitOffsets() - if (zkClient != null) { - zkClient.close() - zkClient = null - } - } - catch { - case e => - fatal("error during consumer connector shutdown", e) - } - info("ZKConsumerConnector shut down completed") - } - } - - def consume[T](topicCountMap: scala.collection.Map[String,Int], - decoder: Decoder[T]) - : Map[String,List[KafkaStream[T]]] = { - debug("entering consume ") - if (topicCountMap == null) - throw new RuntimeException("topicCountMap is null") - - val topicCount = TopicCount.constructTopicCount(consumerIdString, topicCountMap) - - val topicThreadIds = topicCount.getConsumerThreadIdsPerTopic - - // make a list of (queue,stream) pairs, one pair for each threadId - val queuesAndStreams = topicThreadIds.values.map(threadIdSet => - threadIdSet.map(_ => { - val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks) - val stream = new KafkaStream[T]( - queue, config.consumerTimeoutMs, decoder, config.enableShallowIterator) - (queue, stream) - }) - ).flatten.toList - - val dirs = new ZKGroupDirs(config.groupId) - registerConsumerInZK(dirs, consumerIdString, topicCount) - reinitializeConsumer(topicCount, queuesAndStreams) - - loadBalancerListener.kafkaMessageAndMetadataStreams.asInstanceOf[Map[String, List[KafkaStream[T]]]] - } - - private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) { - info("begin registering consumer " + consumerIdString + " in ZK") - createEphemeralPathExpectConflict(zkClient, - dirs.consumerRegistryDir + "/" + consumerIdString, - topicCount.dbString) - info("end registering consumer " + consumerIdString + " in ZK") - } - - private def sendShutdownToAllQueues() = { - for (queue <- topicThreadIdAndQueues.values) { - debug("Clearing up queue") - queue.clear() - queue.put(ZookeeperConsumerConnector.shutdownCommand) - debug("Cleared queue and sent shutdown command") - } - } - - def autoCommit() { - trace("auto committing") - try { - commitOffsets() - } - catch { - case t: Throwable => - // log it and let it go - error("exception during autoCommit: ", t) - } - } - - def commitOffsets() { - if (zkClient == null) { - error("zk client is null. Cannot commit offsets") - return - } - for ((topic, infos) <- topicRegistry) { - val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) - for (info <- infos.values) { - val newOffset = info.getConsumeOffset - try { - updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partition.name, - newOffset.toString) - } - catch { - case t: Throwable => - // log it and let it go - warn("exception during commitOffsets", t) - } - debug("Committed offset " + newOffset + " for topic " + info) - } - } - } - - // for JMX - def getPartOwnerStats(): String = { - val builder = new StringBuilder - for ((topic, infos) <- topicRegistry) { - builder.append("\n" + topic + ": [") - val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) - for(partition <- infos.values) { - builder.append("\n {") - builder.append{partition.partition.name} - builder.append(",fetch offset:" + partition.getFetchOffset) - builder.append(",consumer offset:" + partition.getConsumeOffset) - builder.append("}") - } - builder.append("\n ]") - } - builder.toString - } - - // for JMX - def getConsumerGroup(): String = config.groupId - - def getOffsetLag(topic: String, brokerId: Int, partitionId: Int): Long = - getLatestOffset(topic, brokerId, partitionId) - getConsumedOffset(topic, brokerId, partitionId) - - def getConsumedOffset(topic: String, brokerId: Int, partitionId: Int): Long = { - val partition = new Partition(brokerId, partitionId) - val partitionInfos = topicRegistry.get(topic) - if (partitionInfos != null) { - val partitionInfo = partitionInfos.get(partition) - if (partitionInfo != null) - return partitionInfo.getConsumeOffset - } - - //otherwise, try to get it from zookeeper - try { - val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) - val znode = topicDirs.consumerOffsetDir + "/" + partition.name - val offsetString = readDataMaybeNull(zkClient, znode) - if (offsetString != null) - return offsetString.toLong - else - return -1 - } - catch { - case e => - error("error in getConsumedOffset JMX ", e) - } - return -2 - } - - def getLatestOffset(topic: String, brokerId: Int, partitionId: Int): Long = - earliestOrLatestOffset(topic, brokerId, partitionId, OffsetRequest.LatestTime) - - private def earliestOrLatestOffset(topic: String, brokerId: Int, partitionId: Int, earliestOrLatest: Long): Long = { - var simpleConsumer: SimpleConsumer = null - var producedOffset: Long = -1L - try { - val cluster = getCluster(zkClient) - val broker = cluster.getBroker(brokerId) match { - case Some(b) => b - case None => throw new IllegalStateException("Broker " + brokerId + " is unavailable. Cannot issue " + - "getOffsetsBefore request") - } - simpleConsumer = new SimpleConsumer(broker.host, broker.port, ConsumerConfig.SocketTimeout, - ConsumerConfig.SocketBufferSize) - val offsets = simpleConsumer.getOffsetsBefore(topic, partitionId, earliestOrLatest, 1) - producedOffset = offsets(0) - } - catch { - case e => - error("error in earliestOrLatestOffset() ", e) - } - finally { - if (simpleConsumer != null) - simpleConsumer.close - } - producedOffset - } - - class ZKSessionExpireListener(val dirs: ZKGroupDirs, - val consumerIdString: String, - val topicCount: TopicCount, - val loadBalancerListener: ZKRebalancerListener) - extends IZkStateListener { - @throws(classOf[Exception]) - def handleStateChanged(state: KeeperState) { - // do nothing, since zkclient will do reconnect for us. - } - - /** - * Called after the zookeeper session has expired and a new session has been created. You would have to re-create - * any ephemeral nodes here. - * - * @throws Exception - * On any error. - */ - @throws(classOf[Exception]) - def handleNewSession() { - /** - * When we get a SessionExpired event, we lost all ephemeral nodes and zkclient has reestablished a - * connection for us. We need to release the ownership of the current consumer and re-register this - * consumer in the consumer registry and trigger a rebalance. - */ - info("ZK expired; release old broker parition ownership; re-register consumer " + consumerIdString) - loadBalancerListener.resetState() - registerConsumerInZK(dirs, consumerIdString, topicCount) - // explicitly trigger load balancing for this consumer - loadBalancerListener.syncedRebalance() - - // There is no need to resubscribe to child and state changes. - // The child change watchers will be set inside rebalance when we read the children list. - } - - } - - class ZKRebalancerListener(val group: String, val consumerIdString: String, - val kafkaMessageAndMetadataStreams: mutable.Map[String,List[KafkaStream[_]]]) - extends IZkChildListener { - private var isWatcherTriggered = false - private val lock = new ReentrantLock - private val cond = lock.newCondition() - private val watcherExecutorThread = new Thread(consumerIdString + "_watcher_executor") { - override def run() { - info("starting watcher executor thread for consumer " + consumerIdString) - var doRebalance = false - while (!isShuttingDown.get) { - try { - lock.lock() - try { - if (!isWatcherTriggered) - cond.await(1000, TimeUnit.MILLISECONDS) // wake up periodically so that it can check the shutdown flag - } finally { - doRebalance = isWatcherTriggered - isWatcherTriggered = false - lock.unlock() - } - if (doRebalance) - syncedRebalance - } catch { - case t => error("error during syncedRebalance", t) - } - } - info("stopping watcher executor thread for consumer " + consumerIdString) - } - } - watcherExecutorThread.start() - - @throws(classOf[Exception]) - def handleChildChange(parentPath : String, curChilds : java.util.List[String]) { - lock.lock() - try { - isWatcherTriggered = true - cond.signalAll() - } finally { - lock.unlock() - } - } - - private def deletePartitionOwnershipFromZK(topic: String, partition: String) { - val topicDirs = new ZKGroupTopicDirs(group, topic) - val znode = topicDirs.consumerOwnerDir + "/" + partition - deletePath(zkClient, znode) - debug("Consumer " + consumerIdString + " releasing " + znode) - } - - private def releasePartitionOwnership(localTopicRegistry: Pool[String, Pool[Partition, PartitionTopicInfo]])= { - info("Releasing partition ownership") - for ((topic, infos) <- localTopicRegistry) { - for(partition <- infos.keys) - deletePartitionOwnershipFromZK(topic, partition.toString) - localTopicRegistry.remove(topic) - } - } - - def resetState() { - topicRegistry.clear - } - - def syncedRebalance() { - rebalanceLock synchronized { - for (i <- 0 until config.maxRebalanceRetries) { - info("begin rebalancing consumer " + consumerIdString + " try #" + i) - var done = false - val cluster = getCluster(zkClient) - try { - done = rebalance(cluster) - } - catch { - case e => - /** occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating. - * For example, a ZK node can disappear between the time we get all children and the time we try to get - * the value of a child. Just let this go since another rebalance will be triggered. - **/ - info("exception during rebalance ", e) - } - info("end rebalancing consumer " + consumerIdString + " try #" + i) - if (done) { - return - }else { - /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should - * clear the cache */ - info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") - } - // stop all fetchers and clear all the queues to avoid data duplication - closeFetchersForQueues(cluster, kafkaMessageAndMetadataStreams, topicThreadIdAndQueues.map(q => q._2)) - Thread.sleep(config.rebalanceBackoffMs) - } - } - - throw new ConsumerRebalanceFailedException(consumerIdString + " can't rebalance after " + config.maxRebalanceRetries +" retries") - } - - private def rebalance(cluster: Cluster): Boolean = { - val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic - val consumersPerTopicMap = getConsumersPerTopic(zkClient, group) - val partitionsPerTopicMap = getPartitionsForTopics(zkClient, myTopicThreadIdsMap.keys.iterator) - - /** - * fetchers must be stopped to avoid data duplication, since if the current - * rebalancing attempt fails, the partitions that are released could be owned by another consumer. - * But if we don't stop the fetchers first, this consumer would continue returning data for released - * partitions in parallel. So, not stopping the fetchers leads to duplicate data. - */ - closeFetchers(cluster, kafkaMessageAndMetadataStreams, myTopicThreadIdsMap) - - releasePartitionOwnership(topicRegistry) - - var partitionOwnershipDecision = new collection.mutable.HashMap[(String, String), String]() - var currentTopicRegistry = new Pool[String, Pool[Partition, PartitionTopicInfo]] - - for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) { - currentTopicRegistry.put(topic, new Pool[Partition, PartitionTopicInfo]) - - val topicDirs = new ZKGroupTopicDirs(group, topic) - val curConsumers = consumersPerTopicMap.get(topic).get - var curPartitions: List[String] = partitionsPerTopicMap.get(topic).get - - val nPartsPerConsumer = curPartitions.size / curConsumers.size - val nConsumersWithExtraPart = curPartitions.size % curConsumers.size - - info("Consumer " + consumerIdString + " rebalancing the following partitions: " + curPartitions + - " for topic " + topic + " with consumers: " + curConsumers) - - for (consumerThreadId <- consumerThreadIdSet) { - val myConsumerPosition = curConsumers.findIndexOf(_ == consumerThreadId) - assert(myConsumerPosition >= 0) - val startPart = nPartsPerConsumer*myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) - val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1) - - /** - * Range-partition the sorted partitions to consumers for better locality. - * The first few consumers pick up an extra partition, if any. - */ - if (nParts <= 0) - warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic) - else { - for (i <- startPart until startPart + nParts) { - val partition = curPartitions(i) - info(consumerThreadId + " attempting to claim partition " + partition) - addPartitionTopicInfo(currentTopicRegistry, topicDirs, partition, topic, consumerThreadId) - // record the partition ownership decision - partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) - } - } - } - } - - /** - * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt - * A rebalancing attempt is completed successfully only after the fetchers have been started correctly - */ - if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) { - info("Updating the cache") - debug("Partitions per topic cache " + partitionsPerTopicMap) - debug("Consumers per topic cache " + consumersPerTopicMap) - topicRegistry = currentTopicRegistry - updateFetcher(cluster) - true - }else { - false - } - } - - private def closeFetchersForQueues(cluster: Cluster, - messageStreams: Map[String,List[KafkaStream[_]]], - queuesToBeCleared: Iterable[BlockingQueue[FetchedDataChunk]]) { - var allPartitionInfos = topicRegistry.values.map(p => p.values).flatten - fetcher match { - case Some(f) => f.stopConnectionsToAllBrokers - f.clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, messageStreams) - info("Committing all offsets after clearing the fetcher queues") - /** - * here, we need to commit offsets before stopping the consumer from returning any more messages - * from the current data chunk. Since partition ownership is not yet released, this commit offsets - * call will ensure that the offsets committed now will be used by the next consumer thread owning the partition - * for the current data chunk. Since the fetchers are already shutdown and this is the last chunk to be iterated - * by the consumer, there will be no more messages returned by this iterator until the rebalancing finishes - * successfully and the fetchers restart to fetch more data chunks - **/ - commitOffsets - case None => - } - } - - private def closeFetchers(cluster: Cluster, messageStreams: Map[String,List[KafkaStream[_]]], - relevantTopicThreadIdsMap: Map[String, Set[String]]) { - // only clear the fetcher queues for certain topic partitions that *might* no longer be served by this consumer - // after this rebalancing attempt - val queuesTobeCleared = topicThreadIdAndQueues.filter(q => relevantTopicThreadIdsMap.contains(q._1._1)).map(q => q._2) - closeFetchersForQueues(cluster, messageStreams, queuesTobeCleared) - } - - private def updateFetcher(cluster: Cluster) { - // update partitions for fetcher - var allPartitionInfos : List[PartitionTopicInfo] = Nil - for (partitionInfos <- topicRegistry.values) - for (partition <- partitionInfos.values) - allPartitionInfos ::= partition - info("Consumer " + consumerIdString + " selected partitions : " + - allPartitionInfos.sortWith((s,t) => s.partition < t.partition).map(_.toString).mkString(",")) - - fetcher match { - case Some(f) => - f.startConnections(allPartitionInfos, cluster) - case None => - } - } - - private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, String), String]): Boolean = { - var successfullyOwnedPartitions : List[(String, String)] = Nil - val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner => - val topic = partitionOwner._1._1 - val partition = partitionOwner._1._2 - val consumerThreadId = partitionOwner._2 - val topicDirs = new ZKGroupTopicDirs(group, topic) - val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition - try { - createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId) - info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic) - successfullyOwnedPartitions ::= (topic, partition) - true - } - catch { - case e: ZkNodeExistsException => - // The node hasn't been deleted by the original owner. So wait a bit and retry. - info("waiting for the partition ownership to be deleted: " + partition) - false - case e2 => throw e2 - } - } - val hasPartitionOwnershipFailed = partitionOwnershipSuccessful.foldLeft(0)((sum, decision) => sum + (if(decision) 0 else 1)) - /* even if one of the partition ownership attempt has failed, return false */ - if(hasPartitionOwnershipFailed > 0) { - // remove all paths that we have owned in ZK - successfullyOwnedPartitions.foreach(topicAndPartition => deletePartitionOwnershipFromZK(topicAndPartition._1, topicAndPartition._2)) - false - } - else true - } - - private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Partition, PartitionTopicInfo]], - topicDirs: ZKGroupTopicDirs, partitionString: String, - topic: String, consumerThreadId: String) { - val partition = Partition.parse(partitionString) - val partTopicInfoMap = currentTopicRegistry.get(topic) - - val znode = topicDirs.consumerOffsetDir + "/" + partition.name - val offsetString = readDataMaybeNull(zkClient, znode) - // If first time starting a consumer, set the initial offset based on the config - var offset : Long = 0L - if (offsetString == null) - offset = config.autoOffsetReset match { - case OffsetRequest.SmallestTimeString => - earliestOrLatestOffset(topic, partition.brokerId, partition.partId, OffsetRequest.EarliestTime) - case OffsetRequest.LargestTimeString => - earliestOrLatestOffset(topic, partition.brokerId, partition.partId, OffsetRequest.LatestTime) - case _ => - throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig") - } - else - offset = offsetString.toLong - val queue = topicThreadIdAndQueues.get((topic, consumerThreadId)) - val consumedOffset = new AtomicLong(offset) - val fetchedOffset = new AtomicLong(offset) - val partTopicInfo = new PartitionTopicInfo(topic, - partition.brokerId, - partition, - queue, - consumedOffset, - fetchedOffset, - new AtomicInteger(config.fetchSize)) - partTopicInfoMap.put(partition, partTopicInfo) - debug(partTopicInfo + " selected new offset " + offset) - } - } - - private def reinitializeConsumer[T]( - topicCount: TopicCount, - queuesAndStreams: List[(LinkedBlockingQueue[FetchedDataChunk],KafkaStream[T])]) { - - val dirs = new ZKGroupDirs(config.groupId) - - // listener to consumer and partition changes - if (loadBalancerListener == null) { - val topicStreamsMap = new mutable.HashMap[String,List[KafkaStream[T]]] - loadBalancerListener = new ZKRebalancerListener( - config.groupId, consumerIdString, topicStreamsMap.asInstanceOf[scala.collection.mutable.Map[String, List[KafkaStream[_]]]]) - } - - // register listener for session expired event - if (sessionExpirationListener == null) - sessionExpirationListener = new ZKSessionExpireListener( - dirs, consumerIdString, topicCount, loadBalancerListener) - - val topicStreamsMap = loadBalancerListener.kafkaMessageAndMetadataStreams - - // map of {topic -> Set(thread-1, thread-2, ...)} - val consumerThreadIdsPerTopic: Map[String, Set[String]] = - topicCount.getConsumerThreadIdsPerTopic - - /* - * This usage of map flatten breaks up consumerThreadIdsPerTopic into - * a set of (topic, thread-id) pairs that we then use to construct - * the updated (topic, thread-id) -> queues map - */ - implicit def getTopicThreadIds(v: (String, Set[String])): Set[(String, String)] = v._2.map((v._1, _)) - - // iterator over (topic, thread-id) tuples - val topicThreadIds: Iterable[(String, String)] = - consumerThreadIdsPerTopic.flatten - - // list of (pairs of pairs): e.g., ((topic, thread-id),(queue, stream)) - val threadQueueStreamPairs = topicCount match { - case wildTopicCount: WildcardTopicCount => - for (tt <- topicThreadIds; qs <- queuesAndStreams) yield (tt -> qs) - case statTopicCount: StaticTopicCount => { - require(topicThreadIds.size == queuesAndStreams.size, - "Mismatch between thread ID count (%d) and queue count (%d)".format( - topicThreadIds.size, queuesAndStreams.size)) - topicThreadIds.zip(queuesAndStreams) - } - } - - threadQueueStreamPairs.foreach(e => { - val topicThreadId = e._1 - val q = e._2._1 - topicThreadIdAndQueues.put(topicThreadId, q) - }) - - val groupedByTopic = threadQueueStreamPairs.groupBy(_._1._1) - groupedByTopic.foreach(e => { - val topic = e._1 - val streams = e._2.map(_._2._2).toList - topicStreamsMap += (topic -> streams) - debug("adding topic %s and %d streams to map.".format(topic, streams.size)) - }) - - // listener to consumer and partition changes - zkClient.subscribeStateChanges(sessionExpirationListener) - - zkClient.subscribeChildChanges(dirs.consumerRegistryDir, loadBalancerListener) - - topicStreamsMap.foreach { topicAndStreams => - // register on broker partition path changes - val partitionPath = BrokerTopicsPath + "/" + topicAndStreams._1 - zkClient.subscribeChildChanges(partitionPath, loadBalancerListener) - } - - // explicitly trigger load balancing for this consumer - loadBalancerListener.syncedRebalance() - } - - class WildcardStreamsHandler[T](topicFilter: TopicFilter, - numStreams: Int, - decoder: Decoder[T]) - extends TopicEventHandler[String] { - - if (messageStreamCreated.getAndSet(true)) - throw new RuntimeException("Each consumer connector can create " + - "message streams by filter at most once.") - - private val wildcardQueuesAndStreams = (1 to numStreams) - .map(e => { - val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks) - val stream = new KafkaStream[T]( - queue, config.consumerTimeoutMs, decoder, config.enableShallowIterator) - (queue, stream) - }).toList - - // bootstrap with existing topics - private var wildcardTopics = - getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) - .filter(topicFilter.isTopicAllowed) - - private val wildcardTopicCount = TopicCount.constructTopicCount( - consumerIdString, topicFilter, numStreams, zkClient) - - val dirs = new ZKGroupDirs(config.groupId) - registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount) - reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams) - - if (!topicFilter.requiresTopicEventWatcher) { - info("Not creating event watcher for trivial whitelist " + topicFilter) - } - else { - info("Creating topic event watcher for whitelist " + topicFilter) - wildcardTopicWatcher = new ZookeeperTopicEventWatcher(config, this) - - /* - * Topic events will trigger subsequent synced rebalances. Also, the - * consumer will get registered only after an allowed topic becomes - * available. - */ - } - - def handleTopicEvent(allTopics: Seq[String]) { - debug("Handling topic event") - - val updatedTopics = allTopics.filter(topicFilter.isTopicAllowed) - - val addedTopics = updatedTopics filterNot (wildcardTopics contains) - if (addedTopics.nonEmpty) - info("Topic event: added topics = %s" - .format(addedTopics)) - - /* - * TODO: Deleted topics are interesting (and will not be a concern until - * 0.8 release). We may need to remove these topics from the rebalance - * listener's map in reinitializeConsumer. - */ - val deletedTopics = wildcardTopics filterNot (updatedTopics contains) - if (deletedTopics.nonEmpty) - info("Topic event: deleted topics = %s" - .format(deletedTopics)) - - wildcardTopics = updatedTopics - info("Topics to consume = %s".format(wildcardTopics)) - - if (addedTopics.nonEmpty || deletedTopics.nonEmpty) - reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams) - } - - def streams: Seq[KafkaStream[T]] = - wildcardQueuesAndStreams.map(_._2) - } -} - Index: core/src/main/scala/kafka/consumer/CoordinatorRequestHandler.scala =================================================================== --- core/src/main/scala/kafka/consumer/CoordinatorRequestHandler.scala (revision 0) +++ core/src/main/scala/kafka/consumer/CoordinatorRequestHandler.scala (revision 0) @@ -0,0 +1,217 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.consumer + +import kafka.utils.{Pool, Logging} +import kafka.api.{StartFetcherRequest, StopFetcherRequest, RequestKeys, HeartbeatRequest} +import kafka.network.{InvalidRequestException, Handler, Receive, Send} +import kafka.common.ErrorMapping +import kafka.cluster.Partition +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} + + +/** + * Logic to handle the various coordinator requests + */ + +class CoordinatorRequestHandler (val coordinatorConnector: ConsumerCoordinatorConnector, + val consumerConnector: SimpleConsumerConnector) extends Logging { + + def handle(requestTypeId: Short, receive: Receive): Send = { + var response: Send = null + requestTypeId match { + case RequestKeys.Heartbeat => + trace("Handling heartbeat request from coordinator " + consumerConnector.coordinatorId) + val request = HeartbeatRequest.readFrom(receive.buffer) + response = handleHeartbeatRequest(request) + case RequestKeys.StopFetcher => + trace("Handling stop-fetcher request from coordinator " + consumerConnector.coordinatorId) + val request = StopFetcherRequest.readFrom(receive.buffer) + response = handleStopFetcherRequest(request) + case RequestKeys.StartFetcher => + trace("Handling start-fetcher request from coordinator " + consumerConnector.coordinatorId) + val request = StartFetcherRequest.readFrom(receive.buffer) + response = handleStartFetcherRequest(request) + case _ => + throw new InvalidRequestException("No mapping found for handler id " + requestTypeId) + } + + response + } + + def handleHeartbeatRequest(request: HeartbeatRequest): Send = { + debug("Handling heartbeat request from coordinator") + + if (request.consumerId != consumerConnector.consumerIdString) + throw new IllegalStateException("A heartbeat request is recieved by consumer %s but was intended for consumer %s" + .format(consumerConnector.consumerIdString, request.consumerId)) + + // If shutdown is initiated: 1) autoCommit is not turned on, can shut down immediately; + // otherwise need to wait for last heartbeat to reply + if (coordinatorConnector.shutdownInitiated.get) { + consumerConnector.config.autoCommit match { + case true => { + coordinatorConnector.lastHeartbeatReplied.get match { + case true => { + info("Shutdown is initiated and last heartbeat is already received by the coordinator, can shutdown now") + coordinatorConnector.canShutdown.set(true) + } + case false => { + info("Last heartbeat is going to be replied, can shutdown when the next heartbeat is received") + coordinatorConnector.lastHeartbeatReplied.set(true) + } + } + } + case false => { + info("Shutdown is initiated and do not need to commit offset through heartbeat, can shutdown now") + coordinatorConnector.canShutdown.set(true) + } + } + } + + // Check liveness of the fetchers + if (!coordinatorConnector.shutdownInitiated.get && !coordinatorConnector.fetcherStopped && !consumerConnector.checkFetcherLiveness) { + throw new IllegalStateException("Fetchers have stopped unexpectedly, probably due to offset-outof-range error") + } + + consumerConnector.config.autoCommit match { + case true => { + new ConsumerOffsetsSend(consumerConnector.consumerIdString, + RequestKeys.Heartbeat, + consumerConnector.topicRegistry, + ErrorMapping.NoError) + } + case false => { + new ConsumerOffsetsSend(consumerConnector.consumerIdString, + RequestKeys.Heartbeat, + null, + ErrorMapping.NoError) + } + } + } + + def handleStopFetcherRequest(request: StopFetcherRequest): Send = { + debug("Handling stopfetcher request from coordinator") + + if (request.consumerId != consumerConnector.consumerIdString) + throw new IllegalStateException("A stop fetcher request is recieved by consumer %s but was intended for consumer %s" + .format(consumerConnector.consumerIdString, request.consumerId)) + + // Can receive stop-fetcher when it is already in the stop-fetcher state + coordinatorConnector.fetcherStopped match { + case true => { + debug("Fetchers are already stopped. This is ok due to coordinator or consumer failover") + } + case false => { + /* + * Note Must first set fetcherStopped then close fetchers, otherwise it is possible that the connector + * mistakenly treat the stopped fetchers as unexpected + **/ + coordinatorConnector.fetcherStopped = true + + consumerConnector.closeAllFetchers(consumerConnector.kafkaCluster, consumerConnector.messageStreamsPerTopic) + } + } + + consumerConnector.config.autoCommit match { + case true => + new ConsumerOffsetsSend(consumerConnector.consumerIdString, + RequestKeys.StopFetcher, + consumerConnector.topicRegistry, + ErrorMapping.NoError) + case false => + new ConsumerOffsetsSend(consumerConnector.consumerIdString, + RequestKeys.StopFetcher, + null, + ErrorMapping.NoError) + } + } + + def handleStartFetcherRequest(request: StartFetcherRequest): Send = { + debug("Handling startfetcher request from coordinator") + + if (request.consumerStreamId != consumerConnector.consumerIdString) + throw new IllegalStateException("A start fetcher request is recieved by consumer %s but was intended for consumer %s" + .format(consumerConnector.consumerIdString, request.consumerStreamId)) + + // Can only receive start-fetcher when it is already in the stop-fetcher state + // when the command has the same ownership specification + coordinatorConnector.fetcherStopped match { + case true => { + // First try to update the server cluster metadata + consumerConnector.kafkaCluster = request.cluster + + // If the consumer has subscribed to wildcarded topics, first + // construct the topic thread map, then construct the mappings from topic thread to streams + if (consumerConnector.topicCount.isInstanceOf[WildcardTopicCount]) { + if (consumerConnector.wildcardStreamsHandler == null) + throw new IllegalStateException("Consumer %s with wildcard streams does not created a wildcard stream handler yet" + .format(consumerConnector.consumerIdString)) + + var topics = Set.empty[String] + for (lst <- request.ownership) { + for (part <- lst.parts) { + topics += part.topic + } + } + + consumerConnector.topicCount.asInstanceOf[WildcardTopicCount].setTopics(topics) + consumerConnector.wildcardStreamsHandler.recreateStreams(consumerConnector.topicCount) + } + + // Read the ownership info + debug("Updating the ownership in cache") + + val currentTopicRegistry = new Pool[String, Pool[Partition, PartitionTopicInfo]] + for (threadPartitions <- request.ownership) { + for (consumerThreadPartition <- threadPartitions.parts) { + if (!currentTopicRegistry.contains(consumerThreadPartition.topic)) + currentTopicRegistry.put(consumerThreadPartition.topic, new Pool[Partition, PartitionTopicInfo]) + val partTopicInfoMap = currentTopicRegistry.get(consumerThreadPartition.topic) + val partition = Partition.parse(consumerThreadPartition.partition) + val queue = consumerConnector.topicThreadIdAndQueues.get(TopicAndConsumerStreamId(consumerThreadPartition.topic, threadPartitions.threadId)) + val consumedOffset = new AtomicLong(consumerThreadPartition.offset) + val fetchedOffset = new AtomicLong(consumerThreadPartition.offset) + val partTopicInfo = new PartitionTopicInfo(consumerThreadPartition.topic, + partition.brokerId, + partition, + queue, + consumedOffset, + fetchedOffset, + new AtomicInteger(consumerConnector.config.fetchSize)) + partTopicInfoMap.put(partition, partTopicInfo) + } + } + consumerConnector.topicRegistry = currentTopicRegistry + + // Then start fetchers + consumerConnector.startAllFetchers(consumerConnector.kafkaCluster) + + coordinatorConnector.fetcherStopped = false + + new ConsumerOffsetsSend(consumerConnector.consumerIdString, + RequestKeys.StartFetcher, + null, + ErrorMapping.NoError) + } + case false => { + throw new IllegalStateException("Unexpected start-fetcher request since fetcher is already started") + } + } + } +} Index: core/src/main/scala/kafka/javaapi/consumer/SimpleConsumerConnector.scala =================================================================== --- core/src/main/scala/kafka/javaapi/consumer/SimpleConsumerConnector.scala (revision 0) +++ core/src/main/scala/kafka/javaapi/consumer/SimpleConsumerConnector.scala (revision 0) @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.javaapi.consumer + +import kafka.message.Message +import kafka.serializer.{DefaultDecoder, Decoder} +import kafka.consumer._ +import scala.collection.JavaConversions.asList + +private[kafka] class SimpleConsumerConnector(val config: ConsumerConfig, + val enableFetcher: Boolean) // for testing only + extends ConsumerConnector { + + val underlying = new kafka.consumer.SimpleConsumerConnector(config, enableFetcher) + + def this(config: ConsumerConfig) = this(config, true) + + // for java client + def createMessageStreams[T](topicCountMap: java.util.Map[String,java.lang.Integer], + decoder: Decoder[T]) : java.util.Map[String,java.util.List[KafkaStream[T]]] = { + import scala.collection.JavaConversions._ + + val scalaTopicCountMap: Map[String, Int] = Map.empty[String, Int] ++ asMap(topicCountMap.asInstanceOf[java.util.Map[String, Int]]) + val scalaReturn = underlying.createMessageStreams(scalaTopicCountMap, decoder) + val ret = new java.util.HashMap[String,java.util.List[KafkaStream[T]]] + for ((topic, streams) <- scalaReturn) { + var javaStreamList = new java.util.ArrayList[KafkaStream[T]] + for (stream <- streams) + javaStreamList.add(stream) + ret.put(topic, javaStreamList) + } + ret + } + + def createMessageStreams(topicCountMap: java.util.Map[String,java.lang.Integer]) : java.util.Map[String,java.util.List[KafkaStream[Message]]] = + createMessageStreams(topicCountMap, new DefaultDecoder) + + def createMessageStreamsByFilter[T](topicFilter: TopicFilter, numStreams: Int, decoder: Decoder[T]) = + asList(underlying.createMessageStreamsByFilter(topicFilter, numStreams, decoder)) + + def createMessageStreamsByFilter(topicFilter: TopicFilter, numStreams: Int) = + createMessageStreamsByFilter(topicFilter, numStreams, new DefaultDecoder) + + def createMessageStreamsByFilter(topicFilter: TopicFilter) = + createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder) + + def commitOffsets() { + underlying.commitOffsets + } + + def shutdown() { + underlying.shutdown + } +} Index: core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala =================================================================== --- core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala (revision 1371523) +++ core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala (working copy) @@ -1,109 +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.javaapi.consumer - -import kafka.message.Message -import kafka.serializer.{DefaultDecoder, Decoder} -import kafka.consumer._ -import scala.collection.JavaConversions.asList - - -/** - * This class handles the consumers interaction with zookeeper - * - * Directories: - * 1. Consumer id registry: - * /consumers/[group_id]/ids[consumer_id] -> topic1,...topicN - * A consumer has a unique consumer id within a consumer group. A consumer registers its id as an ephemeral znode - * and puts all topics that it subscribes to as the value of the znode. The znode is deleted when the client is gone. - * A consumer subscribes to event changes of the consumer id registry within its group. - * - * The consumer id is picked up from configuration, instead of the sequential id assigned by ZK. Generated sequential - * ids are hard to recover during temporary connection loss to ZK, since it's difficult for the client to figure out - * whether the creation of a sequential znode has succeeded or not. More details can be found at - * (http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling) - * - * 2. Broker node registry: - * /brokers/[0...N] --> { "host" : "host:port", - * "topics" : {"topic1": ["partition1" ... "partitionN"], ..., - * "topicN": ["partition1" ... "partitionN"] } } - * This is a list of all present broker brokers. A unique logical node id is configured on each broker node. A broker - * node registers itself on start-up and creates a znode with the logical node id under /brokers. The value of the znode - * is a JSON String that contains (1) the host name and the port the broker is listening to, (2) a list of topics that - * the broker serves, (3) a list of logical partitions assigned to each topic on the broker. - * A consumer subscribes to event changes of the broker node registry. - * - * 3. Partition owner registry: - * /consumers/[group_id]/owner/[topic]/[broker_id-partition_id] --> consumer_node_id - * This stores the mapping before broker partitions and consumers. Each partition is owned by a unique consumer - * within a consumer group. The mapping is reestablished after each rebalancing. - * - * 4. Consumer offset tracking: - * /consumers/[group_id]/offsets/[topic]/[broker_id-partition_id] --> offset_counter_value - * Each consumer tracks the offset of the latest message consumed for each partition. - * -*/ - -private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, - val enableFetcher: Boolean) // for testing only - extends ConsumerConnector { - - val underlying = new kafka.consumer.ZookeeperConsumerConnector(config, enableFetcher) - - def this(config: ConsumerConfig) = this(config, true) - - // for java client - def createMessageStreams[T]( - topicCountMap: java.util.Map[String,java.lang.Integer], - decoder: Decoder[T]) - : java.util.Map[String,java.util.List[KafkaStream[T]]] = { - import scala.collection.JavaConversions._ - - val scalaTopicCountMap: Map[String, Int] = Map.empty[String, Int] ++ asMap(topicCountMap.asInstanceOf[java.util.Map[String, Int]]) - val scalaReturn = underlying.consume(scalaTopicCountMap, decoder) - val ret = new java.util.HashMap[String,java.util.List[KafkaStream[T]]] - for ((topic, streams) <- scalaReturn) { - var javaStreamList = new java.util.ArrayList[KafkaStream[T]] - for (stream <- streams) - javaStreamList.add(stream) - ret.put(topic, javaStreamList) - } - ret - } - - def createMessageStreams( - topicCountMap: java.util.Map[String,java.lang.Integer]) - : java.util.Map[String,java.util.List[KafkaStream[Message]]] = - createMessageStreams(topicCountMap, new DefaultDecoder) - - def createMessageStreamsByFilter[T](topicFilter: TopicFilter, numStreams: Int, decoder: Decoder[T]) = - asList(underlying.createMessageStreamsByFilter(topicFilter, numStreams, decoder)) - - def createMessageStreamsByFilter(topicFilter: TopicFilter, numStreams: Int) = - createMessageStreamsByFilter(topicFilter, numStreams, new DefaultDecoder) - - def createMessageStreamsByFilter(topicFilter: TopicFilter) = - createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder) - - def commitOffsets() { - underlying.commitOffsets - } - - def shutdown() { - underlying.shutdown - } -}