diff --git a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala deleted file mode 100644 index f40e19f..0000000 --- a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala +++ /dev/null @@ -1,55 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package kafka.api - -import java.nio.ByteBuffer -import org.apache.kafka.common.requests.AbstractRequestResponse -import kafka.api.ApiUtils._ - -private[kafka] abstract class GenericRequestAndHeader(val versionId: Short, - val correlationId: Int, - val clientId: String, - val body: AbstractRequestResponse, - val name: String, - override val requestId: Option[Short] = None) - extends RequestOrResponse(requestId) { - - def writeTo(buffer: ByteBuffer) { - buffer.putShort(versionId) - buffer.putInt(correlationId) - writeShortString(buffer, clientId) - body.writeTo(buffer) - } - - def sizeInBytes(): Int = { - 2 /* version id */ + - 4 /* correlation id */ + - (2 + clientId.length) /* client id */ + - body.sizeOf(); - } - - override def toString(): String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val strBuffer = new StringBuilder - strBuffer.append("Name: " + name) - strBuffer.append("; Version: " + versionId) - strBuffer.append("; CorrelationId: " + correlationId) - strBuffer.append("; ClientId: " + clientId) - strBuffer.append("; Body: " + body.toString) - strBuffer.toString() - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala new file mode 100644 index 0000000..fb022e8 --- /dev/null +++ b/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala @@ -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.api + +import java.nio.ByteBuffer +import org.apache.kafka.common.requests.AbstractRequestResponse + +private[kafka] abstract class GenericRequestOrResponseAndHeader(val header: AbstractRequestResponse, + val body: AbstractRequestResponse, + val name: String, + override val requestId: Option[Short] = None) + extends RequestOrResponse(requestId) { + + def writeTo(buffer: ByteBuffer) { + header.writeTo(buffer) + body.writeTo(buffer) + } + + def sizeInBytes(): Int = { + header.sizeOf() + body.sizeOf(); + } + + override def toString(): String = { + describe(true) + } + + override def describe(details: Boolean): String = { + val strBuffer = new StringBuilder + strBuffer.append("Name: " + name) + strBuffer.append("; header: " + header.toString) + strBuffer.append("; body: " + body.toString) + strBuffer.toString() + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala deleted file mode 100644 index a4879e2..0000000 --- a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala +++ /dev/null @@ -1,46 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package kafka.api - -import java.nio.ByteBuffer -import org.apache.kafka.common.requests.AbstractRequestResponse - -private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int, - val body: AbstractRequestResponse, - val name: String, - override val requestId: Option[Short] = None) - extends RequestOrResponse(requestId) { - - def writeTo(buffer: ByteBuffer) { - buffer.putInt(correlationId) - body.writeTo(buffer) - } - - def sizeInBytes(): Int = { - 4 /* correlation id */ + - body.sizeOf(); - } - - override def toString(): String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val strBuffer = new StringBuilder - strBuffer.append("Name: " + name) - strBuffer.append("; CorrelationId: " + correlationId) - strBuffer.append("; Body: " + body.toString) - strBuffer.toString() - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala index f168d9f..932418b 100644 --- a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala +++ b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala @@ -16,30 +16,24 @@ package kafka.api import java.nio.ByteBuffer import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping -import org.apache.kafka.common.requests.{HeartbeatResponse, HeartbeatRequest} -import kafka.api.ApiUtils._ import kafka.network.RequestChannel.Response -import scala.Some +import org.apache.kafka.common.requests.{HeartbeatResponse, ResponseHeader, HeartbeatRequest, RequestHeader} object HeartbeatRequestAndHeader { def readFrom(buffer: ByteBuffer): HeartbeatRequestAndHeader = { - val versionId = buffer.getShort - val correlationId = buffer.getInt - val clientId = readShortString(buffer) + val header = RequestHeader.parse(buffer) val body = HeartbeatRequest.parse(buffer) - new HeartbeatRequestAndHeader(versionId, correlationId, clientId, body) + new HeartbeatRequestAndHeader(header, body) } } -case class HeartbeatRequestAndHeader(override val versionId: Short, - override val correlationId: Int, - override val clientId: String, - override val body: HeartbeatRequest) - extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) { +case class HeartbeatRequestAndHeader(override val header: RequestHeader, override val body: HeartbeatRequest) + extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) { override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val errorResponseHeader = new ResponseHeader(header.correlationId) val errorResponseBody = new HeartbeatResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(correlationId, errorResponseBody) + val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(errorResponseHeader, errorResponseBody) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader))) } } diff --git a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala index 9a71faa..556f38d 100644 --- a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala +++ b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala @@ -12,17 +12,17 @@ */ package kafka.api -import org.apache.kafka.common.requests.HeartbeatResponse +import org.apache.kafka.common.requests.{ResponseHeader, HeartbeatResponse} import java.nio.ByteBuffer object HeartbeatResponseAndHeader { def readFrom(buffer: ByteBuffer): HeartbeatResponseAndHeader = { - val correlationId = buffer.getInt + val header = ResponseHeader.parse(buffer) val body = HeartbeatResponse.parse(buffer) - new HeartbeatResponseAndHeader(correlationId, body) + new HeartbeatResponseAndHeader(header, body) } } -case class HeartbeatResponseAndHeader(override val correlationId: Int, override val body: HeartbeatResponse) - extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) { +case class HeartbeatResponseAndHeader(override val header: ResponseHeader, override val body: HeartbeatResponse) + extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) { } diff --git a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala index 3651e86..9aea28c 100644 --- a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala +++ b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala @@ -17,29 +17,24 @@ import java.nio.ByteBuffer import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping import org.apache.kafka.common.requests._ -import kafka.api.ApiUtils._ import kafka.network.RequestChannel.Response import scala.Some object JoinGroupRequestAndHeader { def readFrom(buffer: ByteBuffer): JoinGroupRequestAndHeader = { - val versionId = buffer.getShort - val correlationId = buffer.getInt - val clientId = readShortString(buffer) + val header = RequestHeader.parse(buffer) val body = JoinGroupRequest.parse(buffer) - new JoinGroupRequestAndHeader(versionId, correlationId, clientId, body) + new JoinGroupRequestAndHeader(header, body) } } -case class JoinGroupRequestAndHeader(override val versionId: Short, - override val correlationId: Int, - override val clientId: String, - override val body: JoinGroupRequest) - extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) { +case class JoinGroupRequestAndHeader(override val header: RequestHeader, override val body: JoinGroupRequest) + extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) { override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val errorResponseHeader = new ResponseHeader(header.correlationId) val errorResponseBody = new JoinGroupResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(correlationId, errorResponseBody) + val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(errorResponseHeader, errorResponseBody) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader))) } } diff --git a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala index d0f07e0..7389ae6 100644 --- a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala +++ b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala @@ -12,17 +12,17 @@ */ package kafka.api -import org.apache.kafka.common.requests.JoinGroupResponse +import org.apache.kafka.common.requests.{JoinGroupResponse, ResponseHeader} import java.nio.ByteBuffer object JoinGroupResponseAndHeader { def readFrom(buffer: ByteBuffer): JoinGroupResponseAndHeader = { - val correlationId = buffer.getInt + val header = ResponseHeader.parse(buffer) val body = JoinGroupResponse.parse(buffer) - new JoinGroupResponseAndHeader(correlationId, body) + new JoinGroupResponseAndHeader(header, body) } } -case class JoinGroupResponseAndHeader(override val correlationId: Int, override val body: JoinGroupResponse) - extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) { +case class JoinGroupResponseAndHeader(override val header: ResponseHeader, override val body: JoinGroupResponse) + extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) { } 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..ac30f75 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -27,6 +27,8 @@ import kafka.utils.{VerifiableProperties, ZKConfig, Utils} */ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(props) { + val ReservedBrokerIdMaxValue = 1000 + def this(originalProps: Properties) { this(new VerifiableProperties(originalProps)) props.verify() @@ -35,13 +37,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 +51,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 +59,14 @@ 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)) + /* The broker id for this server. + * To avoid conflicts between zookeeper generated brokerId and user's config.brokerId + * added ReservedBrokerIdMaxValue and zookeeper sequence starts from ReservedBrokerIdMaxValue + 1. + */ + var brokerId: Int = if (props.containsKey("broker.id")) props.getIntInRange("broker.id", (0, ReservedBrokerIdMaxValue)) 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)) @@ -106,10 +111,10 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the maximum number of bytes in a socket request */ val socketRequestMaxBytes: Int = props.getIntInRange("socket.request.max.bytes", 100*1024*1024, (1, Int.MaxValue)) - + /* the maximum number of connections we allow from each ip address */ val maxConnectionsPerIp: Int = props.getIntInRange("max.connections.per.ip", Int.MaxValue, (1, Int.MaxValue)) - + /* per-ip or hostname overrides to the default maximum number of connections */ val maxConnectionsPerIpOverrides = props.getMap("max.connections.per.ip.overrides").map(entry => (entry._1, entry._2.toInt)) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index def1dc2..650afe3 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,62 @@ 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 metaBrokerIdOpt = readBrokerIdFromMetaProps(logDir) + metaBrokerIdOpt match { + case Some(metaBrokerId) => + metaBrokerIdSet.add(metaBrokerId) + case None => + logDirsWithoutMetaProps ++= List(logDir) + } + + } + + 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): Option[Int] = { + try { + val metaProps = new VerifiableProperties(Utils.loadProps(logDir + File.separator + metaPropsFile)) + if (metaProps.containsKey("broker.id")) + return Some(metaProps.getIntInRange("broker.id", (0, Int.MaxValue))) + } catch { + case e: FileNotFoundException => + None + } + None + } + + 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/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index cd16ced..847a36b 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -196,25 +196,29 @@ object SerializationTestUtils { } def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = { + val header = new RequestHeader(ApiKeys.HEARTBEAT.id, 0.asInstanceOf[Short], "", 1) val body = new HeartbeatRequest("group1", 1, "consumer1") - HeartbeatRequestAndHeader(0.asInstanceOf[Short], 1, "", body) + HeartbeatRequestAndHeader(header, body) } def createHeartbeatResponseAndHeader: HeartbeatResponseAndHeader = { + val header = new ResponseHeader(1) val body = new HeartbeatResponse(0.asInstanceOf[Short]) - HeartbeatResponseAndHeader(1, body) + HeartbeatResponseAndHeader(header, body) } def createJoinGroupRequestAndHeader: JoinGroupRequestAndHeader = { import scala.collection.JavaConversions._ + val header = new RequestHeader(ApiKeys.JOIN_GROUP.id, 0.asInstanceOf[Short], "", 1) val body = new JoinGroupRequest("group1", 30000, List("topic1"), "consumer1", "strategy1"); - JoinGroupRequestAndHeader(0.asInstanceOf[Short], 1, "", body) + JoinGroupRequestAndHeader(header, body) } def createJoinGroupResponseAndHeader: JoinGroupResponseAndHeader = { import scala.collection.JavaConversions._ + val header = new ResponseHeader(1) val body = new JoinGroupResponse(0.asInstanceOf[Short], 1, "consumer1", List(new TopicPartition("test11", 1))) - JoinGroupResponseAndHeader(1, body) + JoinGroupResponseAndHeader(header, body) } } 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)