diff --git a/core/src/main/scala/kafka/common/InconsistentBrokerIdException.scala b/core/src/main/scala/kafka/common/InconsistentBrokerIdException.scala new file mode 100644 index 0000000..5649828 --- /dev/null +++ b/core/src/main/scala/kafka/common/InconsistentBrokerIdException.scala @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +/** + * Indicates the client has requested a range no longer available on the server + */ +class InconsistentBrokerIdException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 50b09ed..6427500 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -35,13 +35,13 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro private def getLogRetentionTimeMillis(): Long = { val millisInMinute = 60L * 1000L val millisInHour = 60L * millisInMinute - + if(props.containsKey("log.retention.ms")){ props.getIntInRange("log.retention.ms", (1, Int.MaxValue)) } else if(props.containsKey("log.retention.minutes")){ millisInMinute * props.getIntInRange("log.retention.minutes", (1, Int.MaxValue)) - } + } else { millisInHour * props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue)) } @@ -49,7 +49,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro private def getLogRollTimeMillis(): Long = { val millisInHour = 60L * 60L * 1000L - + if(props.containsKey("log.roll.ms")){ props.getIntInRange("log.roll.ms", (1, Int.MaxValue)) } @@ -57,11 +57,11 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro millisInHour * props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue)) } } - + /*********** General Configuration ***********/ /* the broker id for this server */ - val brokerId: Int = props.getIntInRange("broker.id", (0, Int.MaxValue)) + var brokerId: Int = if (props.containsKey("broker.id")) props.getIntInRange("broker.id", (0, 1000)) else -1 /* the maximum size of message that the server can receive */ val messageMaxBytes = props.getIntInRange("message.max.bytes", 1000000 + MessageSet.LogOverhead, (0, Int.MaxValue)) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index def1dc2..9f84ebc 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -25,11 +25,16 @@ import kafka.utils._ import java.util.concurrent._ import atomic.{AtomicInteger, AtomicBoolean} import java.io.File +import java.io.FileOutputStream +import java.io.FileNotFoundException +import java.util.Properties +import collection.mutable import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} import kafka.cluster.Broker import kafka.api.{ControlledShutdownResponse, ControlledShutdownRequest} import kafka.common.ErrorMapping +import kafka.common.InconsistentBrokerIdException import kafka.network.{Receive, BlockingChannel, SocketServer} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge @@ -39,10 +44,12 @@ import com.yammer.metrics.core.Gauge * to start up and shutdown a single Kafka node. */ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logging with KafkaMetricsGroup { - this.logIdent = "[Kafka Server " + config.brokerId + "], " + private var isShuttingDown = new AtomicBoolean(false) private var shutdownLatch = new CountDownLatch(1) private var startupComplete = new AtomicBoolean(false) + private var brokerId: Int = -1 + private var metaPropsFile = "meta.properties" val brokerState: BrokerState = new BrokerState val correlationId: AtomicInteger = new AtomicInteger(0) var socketServer: SocketServer = null @@ -76,7 +83,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg /* start scheduler */ kafkaScheduler.startup() - + /* setup zookeeper */ zkClient = initZk() @@ -84,6 +91,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg logManager = createLogManager(zkClient, brokerState) logManager.startup() + /* generate brokerId */ + config.brokerId = getBrokerId(zkClient, config.brokerId, config.logDirs) + this.logIdent = "[Kafka Server " + config.brokerId + "], " + socketServer = new SocketServer(config.brokerId, config.hostName, config.port, @@ -101,31 +112,31 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg offsetManager = createOffsetManager() kafkaController = new KafkaController(config, zkClient, brokerState) - + /* start processing requests */ apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, zkClient, config.brokerId, config, kafkaController) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) - + Mx4jLoader.maybeLoad() replicaManager.startup() kafkaController.startup() - + topicConfigManager = new TopicConfigManager(zkClient, logManager) topicConfigManager.startup() - + /* tell everyone we are alive */ kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) kafkaHealthcheck.startup() - + registerStats() startupComplete.set(true) info("started") } - + private def initZk(): ZkClient = { info("Connecting to zookeeper on " + config.zkConnect) val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) @@ -273,9 +284,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def awaitShutdown(): Unit = shutdownLatch.await() def getLogManager(): LogManager = logManager - + private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = { - val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, + val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, segmentMs = config.logRollTimeMillis, flushInterval = config.logFlushIntervalMessages, flushMs = config.logFlushIntervalMs.toLong, @@ -323,5 +334,60 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg new OffsetManager(offsetManagerConfig, replicaManager, zkClient, kafkaScheduler) } -} + /** + * if kafka server config has brokerId and there is no meta.properties file returns the config.brokerId, + * otherwise generates a sequence id from ZK uses it has a brokerId. + * stores the generated zk sequence id in meta.properties under logDirs specified in config. + * if config has brokerId and meta.properties contains brokerId if they don't match throws InconsistentBrokerIdException + */ + private def getBrokerId(zkClient: ZkClient, configBrokerId: Int, logDirs: Seq[String]): Int = { + var brokerId = configBrokerId + var logDirsWithoutMetaProps: List[String] = List() + val metaBrokerIdSet = mutable.HashSet[Int]() + for (logDir <- logDirs) { + val (succeeded, metaBrokerId) = readBrokerIdFromMetaProps(logDir) + if(!succeeded) { + logDirsWithoutMetaProps ++= List(logDir) + } else { + metaBrokerIdSet.add(metaBrokerId) + } + } + + if(metaBrokerIdSet.size > 1) { + throw new InconsistentBrokerIdException("unable to match brokerId across logDirs") + } else if(brokerId >= 0 && metaBrokerIdSet.size == 1 && metaBrokerIdSet.last != brokerId) { + throw new InconsistentBrokerIdException("configured brokerId doesn't match stored brokerId in meta.properties") + } else if(metaBrokerIdSet.size == 0) { + if(brokerId < 0) { + brokerId = ZkUtils.getBrokerSequenceId(zkClient) + } else { + return brokerId + } + } + storeBrokerId(brokerId, logDirsWithoutMetaProps) + return brokerId + } + + private def readBrokerIdFromMetaProps(logDir: String): (Boolean, Int) = { + try { + val metaProps = new VerifiableProperties(Utils.loadProps(logDir + File.separator + metaPropsFile)) + if (metaProps.containsKey("broker.id")) + return (true, metaProps.getIntInRange("broker.id", (0, Int.MaxValue))) + } catch { + case e: FileNotFoundException => + (false, -1) + } + (false, -1) + } + + private def storeBrokerId(brokerId: Int, logDirs: Seq[String]) { + val metaProps = new Properties() + metaProps.setProperty("broker.id", brokerId.toString); + for(logDir <- logDirs) { + val f = Utils.createFile(logDir + File.separator + metaPropsFile) + val out = new FileOutputStream(f) + metaProps.store(out,"") + } + } +} diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index dcdc1ce..6b65b4a 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -46,6 +46,7 @@ object ZkUtils extends Logging { val ReassignPartitionsPath = "/admin/reassign_partitions" val DeleteTopicsPath = "/admin/delete_topics" val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" + val BrokerSequenceIdPath = "/brokers/seqid" def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic @@ -87,7 +88,8 @@ object ZkUtils extends Logging { } def setupCommonPaths(zkClient: ZkClient) { - for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath, DeleteTopicsPath)) + for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, + TopicConfigPath, DeleteTopicsPath, BrokerSequenceIdPath)) makeSurePersistentPathExists(zkClient, path) } @@ -122,6 +124,15 @@ object ZkUtils extends Logging { } } + /** returns a sequence id generated by updating BrokerSequenceIdPath in Zk. + * we add 1000 to the return value from getSequenceId for the backward compatability. + * users can provide brokerId in the config , inorder to avoid conflicts between zk generated + * seqId and config.brokerId we increment zk seqId by 1000. + */ + def getBrokerSequenceId(zkClient: ZkClient): Int = { + getSequenceId(zkClient, BrokerSequenceIdPath) + 1000 + } + /** * Gets the in-sync replicas (ISR) for a specific topic and partition */ @@ -691,6 +702,32 @@ object ZkUtils extends Logging { } } + /** + * This API produces a sequence number by creating / updating given path in zookeeper + * It uses the stat returned by the zookeeper and return the version. Every time + * client updates the path stat.version gets incremented + */ + def getSequenceId(client: ZkClient, path: String): Int = { + try { + val stat = client.writeDataReturnStat(path, "", -1) + return stat.getVersion + } catch { + case e: ZkNoNodeException => { + createParentPath(client, BrokerSequenceIdPath) + try { + client.createPersistent(BrokerSequenceIdPath, "") + return 0 + } catch { + case e: ZkNodeExistsException => + val stat = client.writeDataReturnStat(BrokerSequenceIdPath, "", -1) + return stat.getVersion + case e2: Throwable => throw e2 + } + } + case e2: Throwable => throw e2 + } + } + def getAllTopics(zkClient: ZkClient): Seq[String] = { val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) if(topics == null) diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala new file mode 100644 index 0000000..f8cf294 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server + +import kafka.zk.ZooKeeperTestHarness +import kafka.utils.{IntEncoder, TestUtils, Utils, VerifiableProperties} +import kafka.utils.TestUtils._ +import java.io.File +import org.junit.Test +import org.scalatest.junit.JUnit3Suite +import junit.framework.Assert._ + +class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { + val port = TestUtils.choosePort + var props1 = TestUtils.createBrokerConfig(-1, port) + var config1 = new KafkaConfig(props1) + var props2 = TestUtils.createBrokerConfig(0, port) + var config2 = new KafkaConfig(props2) + + @Test + def testAutoGenerateBrokerId() { + var server1 = new KafkaServer(config1) + server1.startup() + // do a clean shutdown and check that offset checkpoint file exists + server1.shutdown() + for(logDir <- config1.logDirs) { + val metaProps = new VerifiableProperties(Utils.loadProps(logDir+"/meta.properties")) + assertTrue(metaProps.containsKey("broker.id")) + assertEquals(metaProps.getInt("broker.id"),1001) + } + // restart the server check to see if it uses the brokerId generated previously + server1 = new KafkaServer(config1) + server1.startup() + assertEquals(server1.config.brokerId, 1001) + server1.shutdown() + Utils.rm(server1.config.logDirs) + + + // start the server with broker.id as part of config + var server2 = new KafkaServer(config2) + server2.startup() + assertEquals(server2.config.brokerId,0) + server2.shutdown() + Utils.rm(server2.config.logDirs) + + // add multiple logDirs and check if the generate brokerId is stored in all of them + props2 = TestUtils.createBrokerConfig(-1, port) + var logDirs = props2.getProperty("log.dir")+ "," + TestUtils.tempDir().getAbsolutePath + + "," + TestUtils.tempDir().getAbsolutePath + props2.setProperty("log.dir",logDirs) + config2 = new KafkaConfig(props2) + server1 = new KafkaServer(config2) + server1.startup() + server1.shutdown() + for(logDir <- config2.logDirs) { + val metaProps = new VerifiableProperties(Utils.loadProps(logDir+"/meta.properties")) + assertTrue(metaProps.containsKey("broker.id")) + assertEquals(metaProps.getInt("broker.id"),1002) + } + Utils.rm(server1.config.logDirs) + + // addition to log.dirs after generation of a broker.id from zk should be copied over + props2 = TestUtils.createBrokerConfig(-1,port) + config2 = new KafkaConfig(props2) + server1 = new KafkaServer(config2) + server1.startup() + server1.shutdown() + logDirs = props2.getProperty("log.dir")+","+TestUtils.tempDir().getAbsolutePath + server1.startup() + for(logDir <- config2.logDirs) { + val metaProps = new VerifiableProperties(Utils.loadProps(logDir+"/meta.properties")) + assertTrue(metaProps.containsKey("broker.id")) + assertEquals(metaProps.getInt("broker.id"),1003) + } + server1.shutdown() + // check if configured brokerId and stored brokerId are equal or throw InconsistentBrokerException + props2 = TestUtils.createBrokerConfig(0,port) + config2 = new KafkaConfig(props2) + try { + server1.startup() + } catch { + case e: kafka.common.InconsistentBrokerIdException => //success + } + server1.shutdown() + Utils.rm(server1.config.logDirs) + verifyNonDaemonThreadsStatus + } + + def verifyNonDaemonThreadsStatus() { + assertEquals(0, Thread.getAllStackTraces.keySet().toArray + .map(_.asInstanceOf[Thread]) + .count(t => !t.isDaemon && t.isAlive && t.getClass.getCanonicalName.toLowerCase.startsWith("kafka"))) + } + +} diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 3faa884..f992c29 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -90,7 +90,7 @@ object TestUtils extends Logging { Utils.rm(f) } }) - + f } @@ -150,7 +150,7 @@ object TestUtils extends Logging { def createBrokerConfig(nodeId: Int, port: Int = choosePort(), enableControlledShutdown: Boolean = true): Properties = { val props = new Properties - props.put("broker.id", nodeId.toString) + if (nodeId >= 0) props.put("broker.id", nodeId.toString) props.put("host.name", "localhost") props.put("port", port.toString) props.put("log.dir", TestUtils.tempDir().getAbsolutePath)