diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 978447d..1b66655 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -65,3 +65,19 @@ else fi $JAVA $KAFKA_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH "$@" + +exitval=$? + +if [ $exitval -eq "1" ] ; then + $JAVA $KAFKA_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH "$@" >& exception.txt + exception=`cat exception.txt` + noBuildMessage='Please build the project using sbt. Documentation is available at http://kafka.apache.org/' + pattern="(Could not find or load main class)|(java\.lang\.NoClassDefFoundError)" + match=`echo $exception | grep -E "$pattern"` + if [[ -n "$match" ]]; then + echo $noBuildMessage + fi + rm exception.txt +fi + + diff --git a/config/server.properties b/config/server.properties index f9d54bc..01e0b12 100644 --- a/config/server.properties +++ b/config/server.properties @@ -52,7 +52,7 @@ log.dir=/tmp/kafka-logs # The number of logical partitions per topic per server. More partitions allow greater parallelism # for consumption, but also mean more files. -num.partitions=1 +num.partitions=2 ############################# Log Flush Policy ############################# diff --git a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java index 32f096c..0b435b9 100644 --- a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java +++ b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java @@ -49,9 +49,9 @@ public class KafkaOutputFormat extends OutputFormat Map cMap = new HashMap(); // default Hadoop producer configs - cMap.put("producer.type", "sync"); - cMap.put("send.buffer.bytes", Integer.toString(64*1024)); - cMap.put("compression.codec", Integer.toString(1)); + cMap.put("producer.type", "sync"); + cMap.put("compression.codec", Integer.toString(1)); + cMap.put("request.required.acks", Integer.toString(1)); kafkaConfigMap = Collections.unmodifiableMap(cMap); } diff --git a/core/build.sbt b/core/build.sbt index 405ea55..c54cf44 100644 --- a/core/build.sbt +++ b/core/build.sbt @@ -12,7 +12,7 @@ libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _ ) libraryDependencies ++= Seq( "org.apache.zookeeper" % "zookeeper" % "3.3.4", - "com.101tec" % "zkclient" % "0.2", + "com.101tec" % "zkclient" % "0.3", "org.xerial.snappy" % "snappy-java" % "1.0.4.1", "com.yammer.metrics" % "metrics-core" % "2.2.0", "com.yammer.metrics" % "metrics-annotation" % "2.2.0", diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index a1c485e..09568e9 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -21,17 +21,14 @@ import java.util.Random import java.util.Properties import kafka.api.{TopicMetadata, PartitionMetadata} import kafka.cluster.Broker -import kafka.utils.{Logging, ZkUtils} import kafka.log.LogConfig -import kafka.server.TopicConfigManager -import kafka.utils.{Logging, Utils, ZkUtils, Json} +import kafka.utils.{Logging, ZkUtils, Json} import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException import scala.collection._ import mutable.ListBuffer import scala.collection.mutable import kafka.common._ -import scala.Some object AdminUtils extends Logging { val rand = new Random @@ -77,7 +74,7 @@ object AdminUtils extends Logging { val firstReplicaIndex = (i + startIndex) % brokers.size var replicaList = List(brokers(firstReplicaIndex)) for (j <- 0 until replicationFactor - 1) - replicaList ::= brokers(replicaIndex(firstReplicaIndex, secondReplicaShift, j, brokers.size)) + replicaList ::= brokers(getWrappedIndex(firstReplicaIndex, secondReplicaShift, j, brokers.size)) ret.put(i, replicaList.reverse) } ret.toMap @@ -267,7 +264,7 @@ object AdminUtils extends Logging { brokerMetadata.filter(_.isDefined).map(_.get) } - private def replicaIndex(firstReplicaIndex: Int, secondReplicaShift: Int, replicaIndex: Int, nBrokers: Int): Int = { + private def getWrappedIndex(firstReplicaIndex: Int, secondReplicaShift: Int, replicaIndex: Int, nBrokers: Int): Int = { val shift = 1 + (secondReplicaShift + replicaIndex) % (nBrokers - 1) (firstReplicaIndex + shift) % nBrokers } diff --git a/core/src/main/scala/kafka/admin/CheckReassignmentStatus.scala b/core/src/main/scala/kafka/admin/CheckReassignmentStatus.scala index 9e8ccc3..7e85f87 100644 --- a/core/src/main/scala/kafka/admin/CheckReassignmentStatus.scala +++ b/core/src/main/scala/kafka/admin/CheckReassignmentStatus.scala @@ -1,4 +1,19 @@ - +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package kafka.admin import joptsimple.OptionParser diff --git a/core/src/main/scala/kafka/admin/ShutdownBroker.scala b/core/src/main/scala/kafka/admin/ShutdownBroker.scala index bb20edb..2dd47e7 100644 --- a/core/src/main/scala/kafka/admin/ShutdownBroker.scala +++ b/core/src/main/scala/kafka/admin/ShutdownBroker.scala @@ -25,7 +25,7 @@ import javax.management.remote.{JMXServiceURL, JMXConnectorFactory} import javax.management.ObjectName import kafka.controller.KafkaController import scala.Some -import kafka.common.BrokerNotAvailableException +import kafka.common.{TopicAndPartition, BrokerNotAvailableException} object ShutdownBroker extends Logging { @@ -58,8 +58,8 @@ object ShutdownBroker extends Logging { val leaderPartitionsRemaining = mbsc.invoke(new ObjectName(KafkaController.MBeanName), "shutdownBroker", Array(params.brokerId), - Array(classOf[Int].getName)).asInstanceOf[Int] - val shutdownComplete = (leaderPartitionsRemaining == 0) + Array(classOf[Int].getName)).asInstanceOf[Set[TopicAndPartition]] + val shutdownComplete = (leaderPartitionsRemaining.size == 0) info("Shutdown status: " + (if (shutdownComplete) "complete" else "incomplete (broker still leads %d partitions)".format(leaderPartitionsRemaining))) shutdownComplete diff --git a/core/src/main/scala/kafka/api/ApiUtils.scala b/core/src/main/scala/kafka/api/ApiUtils.scala index a6b0328..1f80de1 100644 --- a/core/src/main/scala/kafka/api/ApiUtils.scala +++ b/core/src/main/scala/kafka/api/ApiUtils.scala @@ -1,4 +1,20 @@ -package kafka.api +/** + * 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.common._ diff --git a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala new file mode 100644 index 0000000..ad6a20d --- /dev/null +++ b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer +import kafka.api.ApiUtils._ +import collection.mutable.ListBuffer +import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.common.{TopicAndPartition, ErrorMapping} +import kafka.network.RequestChannel.Response +import kafka.utils.Logging + +object ControlledShutdownRequest extends Logging { + val CurrentVersion = 0.shortValue + val DefaultClientId = "" + + def readFrom(buffer: ByteBuffer): ControlledShutdownRequest = { + val versionId = buffer.getShort + val correlationId = buffer.getInt + val brokerId = buffer.getInt + new ControlledShutdownRequest(versionId, correlationId, brokerId) + } +} + +case class ControlledShutdownRequest(val versionId: Short, + override val correlationId: Int, + val brokerId: Int) + extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey), correlationId){ + + def this(correlationId: Int, brokerId: Int) = + this(ControlledShutdownRequest.CurrentVersion, correlationId, brokerId) + + def writeTo(buffer: ByteBuffer) { + buffer.putShort(versionId) + buffer.putInt(correlationId) + buffer.putInt(brokerId) + } + + def sizeInBytes(): Int = { + 2 + /* version id */ + 4 + /* correlation id */ + 4 /* broker id */ + } + + override def toString(): String = { + val controlledShutdownRequest = new StringBuilder + controlledShutdownRequest.append("Name: " + this.getClass.getSimpleName) + controlledShutdownRequest.append("; Version: " + versionId) + controlledShutdownRequest.append("; CorrelationId: " + correlationId) + controlledShutdownRequest.append("; BrokerId: " + brokerId) + controlledShutdownRequest.toString() + } + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val errorResponse = ControlledShutdownResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]]), Set.empty[TopicAndPartition]) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } +} diff --git a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala new file mode 100644 index 0000000..b7c8448 --- /dev/null +++ b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala @@ -0,0 +1,70 @@ +/** + * 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 collection.mutable.HashMap +import collection.immutable.Map +import kafka.common.{TopicAndPartition, ErrorMapping} +import kafka.api.ApiUtils._ + + +object ControlledShutdownResponse { + def readFrom(buffer: ByteBuffer): ControlledShutdownResponse = { + val correlationId = buffer.getInt + val errorCode = buffer.getShort + val numEntries = buffer.getInt + + var partitionsRemaining = Set[TopicAndPartition]() + for (i<- 0 until numEntries){ + val topic = readShortString(buffer) + val partition = buffer.getInt + partitionsRemaining += new TopicAndPartition(topic, partition) + } + new ControlledShutdownResponse(correlationId, errorCode, partitionsRemaining) + } +} + + +case class ControlledShutdownResponse(override val correlationId: Int, + val errorCode: Short = ErrorMapping.NoError, + val partitionsRemaining: Set[TopicAndPartition]) + extends RequestOrResponse(correlationId = correlationId) { + def sizeInBytes(): Int ={ + var size = + 4 /* correlation id */ + + 2 /* error code */ + + 4 /* number of responses */ + for (topicAndPartition <- partitionsRemaining) { + size += + 2 + topicAndPartition.topic.length /* topic */ + + 4 /* partition */ + } + size + } + + def writeTo(buffer: ByteBuffer) { + buffer.putInt(correlationId) + buffer.putShort(errorCode) + buffer.putInt(partitionsRemaining.size) + for (topicAndPartition:TopicAndPartition <- partitionsRemaining){ + writeShortString(buffer, topicAndPartition.topic) + buffer.putInt(topicAndPartition.partition) + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index 68e64d6..a474474 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -50,23 +50,29 @@ object PartitionStateInfo { val controllerEpoch = buffer.getInt val leader = buffer.getInt val leaderEpoch = buffer.getInt - val isrString = readShortString(buffer) - val isr = Utils.parseCsvList(isrString).map(_.toInt).toList + val isrSize = buffer.getInt + val isr = for(i <- 0 until isrSize) yield buffer.getInt val zkVersion = buffer.getInt val replicationFactor = buffer.getInt - PartitionStateInfo(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, leaderEpoch, isr, zkVersion), controllerEpoch), - replicationFactor) + val replicas = for(i <- 0 until replicationFactor) yield buffer.getInt + PartitionStateInfo(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, leaderEpoch, isr.toList, zkVersion), controllerEpoch), + replicas.toSet) } } -case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, val replicationFactor: Int) { +case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, + val allReplicas: Set[Int]) { + def replicationFactor = allReplicas.size + def writeTo(buffer: ByteBuffer) { buffer.putInt(leaderIsrAndControllerEpoch.controllerEpoch) buffer.putInt(leaderIsrAndControllerEpoch.leaderAndIsr.leader) buffer.putInt(leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch) - writeShortString(buffer, leaderIsrAndControllerEpoch.leaderAndIsr.isr.mkString(",")) + buffer.putInt(leaderIsrAndControllerEpoch.leaderAndIsr.isr.size) + leaderIsrAndControllerEpoch.leaderAndIsr.isr.foreach(buffer.putInt(_)) buffer.putInt(leaderIsrAndControllerEpoch.leaderAndIsr.zkVersion) buffer.putInt(replicationFactor) + allReplicas.foreach(buffer.putInt(_)) } def sizeInBytes(): Int = { @@ -74,9 +80,11 @@ case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndContr 4 /* epoch of the controller that elected the leader */ + 4 /* leader broker id */ + 4 /* leader epoch */ + - (2 + leaderIsrAndControllerEpoch.leaderAndIsr.isr.mkString(",").length) + + 4 /* number of replicas in isr */ + + 4 * leaderIsrAndControllerEpoch.leaderAndIsr.isr.size /* replicas in isr */ + 4 /* zk version */ + - 4 /* replication factor */ + 4 /* replication factor */ + + allReplicas.size * 4 size } @@ -84,6 +92,7 @@ case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndContr val partitionStateInfo = new StringBuilder partitionStateInfo.append("(LeaderAndIsrInfo:" + leaderIsrAndControllerEpoch.toString) partitionStateInfo.append(",ReplicationFactor:" + replicationFactor + ")") + partitionStateInfo.append(",AllReplicas:" + allReplicas.mkString(",") + ")") partitionStateInfo.toString() } } @@ -98,7 +107,6 @@ object LeaderAndIsrRequest { val versionId = buffer.getShort val correlationId = buffer.getInt val clientId = readShortString(buffer) - val ackTimeoutMs = buffer.getInt val controllerId = buffer.getInt val controllerEpoch = buffer.getInt val partitionStateInfosCount = buffer.getInt @@ -117,14 +125,13 @@ object LeaderAndIsrRequest { for (i <- 0 until leadersCount) leaders += Broker.readFrom(buffer) - new LeaderAndIsrRequest(versionId, correlationId, clientId, ackTimeoutMs, controllerId, controllerEpoch, partitionStateInfos.toMap, leaders) + new LeaderAndIsrRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos.toMap, leaders) } } case class LeaderAndIsrRequest (versionId: Short, override val correlationId: Int, clientId: String, - ackTimeoutMs: Int, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[(String, Int), PartitionStateInfo], @@ -133,7 +140,7 @@ case class LeaderAndIsrRequest (versionId: Short, def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], aliveLeaders: Set[Broker], controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String) = { - this(LeaderAndIsrRequest.CurrentVersion, correlationId, clientId, LeaderAndIsrRequest.DefaultAckTimeout, + this(LeaderAndIsrRequest.CurrentVersion, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos, aliveLeaders) } @@ -141,7 +148,6 @@ case class LeaderAndIsrRequest (versionId: Short, buffer.putShort(versionId) buffer.putInt(correlationId) writeShortString(buffer, clientId) - buffer.putInt(ackTimeoutMs) buffer.putInt(controllerId) buffer.putInt(controllerEpoch) buffer.putInt(partitionStateInfos.size) @@ -159,7 +165,6 @@ case class LeaderAndIsrRequest (versionId: Short, 2 /* version id */ + 4 /* correlation id */ + (2 + clientId.length) /* client id */ + - 4 /* ack timeout */ + 4 /* controller id */ + 4 /* controller epoch */ + 4 /* number of partitions */ @@ -179,7 +184,6 @@ case class LeaderAndIsrRequest (versionId: Short, leaderAndIsrRequest.append(";ControllerEpoch:" + controllerEpoch) leaderAndIsrRequest.append(";CorrelationId:" + correlationId) leaderAndIsrRequest.append(";ClientId:" + clientId) - leaderAndIsrRequest.append(";AckTimeoutMs:" + ackTimeoutMs + " ms") leaderAndIsrRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) leaderAndIsrRequest.append(";Leaders:" + aliveLeaders.mkString(",")) leaderAndIsrRequest.toString() diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala index b4cfae8..378b2b3 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala @@ -63,7 +63,7 @@ case class LeaderAndIsrResponse(override val correlationId: Int, buffer.putInt(correlationId) buffer.putShort(errorCode) buffer.putInt(responseMap.size) - for ((key:(String, Int), value) <- responseMap){ + for ((key:(String, Int), value) <- responseMap) { writeShortString(buffer, key._1) buffer.putInt(key._2) buffer.putShort(value) diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala index 32ebfd4..0a94a6c 100644 --- a/core/src/main/scala/kafka/api/OffsetRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetRequest.scala @@ -49,7 +49,7 @@ object OffsetRequest { (TopicAndPartition(topic, partitionId), PartitionOffsetRequestInfo(time, maxNumOffsets)) }) }) - OffsetRequest(Map(pairs:_*), versionId = versionId, clientId = clientId, replicaId = replicaId) + OffsetRequest(Map(pairs:_*), versionId= versionId, clientId = clientId, correlationId = correlationId, replicaId = replicaId) } } diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala index 89ce92a..c81214f 100644 --- a/core/src/main/scala/kafka/api/RequestKeys.scala +++ b/core/src/main/scala/kafka/api/RequestKeys.scala @@ -27,8 +27,10 @@ object RequestKeys { val MetadataKey: Short = 3 val LeaderAndIsrKey: Short = 4 val StopReplicaKey: Short = 5 - val OffsetCommitKey: Short = 6 - val OffsetFetchKey: Short = 7 + val UpdateMetadataKey: Short = 6 + val ControlledShutdownKey: Short = 7 + val OffsetCommitKey: Short = 8 + val OffsetFetchKey: Short = 9 val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]= Map(ProduceKey -> ("Produce", ProducerRequest.readFrom), @@ -37,6 +39,8 @@ object RequestKeys { MetadataKey -> ("Metadata", TopicMetadataRequest.readFrom), LeaderAndIsrKey -> ("LeaderAndIsr", LeaderAndIsrRequest.readFrom), StopReplicaKey -> ("StopReplica", StopReplicaRequest.readFrom), + UpdateMetadataKey -> ("UpdateMetadata", UpdateMetadataRequest.readFrom), + ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom), OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom), OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom)) diff --git a/core/src/main/scala/kafka/api/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala index cd55db4..efd7046 100644 --- a/core/src/main/scala/kafka/api/StopReplicaRequest.scala +++ b/core/src/main/scala/kafka/api/StopReplicaRequest.scala @@ -35,7 +35,6 @@ object StopReplicaRequest extends Logging { val versionId = buffer.getShort val correlationId = buffer.getInt val clientId = readShortString(buffer) - val ackTimeoutMs = buffer.getInt val controllerId = buffer.getInt val controllerEpoch = buffer.getInt val deletePartitions = buffer.get match { @@ -49,7 +48,7 @@ object StopReplicaRequest extends Logging { (1 to topicPartitionPairCount) foreach { _ => topicPartitionPairSet.add(readShortString(buffer), buffer.getInt) } - StopReplicaRequest(versionId, correlationId, clientId, ackTimeoutMs, controllerId, controllerEpoch, + StopReplicaRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, deletePartitions, topicPartitionPairSet.toSet) } } @@ -57,7 +56,6 @@ object StopReplicaRequest extends Logging { case class StopReplicaRequest(versionId: Short, override val correlationId: Int, clientId: String, - ackTimeoutMs: Int, controllerId: Int, controllerEpoch: Int, deletePartitions: Boolean, @@ -65,7 +63,7 @@ case class StopReplicaRequest(versionId: Short, extends RequestOrResponse(Some(RequestKeys.StopReplicaKey), correlationId) { def this(deletePartitions: Boolean, partitions: Set[(String, Int)], controllerId: Int, controllerEpoch: Int, correlationId: Int) = { - this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId, StopReplicaRequest.DefaultAckTimeout, + this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId, controllerId, controllerEpoch, deletePartitions, partitions) } @@ -73,7 +71,6 @@ case class StopReplicaRequest(versionId: Short, buffer.putShort(versionId) buffer.putInt(correlationId) writeShortString(buffer, clientId) - buffer.putInt(ackTimeoutMs) buffer.putInt(controllerId) buffer.putInt(controllerEpoch) buffer.put(if (deletePartitions) 1.toByte else 0.toByte) @@ -89,7 +86,6 @@ case class StopReplicaRequest(versionId: Short, 2 + /* versionId */ 4 + /* correlation id */ ApiUtils.shortStringLength(clientId) + - 4 + /* ackTimeoutMs */ 4 + /* controller id*/ 4 + /* controller epoch */ 1 + /* deletePartitions */ @@ -107,7 +103,6 @@ case class StopReplicaRequest(versionId: Short, stopReplicaRequest.append("; Version: " + versionId) stopReplicaRequest.append("; CorrelationId: " + correlationId) stopReplicaRequest.append("; ClientId: " + clientId) - stopReplicaRequest.append("; AckTimeoutMs: " + ackTimeoutMs + " ms") stopReplicaRequest.append("; DeletePartitions: " + deletePartitions) stopReplicaRequest.append("; ControllerId: " + controllerId) stopReplicaRequest.append("; ControllerEpoch: " + controllerEpoch) diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index a0d68c5..0513a59 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -55,6 +55,34 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat buffer.putInt(partitionsMetadata.size) partitionsMetadata.foreach(m => m.writeTo(buffer)) } + + override def toString(): String = { + val topicMetadataInfo = new StringBuilder + topicMetadataInfo.append("{TopicMetadata for topic %s -> ".format(topic)) + errorCode match { + case ErrorMapping.NoError => + partitionsMetadata.foreach { partitionMetadata => + partitionMetadata.errorCode match { + case ErrorMapping.NoError => + topicMetadataInfo.append("\nMetadata for partition [%s,%d] is %s".format(topic, + partitionMetadata.partitionId, partitionMetadata.toString())) + case ErrorMapping.ReplicaNotAvailableCode => + // this error message means some replica other than the leader is not available. The consumer + // doesn't care about non leader replicas, so ignore this + topicMetadataInfo.append("\nMetadata for partition [%s,%d] is %s".format(topic, + partitionMetadata.partitionId, partitionMetadata.toString())) + case _ => + topicMetadataInfo.append("\nMetadata for partition [%s,%d] is not available due to %s".format(topic, + partitionMetadata.partitionId, ErrorMapping.exceptionFor(partitionMetadata.errorCode).getClass.getName)) + } + } + case _ => + topicMetadataInfo.append("\nNo partition metadata for topic %s due to %s".format(topic, + ErrorMapping.exceptionFor(errorCode).getClass.getName)) + } + topicMetadataInfo.append("}") + topicMetadataInfo.toString() + } } object PartitionMetadata { diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 7477cfd..c5221c4 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -81,7 +81,7 @@ case class TopicMetadataRequest(val versionId: Short, topicMetadataRequest.toString() } - override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { val topicMetadata = topics.map { topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala new file mode 100644 index 0000000..2ead364 --- /dev/null +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -0,0 +1,120 @@ +/** + * 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.api.ApiUtils._ +import kafka.cluster.Broker +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.network.RequestChannel.Response + +object UpdateMetadataRequest { + val CurrentVersion = 0.shortValue + val IsInit: Boolean = true + val NotInit: Boolean = false + val DefaultAckTimeout: Int = 1000 + + def readFrom(buffer: ByteBuffer): UpdateMetadataRequest = { + val versionId = buffer.getShort + val correlationId = buffer.getInt + val clientId = readShortString(buffer) + val controllerId = buffer.getInt + val controllerEpoch = buffer.getInt + val partitionStateInfosCount = buffer.getInt + val partitionStateInfos = new collection.mutable.HashMap[TopicAndPartition, PartitionStateInfo] + + for(i <- 0 until partitionStateInfosCount){ + val topic = readShortString(buffer) + val partition = buffer.getInt + val partitionStateInfo = PartitionStateInfo.readFrom(buffer) + + partitionStateInfos.put(TopicAndPartition(topic, partition), partitionStateInfo) + } + + val numAliveBrokers = buffer.getInt + val aliveBrokers = for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer) + new UpdateMetadataRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, + partitionStateInfos.toMap, aliveBrokers.toSet) + } +} + +case class UpdateMetadataRequest (versionId: Short, + override val correlationId: Int, + clientId: String, + controllerId: Int, + controllerEpoch: Int, + partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], + aliveBrokers: Set[Broker]) + extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey), correlationId) { + + def this(controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String, + partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker]) = { + this(UpdateMetadataRequest.CurrentVersion, correlationId, clientId, + controllerId, controllerEpoch, partitionStateInfos, aliveBrokers) + } + + def writeTo(buffer: ByteBuffer) { + buffer.putShort(versionId) + buffer.putInt(correlationId) + writeShortString(buffer, clientId) + buffer.putInt(controllerId) + buffer.putInt(controllerEpoch) + buffer.putInt(partitionStateInfos.size) + for((key, value) <- partitionStateInfos){ + writeShortString(buffer, key.topic) + buffer.putInt(key.partition) + value.writeTo(buffer) + } + buffer.putInt(aliveBrokers.size) + aliveBrokers.foreach(_.writeTo(buffer)) + } + + def sizeInBytes(): Int = { + var size = + 2 /* version id */ + + 4 /* correlation id */ + + (2 + clientId.length) /* client id */ + + 4 /* controller id */ + + 4 /* controller epoch */ + + 4 /* number of partitions */ + for((key, value) <- partitionStateInfos) + size += (2 + key.topic.length) /* topic */ + 4 /* partition */ + value.sizeInBytes /* partition state info */ + size += 4 /* number of alive brokers in the cluster */ + for(broker <- aliveBrokers) + size += broker.sizeInBytes /* broker info */ + size + } + + override def toString(): String = { + val updateMetadataRequest = new StringBuilder + updateMetadataRequest.append("Name:" + this.getClass.getSimpleName) + updateMetadataRequest.append(";Version:" + versionId) + updateMetadataRequest.append(";Controller:" + controllerId) + updateMetadataRequest.append(";ControllerEpoch:" + controllerEpoch) + updateMetadataRequest.append(";CorrelationId:" + correlationId) + updateMetadataRequest.append(";ClientId:" + clientId) + updateMetadataRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) + updateMetadataRequest.append(";AliveBrokers:" + aliveBrokers.mkString(",")) + updateMetadataRequest.toString() + } + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val errorResponse = new UpdateMetadataResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]])) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } +} diff --git a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala new file mode 100644 index 0000000..b1e42c3 --- /dev/null +++ b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala @@ -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.api + +import kafka.common.{TopicAndPartition, ErrorMapping} +import java.nio.ByteBuffer +import kafka.api.ApiUtils._ +import collection.mutable.HashMap +import collection.Map + + +object UpdateMetadataResponse { + def readFrom(buffer: ByteBuffer): UpdateMetadataResponse = { + val correlationId = buffer.getInt + val errorCode = buffer.getShort + new UpdateMetadataResponse(correlationId, errorCode) + } +} + +case class UpdateMetadataResponse(override val correlationId: Int, + errorCode: Short = ErrorMapping.NoError) + extends RequestOrResponse(correlationId = correlationId) { + def sizeInBytes(): Int = 4 /* correlation id */ + 2 /* error code */ + + def writeTo(buffer: ByteBuffer) { + buffer.putInt(correlationId) + buffer.putShort(errorCode) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 025d3ab..cc526ec 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -1,4 +1,20 @@ -package kafka.client +/** + * 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.client import scala.collection._ import kafka.cluster._ @@ -47,7 +63,7 @@ object ClientUtils extends Logging{ producer.close() } } - if(!fetchMetaDataSucceeded){ + if(!fetchMetaDataSucceeded) { throw new KafkaException("fetching topic metadata for topics [%s] from broker [%s] failed".format(topics, shuffledBrokers), t) } else { debug("Successfully fetched metadata for %d topic(s) %s".format(topics.size, topics)) @@ -62,13 +78,14 @@ object ClientUtils extends Logging{ * @param clientId The client's identifier * @return topic metadata response */ - def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String, timeoutMs: Int): TopicMetadataResponse = { + def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String, timeoutMs: Int, + correlationId: Int = 0): TopicMetadataResponse = { val props = new Properties() props.put("metadata.broker.list", brokers.map(_.getConnectionString()).mkString(",")) props.put("client.id", clientId) props.put("request.timeout.ms", timeoutMs.toString) val producerConfig = new ProducerConfig(props) - fetchTopicMetadata(topics, brokers, producerConfig, 0) + fetchTopicMetadata(topics, brokers, producerConfig, correlationId) } /** diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index ce2a634..4b803bb 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -201,8 +201,15 @@ class Partition(val topic: String, leaderEpoch = leaderAndIsr.leaderEpoch zkVersion = leaderAndIsr.zkVersion leaderReplicaIdOpt = Some(newLeaderBrokerId) - // start fetcher thread to current leader - replicaFetcherManager.addFetcher(topic, partitionId, localReplica.logEndOffset, leaderBroker) + if (!replicaManager.isShuttingDown.get()) { + // start fetcher thread to current leader if we are not shutting down + replicaFetcherManager.addFetcher(topic, partitionId, localReplica.logEndOffset, leaderBroker) + } + else { + stateChangeLogger.trace("Broker %d ignored the become-follower state change with correlation id %d from " + + " controller %d epoch %d since it is shutting down" + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch)) + } case None => // leader went down stateChangeLogger.trace("Broker %d aborted the become-follower state change with correlation id %d from " + " controller %d epoch %d since leader %d for partition [%s,%d] is unavailable during the state change operation" diff --git a/core/src/main/scala/kafka/common/BrokerNotExistException.scala b/core/src/main/scala/kafka/common/BrokerNotExistException.scala index e69de29..487329a 100644 --- a/core/src/main/scala/kafka/common/BrokerNotExistException.scala +++ b/core/src/main/scala/kafka/common/BrokerNotExistException.scala @@ -0,0 +1,16 @@ +/** + * 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. + */ \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/InvalidOffsetException.scala b/core/src/main/scala/kafka/common/InvalidOffsetException.scala new file mode 100644 index 0000000..c6811d7 --- /dev/null +++ b/core/src/main/scala/kafka/common/InvalidOffsetException.scala @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +class InvalidOffsetException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/UnknownTopicException.scala b/core/src/main/scala/kafka/common/UnknownTopicException.scala deleted file mode 100644 index 710d3bf..0000000 --- a/core/src/main/scala/kafka/common/UnknownTopicException.scala +++ /dev/null @@ -1,25 +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.common - -/** - * Thrown when a request is made for a topic, that hasn't been created in a Kafka cluster - */ -class UnknownTopicException(message: String) extends RuntimeException(message) { - def this() = this(null) -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala index 576ff47..140f2e3 100644 --- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala @@ -84,6 +84,11 @@ object ConsoleConsumer extends Logging { .describedAs("ms") .ofType(classOf[java.lang.Integer]) .defaultsTo(ConsumerConfig.SocketTimeout) + val refreshMetadataBackoffMsOpt = parser.accepts("refresh-leader-backoff-ms", "Backoff time before refreshing metadata") + .withRequiredArg + .describedAs("ms") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(ConsumerConfig.RefreshMetadataBackoffMs) val consumerTimeoutMsOpt = parser.accepts("consumer-timeout-ms", "consumer throws timeout exception after waiting this much " + "of time without incoming messages") .withRequiredArg @@ -105,7 +110,7 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("ms") .ofType(classOf[java.lang.Integer]) - .defaultsTo(10*1000) + .defaultsTo(ConsumerConfig.AutoCommitInterval) val maxMessagesOpt = parser.accepts("max-messages", "The maximum number of messages to consume before exiting. If not set, consumption is continual.") .withRequiredArg .describedAs("num_messages") @@ -160,6 +165,8 @@ object ConsoleConsumer extends Logging { props.put("auto.offset.reset", if(options.has(resetBeginningOpt)) "smallest" else "largest") props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) props.put("consumer.timeout.ms", options.valueOf(consumerTimeoutMsOpt).toString) + props.put("refresh.leader.backoff.ms", options.valueOf(refreshMetadataBackoffMsOpt).toString) + val config = new ConsumerConfig(props) val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index e9cfd10..c8c4212 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -23,6 +23,7 @@ import kafka.utils._ import kafka.common.{InvalidConfigException, Config} object ConsumerConfig extends Config { + val RefreshMetadataBackoffMs = 200 val SocketTimeout = 30 * 1000 val SocketBufferSize = 64*1024 val FetchSize = 1024 * 1024 @@ -30,9 +31,9 @@ object ConsumerConfig extends Config { val DefaultFetcherBackoffMs = 1000 val AutoCommit = true val AutoCommitInterval = 60 * 1000 - val MaxQueuedChunks = 10 + val MaxQueuedChunks = 2 val MaxRebalanceRetries = 4 - val AutoOffsetReset = OffsetRequest.SmallestTimeString + val AutoOffsetReset = OffsetRequest.LargestTimeString val ConsumerTimeoutMs = -1 val MinFetchBytes = 1 val MaxFetchWaitMs = 100 @@ -99,8 +100,8 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** the frequency in ms that the consumer offsets are committed to zookeeper */ val autoCommitIntervalMs = props.getInt("auto.commit.interval.ms", AutoCommitInterval) - /** max number of messages buffered for consumption */ - val queuedMaxMessages = props.getInt("queued.max.messages", MaxQueuedChunks) + /** max number of message chunks buffered for consumption, each chunk can be up to fetch.message.max.bytes*/ + val queuedMaxMessages = props.getInt("queued.max.message.chunks", MaxQueuedChunks) /** max number of retries during rebalance */ val rebalanceMaxRetries = props.getInt("rebalance.max.retries", MaxRebalanceRetries) @@ -115,7 +116,7 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( val rebalanceBackoffMs = props.getInt("rebalance.backoff.ms", zkSyncTimeMs) /** backoff time to refresh the leader of a partition after it loses the current leader */ - val refreshLeaderBackoffMs = props.getInt("refresh.leader.backoff.ms", 200) + val refreshLeaderBackoffMs = props.getInt("refresh.leader.backoff.ms", RefreshMetadataBackoffMs) /* what to do if an offset is out of range. smallest : automatically reset the offset to the smallest offset diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index c6250dc..71ae640 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -28,6 +28,7 @@ import kafka.utils.ZkUtils._ import kafka.utils.{ShutdownableThread, SystemTime} import kafka.common.TopicAndPartition import kafka.client.ClientUtils +import java.util.concurrent.atomic.AtomicInteger /** * Usage: @@ -37,17 +38,20 @@ import kafka.client.ClientUtils class ConsumerFetcherManager(private val consumerIdString: String, private val config: ConsumerConfig, private val zkClient : ZkClient) - extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(SystemTime.milliseconds), 1) { + extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(SystemTime.milliseconds), + config.groupId, 1) { private var partitionMap: immutable.Map[TopicAndPartition, PartitionTopicInfo] = null private var cluster: Cluster = null private val noLeaderPartitionSet = new mutable.HashSet[TopicAndPartition] private val lock = new ReentrantLock private val cond = lock.newCondition() private var leaderFinderThread: ShutdownableThread = null + private val correlationId = new AtomicInteger(0) private class LeaderFinderThread(name: String) extends ShutdownableThread(name) { // thread responsible for adding the fetcher to the right broker when leader is available override def doWork() { + val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker] lock.lock() try { if (noLeaderPartitionSet.isEmpty) { @@ -55,45 +59,43 @@ class ConsumerFetcherManager(private val consumerIdString: String, cond.await() } - try { - trace("Partitions without leader %s".format(noLeaderPartitionSet)) - val brokers = getAllBrokersInCluster(zkClient) - val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, - brokers, - config.clientId, - config.socketTimeoutMs).topicsMetadata - val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker] - topicsMetadata.foreach( - tmd => { - val topic = tmd.topic - tmd.partitionsMetadata.foreach( - pmd => { - val topicAndPartition = TopicAndPartition(topic, pmd.partitionId) - if(pmd.leader.isDefined && noLeaderPartitionSet.contains(topicAndPartition)) { - val leaderBroker = pmd.leader.get - leaderForPartitionsMap.put(topicAndPartition, leaderBroker) - } - }) - }) - - leaderForPartitionsMap.foreach{ - case(topicAndPartition, leaderBroker) => - val pti = partitionMap(topicAndPartition) - try { - addFetcher(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset(), leaderBroker) - noLeaderPartitionSet -= topicAndPartition - } catch { - case t => warn("Failed to add fetcher for %s to broker %s".format(topicAndPartition, leaderBroker), t) - } + trace("Partitions without leader %s".format(noLeaderPartitionSet)) + val brokers = getAllBrokersInCluster(zkClient) + val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, + brokers, + config.clientId, + config.socketTimeoutMs, + correlationId.getAndIncrement).topicsMetadata + if(logger.isDebugEnabled) topicsMetadata.foreach(topicMetadata => debug(topicMetadata.toString())) + topicsMetadata.foreach { tmd => + val topic = tmd.topic + tmd.partitionsMetadata.foreach { pmd => + val topicAndPartition = TopicAndPartition(topic, pmd.partitionId) + if(pmd.leader.isDefined && noLeaderPartitionSet.contains(topicAndPartition)) { + val leaderBroker = pmd.leader.get + leaderForPartitionsMap.put(topicAndPartition, leaderBroker) + noLeaderPartitionSet -= topicAndPartition + } } - - shutdownIdleFetcherThreads() - } catch { - case t => warn("Failed to find leader for %s".format(noLeaderPartitionSet), t) } + } catch { + case t => { + if (!isRunning.get()) + throw t /* If this thread is stopped, propagate this exception to kill the thread. */ + else + warn("Failed to find leader for %s".format(noLeaderPartitionSet), t) + } } finally { lock.unlock() } + + leaderForPartitionsMap.foreach { + case(topicAndPartition, leaderBroker) => + val pti = partitionMap(topicAndPartition) + addFetcher(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset(), leaderBroker) + } + + shutdownIdleFetcherThreads() Thread.sleep(config.refreshLeaderBackoffMs) } } @@ -120,6 +122,11 @@ class ConsumerFetcherManager(private val consumerIdString: String, } def stopConnections() { + /* + * Stop the leader finder thread first before stopping fetchers. Otherwise, if there are more partitions without + * leader, then the leader finder thread will process these partitions (before shutting down) and add fetchers for + * these partitions. + */ info("Stopping leader finder thread") if (leaderFinderThread != null) { leaderFinderThread.shutdown() @@ -141,7 +148,6 @@ class ConsumerFetcherManager(private val consumerIdString: String, lock.lock() try { if (partitionMap != null) { - partitionList.foreach(tp => removeFetcher(tp.topic, tp.partition)) noLeaderPartitionSet ++= partitionList cond.signalAll() } diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 5f9c902..dda0a8f 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -20,9 +20,8 @@ package kafka.consumer import kafka.cluster.Broker import kafka.server.AbstractFetcherThread import kafka.message.ByteBufferMessageSet -import kafka.api.{PartitionOffsetRequestInfo, Request, OffsetRequest, FetchResponsePartitionData} +import kafka.api.{Request, OffsetRequest, FetchResponsePartitionData} import kafka.common.TopicAndPartition -import kafka.common.ErrorMapping class ConsumerFetcherThread(name: String, @@ -67,6 +66,7 @@ class ConsumerFetcherThread(name: String, // any logic for partitions whose leader has changed def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) { + partitions.foreach(tap => removePartition(tap.topic, tap.partition)) consumerFetcherManager.addPartitionsWithError(partitions) } } diff --git a/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala b/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala index e69de29..487329a 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala @@ -0,0 +1,16 @@ +/** + * 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. + */ \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index a0f5770..1c4e1da 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -20,11 +20,7 @@ package kafka.consumer import kafka.api._ import kafka.network._ import kafka.utils._ -import kafka.utils.ZkUtils._ -import collection.immutable -import kafka.common.{ErrorMapping, TopicAndPartition, KafkaException} -import org.I0Itec.zkclient.ZkClient -import kafka.cluster.Broker +import kafka.common.{ErrorMapping, TopicAndPartition} /** * A consumer of kafka messages @@ -41,6 +37,7 @@ class SimpleConsumer(val host: String, private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout) val brokerInfo = "host_%s-port_%s".format(host, port) private val fetchRequestAndResponseStats = FetchRequestAndResponseStatsRegistry.getFetchRequestAndResponseStats(clientId) + private var isClosed = false private def connect(): BlockingChannel = { close @@ -62,7 +59,8 @@ class SimpleConsumer(val host: String, def close() { lock synchronized { - disconnect() + disconnect() + isClosed = true } } @@ -141,7 +139,7 @@ class SimpleConsumer(val host: String, def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).buffer) private def getOrMakeConnection() { - if(!blockingChannel.isConnected) { + if(!isClosed && !blockingChannel.isConnected) { connect() } } diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 3f3a239..e3a6420 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -306,6 +306,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, class ZKRebalancerListener(val group: String, val consumerIdString: String, val kafkaMessageAndMetadataStreams: mutable.Map[String,List[KafkaStream[_,_]]]) extends IZkChildListener { + private val correlationId = new AtomicInteger(0) private var isWatcherTriggered = false private val lock = new ReentrantLock private val cond = lock.newCondition() @@ -406,9 +407,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val topicsMetadata = ClientUtils.fetchTopicMetadata(myTopicThreadIdsMap.keySet, brokers, config.clientId, - config.socketTimeoutMs).topicsMetadata + config.socketTimeoutMs, + correlationId.getAndIncrement).topicsMetadata val partitionsPerTopicMap = new mutable.HashMap[String, Seq[Int]] - topicsMetadata.foreach(m =>{ + topicsMetadata.foreach(m => { val topic = m.topic val partitions = m.partitionsMetadata.map(m1 => m1.partitionId) partitionsPerTopicMap.put(topic, partitions) @@ -497,6 +499,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * 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 **/ + if (config.autoCommitEnable) commitOffsets case None => } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 3164f78..38b8674 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -25,6 +25,7 @@ import kafka.server.KafkaConfig import collection.mutable import kafka.api._ import org.apache.log4j.Logger +import kafka.common.TopicAndPartition class ControllerChannelManager (private val controllerContext: ControllerContext, config: KafkaConfig) extends Logging { private val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] @@ -75,6 +76,7 @@ class ControllerChannelManager (private val controllerContext: ControllerContext private def addNewBroker(broker: Broker) { val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize) + debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id)) val channel = new BlockingChannel(broker.host, broker.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, @@ -120,6 +122,7 @@ class RequestSendThread(val controllerId: Int, try{ lock synchronized { + channel.connect() // establish a socket connection if needed channel.send(request) receive = channel.receive() var response: RequestOrResponse = null @@ -128,6 +131,8 @@ class RequestSendThread(val controllerId: Int, response = LeaderAndIsrResponse.readFrom(receive.buffer) case RequestKeys.StopReplicaKey => response = StopReplicaResponse.readFrom(receive.buffer) + case RequestKeys.UpdateMetadataKey => + response = UpdateMetadataResponse.readFrom(receive.buffer) } stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %d" .format(controllerId, controllerContext.epoch, response.correlationId, toBrokerId)) @@ -138,36 +143,51 @@ class RequestSendThread(val controllerId: Int, } } catch { case e => - // log it and let it go. Let controller shut it down. - debug("Exception occurs", e) + warn("Controller %d fails to send a request to broker %d".format(controllerId, toBrokerId), e) + // If there is any socket error (eg, socket timeout), the channel is no longer usable and needs to be recreated. + channel.disconnect() } } } -class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (RequestOrResponse) => Unit) => Unit, +class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendRequest: (Int, RequestOrResponse, (RequestOrResponse) => Unit) => Unit, controllerId: Int, clientId: String) extends Logging { val leaderAndIsrRequestMap = new mutable.HashMap[Int, mutable.HashMap[(String, Int), PartitionStateInfo]] val stopReplicaRequestMap = new mutable.HashMap[Int, Seq[(String, Int)]] val stopAndDeleteReplicaRequestMap = new mutable.HashMap[Int, Seq[(String, Int)]] + val updateMetadataRequestMap = new mutable.HashMap[Int, mutable.HashMap[TopicAndPartition, PartitionStateInfo]] private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) def newBatch() { // raise error if the previous batch is not empty - if(leaderAndIsrRequestMap.size > 0 || stopReplicaRequestMap.size > 0) + if(leaderAndIsrRequestMap.size > 0) throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating " + - "a new one. Some state changes %s might be lost ".format(leaderAndIsrRequestMap.toString())) + "a new one. Some LeaderAndIsr state changes %s might be lost ".format(leaderAndIsrRequestMap.toString())) + if(stopReplicaRequestMap.size > 0) + throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + + "new one. Some StopReplica state changes %s might be lost ".format(stopReplicaRequestMap.toString())) + if(updateMetadataRequestMap.size > 0) + throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + + "new one. Some UpdateMetadata state changes %s might be lost ".format(updateMetadataRequestMap.toString())) + if(stopAndDeleteReplicaRequestMap.size > 0) + throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + + "new one. Some StopReplica with delete state changes %s might be lost ".format(stopAndDeleteReplicaRequestMap.toString())) leaderAndIsrRequestMap.clear() stopReplicaRequestMap.clear() + updateMetadataRequestMap.clear() stopAndDeleteReplicaRequestMap.clear() } def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, - leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, replicationFactor: Int) { + leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, + replicas: Seq[Int]) { brokerIds.foreach { brokerId => leaderAndIsrRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[(String, Int), PartitionStateInfo]) - leaderAndIsrRequestMap(brokerId).put((topic, partition), PartitionStateInfo(leaderIsrAndControllerEpoch, replicationFactor)) + leaderAndIsrRequestMap(brokerId).put((topic, partition), + PartitionStateInfo(leaderIsrAndControllerEpoch, replicas.toSet)) } + addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(TopicAndPartition(topic, partition))) } def addStopReplicaRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, deletePartition: Boolean) { @@ -185,6 +205,30 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques } } + def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int], + partitions:scala.collection.Set[TopicAndPartition] = Set.empty[TopicAndPartition]) { + val partitionList = + if(partitions.isEmpty) { + controllerContext.partitionLeadershipInfo.keySet + } else { + partitions + } + partitionList.foreach { partition => + val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition) + leaderIsrAndControllerEpochOpt match { + case Some(leaderIsrAndControllerEpoch) => + val replicas = controllerContext.partitionReplicaAssignment(partition).toSet + val partitionStateInfo = PartitionStateInfo(leaderIsrAndControllerEpoch, replicas) + brokerIds.foreach { brokerId => + updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo]) + updateMetadataRequestMap(brokerId).put(partition, partitionStateInfo) + } + case None => + info("Leader not assigned yet for partition %s. Skip sending udpate metadata request".format(partition)) + } + } + } + def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int, liveBrokers: Set[Broker]) { leaderAndIsrRequestMap.foreach { m => val broker = m._1 @@ -202,6 +246,16 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques sendRequest(broker, leaderAndIsrRequest, null) } leaderAndIsrRequestMap.clear() + updateMetadataRequestMap.foreach { m => + val broker = m._1 + val partitionStateInfos = m._2.toMap + val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, correlationId, clientId, + partitionStateInfos, controllerContext.liveOrShuttingDownBrokers) + partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request with " + + "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, correlationId, broker, p._1))) + sendRequest(broker, updateMetadataRequest, null) + } + updateMetadataRequestMap.clear() Seq((stopReplicaRequestMap, false), (stopAndDeleteReplicaRequestMap, true)) foreach { case(m, deletePartitions) => { m foreach { diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 02510bd..5ac38fd 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -32,9 +32,9 @@ import kafka.utils.{Utils, ZkUtils, Logging} import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} +import java.util.concurrent.atomic.AtomicInteger import scala.Some import kafka.common.TopicAndPartition -import java.util.concurrent.atomic.AtomicInteger class ControllerContext(val zkClient: ZkClient, var controllerChannelManager: ControllerChannelManager = null, @@ -65,11 +65,12 @@ class ControllerContext(val zkClient: ZkClient, def liveBrokers = liveBrokersUnderlying.filter(broker => !shuttingDownBrokerIds.contains(broker.id)) def liveBrokerIds = liveBrokerIdsUnderlying.filter(brokerId => !shuttingDownBrokerIds.contains(brokerId)) - def liveOrShuttingDownBrokerIds = liveBrokerIdsUnderlying ++ shuttingDownBrokerIds + def liveOrShuttingDownBrokerIds = liveBrokerIdsUnderlying + def liveOrShuttingDownBrokers = liveBrokersUnderlying } trait KafkaControllerMBean { - def shutdownBroker(id: Int): Int + def shutdownBroker(id: Int): Set[TopicAndPartition] } object KafkaController { @@ -91,7 +92,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext) - private val brokerRequestBatch = new ControllerBrokerRequestBatch(sendRequest, this.config.brokerId, this.clientId) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(controllerContext, sendRequest, this.config.brokerId, this.clientId) registerControllerChangedListener() newGauge( @@ -117,17 +118,18 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg def clientId = "id_%d-host_%s-port_%d".format(config.brokerId, config.hostName, config.port) /** - * JMX operation to initiate clean shutdown of a broker. On clean shutdown, - * the controller first determines the partitions that the shutting down - * broker leads, and moves leadership of those partitions to another broker - * that is in that partition's ISR. When all partitions have been moved, the - * broker process can be stopped normally (i.e., by sending it a SIGTERM or - * SIGINT) and no data loss should be observed. + * On clean shutdown, the controller first determines the partitions that the + * shutting down broker leads, and moves leadership of those partitions to another broker + * that is in that partition's ISR. * * @param id Id of the broker to shutdown. * @return The number of partitions that the broker still leads. */ - def shutdownBroker(id: Int) = { + def shutdownBroker(id: Int) : Set[TopicAndPartition] = { + + if (!isActive()) { + throw new ControllerMovedException("Controller moved to another broker. Aborting controlled shutdown") + } controllerContext.brokerShutdownLock synchronized { info("Shutting down broker " + id) @@ -150,67 +152,40 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def replicatedPartitionsBrokerLeads() = controllerContext.controllerLock.synchronized { - trace("All leaders = " + controllerContext.partitionLeadershipInfo.mkString(",")) - controllerContext.partitionLeadershipInfo.filter { - case (topicAndPartition, leaderIsrAndControllerEpoch) => - leaderIsrAndControllerEpoch.leaderAndIsr.leader == id && controllerContext.partitionReplicaAssignment(topicAndPartition).size > 1 - }.map(_._1) - } - - val partitionsToMove = replicatedPartitionsBrokerLeads().toSet - debug("Partitions to move leadership from broker %d: %s".format(id, partitionsToMove.mkString(","))) - - partitionsToMove.foreach{ topicAndPartition => - val (topic, partition) = topicAndPartition.asTuple - // move leadership serially to relinquish lock. + allPartitionsAndReplicationFactorOnBroker.foreach { + case(topicAndPartition, replicationFactor) => + // Move leadership serially to relinquish lock. controllerContext.controllerLock synchronized { controllerContext.partitionLeadershipInfo.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch => if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id) { + // If the broker leads the topic partition, transition the leader and update isr. Updates zk and + // notifies all affected brokers partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, - controlledShutdownPartitionLeaderSelector) - val newLeaderIsrAndControllerEpoch = controllerContext.partitionLeadershipInfo(topicAndPartition) - - // mark replica offline only if leadership was moved successfully - if (newLeaderIsrAndControllerEpoch.leaderAndIsr.leader != currLeaderIsrAndControllerEpoch.leaderAndIsr.leader) - replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topic, partition, id)), OfflineReplica) - } else - debug("Partition %s moved from leader %d to new leader %d during shutdown." - .format(topicAndPartition, id, currLeaderIsrAndControllerEpoch.leaderAndIsr.leader)) + controlledShutdownPartitionLeaderSelector) + } + else { + // Stop the replica first. The state change below initiates ZK changes which should take some time + // before which the stop replica request should be completed (in most cases) + brokerRequestBatch.newBatch() + brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, topicAndPartition.partition, deletePartition = false) + brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) + + // If the broker is a follower, updates the isr in ZK and notifies the current leader + replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic, + topicAndPartition.partition, id)), OfflineReplica) + } } } } - val partitionsRemaining = replicatedPartitionsBrokerLeads().toSet - - /* - * Force the shutting down broker out of the ISR of partitions that it - * follows, and shutdown the corresponding replica fetcher threads. - * This is really an optimization, so no need to register any callback - * to wait until completion. - */ - if (partitionsRemaining.size == 0) { - brokerRequestBatch.newBatch() - allPartitionsAndReplicationFactorOnBroker foreach { - case(topicAndPartition, replicationFactor) => - val (topic, partition) = topicAndPartition.asTuple - if (controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader != id) { - brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topic, partition, deletePartition = false) - removeReplicaFromIsr(topic, partition, id) match { - case Some(updatedLeaderIsrAndControllerEpoch) => - brokerRequestBatch.addLeaderAndIsrRequestForBrokers( - Seq(updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader), topic, partition, - updatedLeaderIsrAndControllerEpoch, replicationFactor) - case None => - // ignore - } - } - } - brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) + def replicatedPartitionsBrokerLeads() = controllerContext.controllerLock.synchronized { + trace("All leaders = " + controllerContext.partitionLeadershipInfo.mkString(",")) + controllerContext.partitionLeadershipInfo.filter { + case (topicAndPartition, leaderIsrAndControllerEpoch) => + leaderIsrAndControllerEpoch.leaderAndIsr.leader == id && controllerContext.partitionReplicaAssignment(topicAndPartition).size > 1 + }.map(_._1) } - - debug("Remaining partitions to move from broker %d: %s".format(id, partitionsRemaining.mkString(","))) - partitionsRemaining.size + replicatedPartitionsBrokerLeads().toSet } } @@ -244,6 +219,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("Broker %d is ready to serve as the new controller with epoch %d".format(config.brokerId, epoch)) initializeAndMaybeTriggerPartitionReassignment() initializeAndMaybeTriggerPreferredReplicaElection() + /* send partition leadership info to all live brokers */ + sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) } else info("Controller has been shut down, aborting startup/failover") @@ -275,13 +252,16 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("New broker startup callback for %s".format(newBrokers.mkString(","))) val newBrokersSet = newBrokers.toSet + // send update metadata request for all partitions to the newly restarted brokers. In cases of controlled shutdown + // leaders will not be elected when a new broker comes up. So at least in the common controlled shutdown case, the + // metadata will reach the new brokers faster + sendUpdateMetadataRequest(newBrokers) // the very first thing to do when a new broker comes up is send it the entire list of partitions that it is // supposed to host. Based on that the broker starts the high watermark threads for the input list of partitions replicaStateMachine.handleStateChanges(getAllReplicasOnBroker(zkClient, controllerContext.allTopics.toSeq, newBrokers), OnlineReplica) // when a new broker comes up, the controller needs to trigger leader election for all new and offline partitions // to see if these brokers can become leaders for some/all of those partitionStateMachine.triggerOnlinePartitionStateChange() - // check if reassignment of some partitions need to be restarted val partitionsWithReplicasOnNewBrokers = controllerContext.partitionsBeingReassigned.filter{ case (topicAndPartition, reassignmentContext) => @@ -379,6 +359,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg removePartitionFromReassignedPartitions(topicAndPartition) info("Removed partition %s from the list of reassigned partitions in zookeeper".format(topicAndPartition)) controllerContext.partitionsBeingReassigned.remove(topicAndPartition) + // after electing leader, the replicas and isr information changes, so resend the update metadata request + sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicAndPartition)) case false => info("New replicas %s for partition %s being ".format(reassignedReplicas.mkString(","), topicAndPartition) + "reassigned not yet caught up with the leader") @@ -478,6 +460,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg controllerContext.allTopics = ZkUtils.getAllTopics(zkClient).toSet controllerContext.partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, controllerContext.allTopics.toSeq) controllerContext.partitionLeadershipInfo = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch] + controllerContext.shuttingDownBrokerIds = mutable.Set.empty[Int] // update the leader and isr cache for all existing partitions from Zookeeper updateLeaderAndIsrCache() // start the channel manager @@ -522,7 +505,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } private def updateLeaderAndIsrCache() { - val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, controllerContext.allTopics.toSeq) + val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, controllerContext.partitionReplicaAssignment.keySet) for((topicPartition, leaderIsrAndControllerEpoch) <- leaderAndIsrInfo) controllerContext.partitionLeadershipInfo.put(topicPartition, leaderIsrAndControllerEpoch) } @@ -663,6 +646,18 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } /** + * Send the leader information for selected partitions to selected brokers so that they can correctly respond to + * metadata requests + * @param brokers The brokers that the update metadata request should be sent to + * @param partitions The partitions for which the metadata is to be sent + */ + private def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) { + brokerRequestBatch.newBatch() + brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) + brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) + } + + /** * Removes a given partition replica from the ISR; if it is not the current * leader and there are sufficient remaining replicas in ISR. * @param topic topic @@ -673,7 +668,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg */ def removeReplicaFromIsr(topic: String, partition: Int, replicaId: Int): Option[LeaderIsrAndControllerEpoch] = { val topicAndPartition = TopicAndPartition(topic, partition) - debug("Removing replica %d from ISR of %s.".format(replicaId, topicAndPartition)) + debug("Removing replica %d from ISR %s for partition %s.".format(replicaId, + controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.isr.mkString(","), topicAndPartition)) var finalLeaderIsrAndControllerEpoch: Option[LeaderIsrAndControllerEpoch] = None var zkWriteCompleteOrUnnecessary = false while (!zkWriteCompleteOrUnnecessary) { @@ -701,6 +697,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg newLeaderAndIsr.zkVersion = newVersion finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newLeaderAndIsr, epoch)) + controllerContext.partitionLeadershipInfo.put(topicAndPartition, finalLeaderIsrAndControllerEpoch.get) if (updateSucceeded) info("New leader and ISR for partition %s is %s".format(topicAndPartition, newLeaderAndIsr.toString())) updateSucceeded @@ -708,6 +705,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg warn("Cannot remove replica %d from ISR of %s. Leader = %d ; ISR = %s" .format(replicaId, topicAndPartition, leaderAndIsr.leader, leaderAndIsr.isr)) finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(leaderAndIsr, epoch)) + controllerContext.partitionLeadershipInfo.put(topicAndPartition, finalLeaderIsrAndControllerEpoch.get) true } case None => diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index 21b0e24..a47b142 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -63,13 +63,14 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten case false => ControllerStats.uncleanLeaderElectionRate.mark() val newLeader = liveAssignedReplicasToThisPartition.head - warn("No broker in ISR is alive for %s. Elect leader from broker %s. There's potential data loss." - .format(topicAndPartition, newLeader)) + warn("No broker in ISR is alive for %s. Elect leader %d from live brokers %s. There's potential data loss." + .format(topicAndPartition, newLeader, liveAssignedReplicasToThisPartition.mkString(","))) new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, List(newLeader), currentLeaderIsrZkPathVersion + 1) } case false => val newLeader = liveBrokersInIsr.head - debug("Some broker in ISR is alive for %s. Select %d from ISR to be the leader.".format(topicAndPartition, newLeader)) + debug("Some broker in ISR is alive for %s. Select %d from ISR %s to be the leader." + .format(topicAndPartition, newLeader, liveBrokersInIsr.mkString(","))) new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, liveBrokersInIsr.toList, currentLeaderIsrZkPathVersion + 1) } info("Selected new leader and ISR %s for offline partition %s".format(newLeaderAndIsr.toString(), topicAndPartition)) diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index c017727..deebed0 100644 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -43,9 +43,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { private val controllerId = controller.config.brokerId private val zkClient = controllerContext.zkClient var partitionState: mutable.Map[TopicAndPartition, PartitionState] = mutable.Map.empty - val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.sendRequest, controllerId, controller.clientId) + val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.controllerContext, controller.sendRequest, + controllerId, controller.clientId) private val hasStarted = new AtomicBoolean(false) - private val hasShutdown = new AtomicBoolean(false) private val noOpPartitionLeaderSelector = new NoOpLeaderSelector(controllerContext) this.logIdent = "[Partition state machine on Controller " + controllerId + "]: " private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) @@ -73,7 +73,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { * Invoked on controller shutdown. */ def shutdown() { - hasShutdown.compareAndSet(false, true) + hasStarted.set(false) partitionState.clear() } @@ -188,13 +188,11 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { */ private def initializePartitionState() { for((topicPartition, replicaAssignment) <- controllerContext.partitionReplicaAssignment) { - val topic = topicPartition.topic - val partition = topicPartition.partition // check if leader and isr path exists for partition. If not, then it is in NEW state - ZkUtils.getLeaderAndIsrForPartition(zkClient, topic, partition) match { - case Some(currentLeaderAndIsr) => + controllerContext.partitionLeadershipInfo.get(topicPartition) match { + case Some(currentLeaderIsrAndEpoch) => // else, check if the leader for partition is alive. If yes, it is in Online state, else it is in Offline state - controllerContext.liveBrokerIds.contains(currentLeaderAndIsr.leader) match { + controllerContext.liveBrokerIds.contains(currentLeaderIsrAndEpoch.leaderAndIsr.leader) match { case true => // leader is alive partitionState.put(topicPartition, OnlinePartition) case false => @@ -233,7 +231,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { * @param topicAndPartition The topic/partition whose leader and isr path is to be initialized */ private def initializeLeaderAndIsrForPartition(topicAndPartition: TopicAndPartition) { - debug("Initializing leader and isr for partition %s".format(topicAndPartition)) val replicaAssignment = controllerContext.partitionReplicaAssignment(topicAndPartition) val liveAssignedReplicas = replicaAssignment.filter(r => controllerContext.liveBrokerIds.contains(r)) liveAssignedReplicas.size match { @@ -249,6 +246,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { val leader = liveAssignedReplicas.head val leaderIsrAndControllerEpoch = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader, liveAssignedReplicas.toList), controller.epoch) + debug("Initializing leader and isr for partition %s to %s".format(topicAndPartition, leaderIsrAndControllerEpoch)) try { ZkUtils.createPersistentPath(controllerContext.zkClient, ZkUtils.getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition), @@ -256,9 +254,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { // NOTE: the above write can fail only if the current controller lost its zk session and the new controller // took over and initialized this partition. This can happen if the current controller went into a long // GC pause - brokerRequestBatch.addLeaderAndIsrRequestForBrokers(liveAssignedReplicas, topicAndPartition.topic, - topicAndPartition.partition, leaderIsrAndControllerEpoch, replicaAssignment.size) controllerContext.partitionLeadershipInfo.put(topicAndPartition, leaderIsrAndControllerEpoch) + brokerRequestBatch.addLeaderAndIsrRequestForBrokers(liveAssignedReplicas, topicAndPartition.topic, + topicAndPartition.partition, leaderIsrAndControllerEpoch, replicaAssignment) } catch { case e: ZkNodeExistsException => // read the controller epoch @@ -316,9 +314,10 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { controllerContext.partitionLeadershipInfo.put(TopicAndPartition(topic, partition), newLeaderIsrAndControllerEpoch) stateChangeLogger.trace("Controller %d epoch %d elected leader %d for Offline partition %s" .format(controllerId, controller.epoch, newLeaderAndIsr.leader, topicAndPartition)) + val replicas = controllerContext.partitionReplicaAssignment(TopicAndPartition(topic, partition)) // store new leader and isr info in cache brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasForThisPartition, topic, partition, - newLeaderIsrAndControllerEpoch, controllerContext.partitionReplicaAssignment(TopicAndPartition(topic, partition)).size) + newLeaderIsrAndControllerEpoch, replicas) } catch { case lenne: LeaderElectionNotNeededException => // swallow case nroe: NoReplicaOnlineException => throw nroe @@ -358,7 +357,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { @throws(classOf[Exception]) def handleChildChange(parentPath : String, children : java.util.List[String]) { controllerContext.controllerLock synchronized { - if (!hasShutdown.get) { + if (hasStarted.get) { try { debug("Topic change listener fired for path %s with children %s".format(parentPath, children.mkString(","))) val currentChildren = JavaConversions.asBuffer(children).toSet diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index bea1644..0c9d436 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -42,9 +42,9 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { private val controllerId = controller.config.brokerId private val zkClient = controllerContext.zkClient var replicaState: mutable.Map[(String, Int, Int), ReplicaState] = mutable.Map.empty - val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.sendRequest, controllerId, controller.clientId) + val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.controllerContext, controller.sendRequest, + controllerId, controller.clientId) private val hasStarted = new AtomicBoolean(false) - private val hasShutdown = new AtomicBoolean(false) this.logIdent = "[Replica state machine on controller " + controller.config.brokerId + "]: " private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) @@ -58,7 +58,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { initializeReplicaState() hasStarted.set(true) // move all Online replicas to Online - handleStateChanges(ZkUtils.getAllReplicasOnBroker(zkClient, controllerContext.allTopics.toSeq, + handleStateChanges(getAllReplicasOnBroker(controllerContext.allTopics.toSeq, controllerContext.liveBrokerIds.toSeq), OnlineReplica) info("Started replica state machine with initial state -> " + replicaState.toString()) } @@ -72,7 +72,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * Invoked on controller shutdown. */ def shutdown() { - hasShutdown.compareAndSet(false, true) + hasStarted.set(false) replicaState.clear() } @@ -121,7 +121,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { throw new StateChangeFailedException("Replica %d for partition %s cannot be moved to NewReplica" .format(replicaId, topicAndPartition) + "state as it is being requested to become leader") brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId), - topic, partition, leaderIsrAndControllerEpoch, replicaAssignment.size) + topic, partition, leaderIsrAndControllerEpoch, + replicaAssignment) case None => // new leader request will be sent to this replica when one gets elected } replicaState.put((topic, partition, replicaId), NewReplica) @@ -152,7 +153,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { controllerContext.partitionLeadershipInfo.get(topicAndPartition) match { case Some(leaderIsrAndControllerEpoch) => brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId), topic, partition, leaderIsrAndControllerEpoch, - replicaAssignment.size) + replicaAssignment) replicaState.put((topic, partition, replicaId), OnlineReplica) stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica" .format(controllerId, controller.epoch, replicaId, topicAndPartition)) @@ -173,8 +174,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case Some(updatedLeaderIsrAndControllerEpoch) => // send the shrunk ISR state change request only to the leader brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader), - topic, partition, updatedLeaderIsrAndControllerEpoch, - replicaAssignment.size) + topic, partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment) replicaState.put((topic, partition, replicaId), OfflineReplica) stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OfflineReplica" .format(controllerId, controller.epoch, replicaId, topicAndPartition)) @@ -228,6 +228,16 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { } } + private def getAllReplicasOnBroker(topics: Seq[String], brokerIds: Seq[Int]): Set[PartitionAndReplica] = { + brokerIds.map { brokerId => + val partitionsAssignedToThisBroker = + controllerContext.partitionReplicaAssignment.filter(p => topics.contains(p._1.topic) && p._2.contains(brokerId)) + if(partitionsAssignedToThisBroker.size == 0) + info("No state transitions triggered since no partitions are assigned to brokers %s".format(brokerIds.mkString(","))) + partitionsAssignedToThisBroker.map(p => new PartitionAndReplica(p._1.topic, p._1.partition, brokerId)) + }.flatten.toSet + } + def getPartitionsAssignedToBroker(topics: Seq[String], brokerId: Int):Seq[TopicAndPartition] = { controllerContext.partitionReplicaAssignment.filter(_._2.contains(brokerId)).keySet.toSeq } @@ -238,14 +248,15 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { class BrokerChangeListener() extends IZkChildListener with Logging { this.logIdent = "[BrokerChangeListener on Controller " + controller.config.brokerId + "]: " def handleChildChange(parentPath : String, currentBrokerList : java.util.List[String]) { - ControllerStats.leaderElectionTimer.time { - info("Broker change listener fired for path %s with children %s".format(parentPath, currentBrokerList.mkString(","))) - controllerContext.controllerLock synchronized { - if (!hasShutdown.get) { + info("Broker change listener fired for path %s with children %s".format(parentPath, currentBrokerList.mkString(","))) + controllerContext.controllerLock synchronized { + if (hasStarted.get) { + ControllerStats.leaderElectionTimer.time { try { val curBrokerIds = currentBrokerList.map(_.toInt).toSet val newBrokerIds = curBrokerIds -- controllerContext.liveOrShuttingDownBrokerIds - val newBrokers = newBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) + val newBrokerInfo = newBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _)) + val newBrokers = newBrokerInfo.filter(_.isDefined).map(_.get) val deadBrokerIds = controllerContext.liveOrShuttingDownBrokerIds -- curBrokerIds controllerContext.liveBrokers = curBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) info("Newly added brokers: %s, deleted brokers: %s, all live brokers: %s" diff --git a/core/src/main/scala/kafka/javaapi/Implicits.scala b/core/src/main/scala/kafka/javaapi/Implicits.scala index 66ab821..9a63914 100644 --- a/core/src/main/scala/kafka/javaapi/Implicits.scala +++ b/core/src/main/scala/kafka/javaapi/Implicits.scala @@ -43,7 +43,7 @@ private[javaapi] object Implicits extends Logging { implicit def optionToJavaRef[T](opt: Option[T]): T = { opt match { case Some(obj) => obj - case None => null + case None => null.asInstanceOf[T] } } } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 8c8d877..87151b9 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -163,8 +163,17 @@ class Log(val dir: File, // run recovery on the active segment if necessary if(needsRecovery) { - info("Recovering active segment of %s.".format(name)) - active.recover(config.maxMessageSize) + try { + info("Recovering active segment of %s.".format(name)) + active.recover(config.maxMessageSize) + } catch { + case e: InvalidOffsetException => + val startOffset = active.baseOffset + warn("Found invalid offset during recovery of the active segment for topic partition " + dir.getName +". Deleting the segment and " + + "creating an empty one with starting offset " + startOffset) + // truncate the active segment to its starting offset + active.truncateTo(startOffset) + } } } diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index fbdc553..38e6cd5 100644 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -1,4 +1,20 @@ -package kafka.log +/** + * 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.log import scala.math._ import java.io.File @@ -189,6 +205,7 @@ class LogSegment(val log: FileMessageSet, val bytesTruncated = log.truncateTo(mapping.position) if(log.sizeInBytes == 0) created = time.milliseconds + bytesSinceLastIndexEntry = 0 bytesTruncated } diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index 361a9db..fbc728c 100644 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -23,6 +23,7 @@ import java.nio._ import java.nio.channels._ import java.util.concurrent.atomic._ import kafka.utils._ +import kafka.common.InvalidOffsetException /** * An index that maps offsets to physical file locations for a particular log segment. This index may be sparse: @@ -190,13 +191,18 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi def append(offset: Long, position: Int) { this synchronized { require(!isFull, "Attempt to append to a full index (size = " + size + ").") - require(size.get == 0 || offset > lastOffset, "Attempt to append an offset (%d) to position %d no larger than the last offset appended (%d).".format(offset, entries, lastOffset)) - debug("Adding index entry %d => %d to %s.".format(offset, position, file.getName)) - this.mmap.putInt((offset - baseOffset).toInt) - this.mmap.putInt(position) - this.size.incrementAndGet() - this.lastOffset = offset - require(entries * 8 == mmap.position, entries + " entries but file position in index is " + mmap.position + ".") + if (size.get == 0 || offset > lastOffset) { + debug("Adding index entry %d => %d to %s.".format(offset, position, file.getName)) + this.mmap.putInt((offset - baseOffset).toInt) + this.mmap.putInt(position) + this.size.incrementAndGet() + this.lastOffset = offset + require(entries * 8 == mmap.position, entries + " entries but file position in index is " + mmap.position + ".") + } + else { + throw new InvalidOffsetException("Attempt to append an offset (%d) to position %d no larger than the last offset appended (%d) to %s." + .format(offset, entries, lastOffset, file.getName)) + } } } diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 1a43fdf..52c082f 100644 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -74,8 +74,9 @@ object Message { * 3. 1 byte "attributes" identifier to allow annotations on the message independent of the version (e.g. compression enabled, type of codec used) * 4. 4 byte key length, containing length K * 5. K byte key - * 6. (N - K - 10) byte payload - * + * 6. 4 byte payload length, containing length V + * 7. V byte payload + * * Default constructor wraps an existing ByteBuffer with the Message object with no change to the contents. */ class Message(val buffer: ByteBuffer) { diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 7b8d1f0..1437496 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -49,7 +49,7 @@ object RequestChannel extends Logging { val requestObj: RequestOrResponse = RequestKeys.deserializerForKey(requestId)(buffer) buffer = null private val requestLogger = Logger.getLogger("kafka.request.logger") - trace("Received request : %s".format(requestObj)) + trace("Processor %d received request : %s".format(processor, requestObj)) def updateRequestMetrics() { val endTimeMs = SystemTime.milliseconds diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 134e60c..9767f4d 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -61,20 +61,19 @@ class SocketServer(val brokerId: Int, this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize) Utils.newThread("kafka-acceptor", acceptor, false).start() acceptor.awaitStartup - info("started") + info("Started") } /** * Shutdown the socket server */ def shutdown() = { - info("shutting down") + info("Shutting down") if(acceptor != null) acceptor.shutdown() for(processor <- processors) processor.shutdown() - requestChannel.shutdown - info("shut down completely") + info("Shutdown completed") } } diff --git a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala index 82e6e4d..13a8aa6 100644 --- a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala +++ b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala @@ -54,6 +54,13 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig, } } val partitionMetadata = metadata.partitionsMetadata + if(partitionMetadata.size == 0) { + if(metadata.errorCode != ErrorMapping.NoError) { + throw new KafkaException(ErrorMapping.exceptionFor(metadata.errorCode)) + } else { + throw new KafkaException("Topic metadata %s has empty partition metadata and no error code".format(metadata)) + } + } partitionMetadata.map { m => m.leader match { case Some(leader) => @@ -77,7 +84,7 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig, // throw partition specific exception topicsMetadata.foreach(tmd =>{ trace("Metadata for topic %s is %s".format(tmd.topic, tmd)) - if(tmd.errorCode == ErrorMapping.NoError){ + if(tmd.errorCode == ErrorMapping.NoError) { topicPartitionInfo.put(tmd.topic, tmd) } else warn("Error while fetching metadata [%s] for topic [%s]: %s ".format(tmd, tmd.topic, ErrorMapping.exceptionFor(tmd.errorCode).getClass)) diff --git a/core/src/main/scala/kafka/producer/Producer.scala b/core/src/main/scala/kafka/producer/Producer.scala index c837091..bb16a29 100644 --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -106,7 +106,6 @@ class Producer[K,V](val config: ProducerConfig, if(!added) { producerTopicStats.getProducerTopicStats(message.topic).droppedMessageRate.mark() producerTopicStats.getProducerAllTopicsStats.droppedMessageRate.mark() - error("Event queue is full of unsent messages, could not send event: " + message.toString) throw new QueueFullException("Event queue is full of unsent messages, could not send event: " + message.toString) }else { trace("Added to send queue an event: " + message.toString) diff --git a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala index ef32620..69b2d0c 100644 --- a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala @@ -58,5 +58,5 @@ trait SyncProducerConfigShared { object SyncProducerConfig { val DefaultClientId = "" val DefaultRequiredAcks : Short = 0 - val DefaultAckTimeoutMs = 1500 + val DefaultAckTimeoutMs = 10000 } \ No newline at end of file diff --git a/core/src/main/scala/kafka/producer/async/AsyncProducerStats.scala b/core/src/main/scala/kafka/producer/async/AsyncProducerStats.scala index e69de29..487329a 100644 --- a/core/src/main/scala/kafka/producer/async/AsyncProducerStats.scala +++ b/core/src/main/scala/kafka/producer/async/AsyncProducerStats.scala @@ -0,0 +1,16 @@ +/** + * 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. + */ \ No newline at end of file diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index 698ad1c..1ecaeaa 100644 --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -52,7 +52,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, def handle(events: Seq[KeyedMessage[K,V]]) { lock synchronized { val serializedData = serialize(events) - serializedData.foreach{ + serializedData.foreach { keyed => val dataSize = keyed.message.payloadSize producerTopicStats.getProducerTopicStats(keyed.topic).byteRate.mark(dataSize) @@ -61,6 +61,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, var outstandingProduceRequests = serializedData var remainingRetries = config.messageSendMaxRetries + 1 val correlationIdStart = correlationId.get() + debug("Handling %d events".format(events.size)) while (remainingRetries > 0 && outstandingProduceRequests.size > 0) { topicMetadataToRefresh ++= outstandingProduceRequests.map(_.topic) if (topicMetadataRefreshInterval >= 0 && @@ -70,7 +71,8 @@ class DefaultEventHandler[K,V](config: ProducerConfig, lastTopicMetadataRefreshTime = SystemTime.milliseconds } outstandingProduceRequests = dispatchSerializedData(outstandingProduceRequests) - if (outstandingProduceRequests.size > 0) { + if (outstandingProduceRequests.size > 0) { + info("Back off for %d ms before retrying send. Remaining retries = %d".format(config.retryBackoffMs, remainingRetries-1)) // back off and update the topic metadata cache before attempting another send operation Thread.sleep(config.retryBackoffMs) // get topics of the outstanding produce requests and refresh metadata for those @@ -147,7 +149,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, try { for (message <- messages) { val topicPartitionsList = getPartitionListForTopic(message) - val partitionIndex = getPartition(message.key, topicPartitionsList) + val partitionIndex = getPartition(message.topic, message.key, topicPartitionsList) val brokerPartition = topicPartitionsList(partitionIndex) // postpone the failure until the send operation, so that requests for other brokers are handled correctly @@ -175,9 +177,9 @@ class DefaultEventHandler[K,V](config: ProducerConfig, } Some(ret) }catch { // Swallow recoverable exceptions and return None so that they can be retried. - case ute: UnknownTopicException => warn("Failed to collate messages by topic,partition due to", ute); None - case lnae: LeaderNotAvailableException => warn("Failed to collate messages by topic,partition due to", lnae); None - case oe => error("Failed to collate messages by topic, partition due to", oe); throw oe + case ute: UnknownTopicOrPartitionException => warn("Failed to collate messages by topic,partition due to: " + ute.getMessage); None + case lnae: LeaderNotAvailableException => warn("Failed to collate messages by topic,partition due to: " + lnae.getMessage); None + case oe => error("Failed to collate messages by topic, partition due to: " + oe.getMessage); None } } @@ -198,25 +200,24 @@ class DefaultEventHandler[K,V](config: ProducerConfig, * @param topicPartitionList the list of available partitions * @return the partition id */ - private def getPartition(key: K, topicPartitionList: Seq[PartitionAndLeader]): Int = { + private def getPartition(topic: String, key: K, topicPartitionList: Seq[PartitionAndLeader]): Int = { val numPartitions = topicPartitionList.size if(numPartitions <= 0) - throw new UnknownTopicOrPartitionException("Invalid number of partitions: " + numPartitions + - "\n Valid values are > 0") + throw new UnknownTopicOrPartitionException("Topic " + topic + " doesn't exist") val partition = if(key == null) { // If the key is null, we don't really need a partitioner so we just send to the next // available partition val availablePartitions = topicPartitionList.filter(_.leaderBrokerIdOpt.isDefined) if (availablePartitions.isEmpty) - throw new LeaderNotAvailableException("No leader for any partition") + throw new LeaderNotAvailableException("No leader for any partition in topic " + topic) val index = Utils.abs(partitionCounter.getAndIncrement()) % availablePartitions.size availablePartitions(index).partitionId } else partitioner.partition(key, numPartitions) if(partition < 0 || partition >= numPartitions) - throw new UnknownTopicOrPartitionException("Invalid partition id : " + partition + - "\n Valid values are in the range inclusive [0, " + (numPartitions-1) + "]") + throw new UnknownTopicOrPartitionException("Invalid partition id: " + partition + " for topic " + topic + + "; Valid values are in the inclusive range of [0, " + (numPartitions-1) + "]") partition } @@ -251,11 +252,18 @@ class DefaultEventHandler[K,V](config: ProducerConfig, successfullySentData.foreach(m => messagesPerTopic(m._1).foreach(message => trace("Successfully sent message: %s".format(if(message.message.isNull) null else Utils.readString(message.message.payload))))) } - failedTopicPartitions = response.status.filter(_._2.error != ErrorMapping.NoError).toSeq - .map(partitionStatus => partitionStatus._1) - if(failedTopicPartitions.size > 0) - error("Produce request with correlation id %d failed due to response %s. List of failed topic partitions is %s" - .format(currentCorrelationId, response.toString, failedTopicPartitions.mkString(","))) + val failedPartitionsAndStatus = response.status.filter(_._2.error != ErrorMapping.NoError).toSeq + failedTopicPartitions = failedPartitionsAndStatus.map(partitionStatus => partitionStatus._1) + if(failedTopicPartitions.size > 0) { + val errorString = failedPartitionsAndStatus + .sortWith((p1, p2) => p1._1.topic.compareTo(p2._1.topic) < 0 || + (p1._1.topic.compareTo(p2._1.topic) == 0 && p1._1.partition < p2._1.partition)) + .map{ + case(topicAndPartition, status) => + topicAndPartition.toString + ": " + ErrorMapping.exceptionFor(status.error).getClass.getName + }.mkString(",") + warn("Produce request with correlation id %d failed due to %s".format(currentCorrelationId, errorString)) + } failedTopicPartitions } else { Seq.empty[TopicAndPartition] diff --git a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala index 090400d..2b41a49 100644 --- a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala +++ b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala @@ -50,7 +50,7 @@ class ProducerSendThread[K,V](val threadName: String, } def shutdown = { - info("Beging shutting down ProducerSendThread") + info("Begin shutting down ProducerSendThread") queue.put(shutdownCommand) shutdownLatch.await info("Shutdown ProducerSendThread complete") diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 4269219..15b7bd3 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -20,13 +20,45 @@ package kafka.server import scala.collection.mutable import kafka.utils.Logging import kafka.cluster.Broker +import kafka.metrics.KafkaMetricsGroup +import com.yammer.metrics.core.Gauge -abstract class AbstractFetcherManager(protected val name: String, numFetchers: Int = 1) extends Logging { +abstract class AbstractFetcherManager(protected val name: String, metricPrefix: String, numFetchers: Int = 1) + extends Logging with KafkaMetricsGroup { // map of (source brokerid, fetcher Id per source broker) => fetcher private val fetcherThreadMap = new mutable.HashMap[BrokerAndFetcherId, AbstractFetcherThread] private val mapLock = new Object this.logIdent = "[" + name + "] " + newGauge( + metricPrefix + "-MaxLag", + new Gauge[Long] { + // current max lag across all fetchers/topics/partitions + def value = fetcherThreadMap.foldLeft(0L)((curMaxAll, fetcherThreadMapEntry) => { + fetcherThreadMapEntry._2.fetcherLagStats.stats.foldLeft(0L)((curMaxThread, fetcherLagStatsEntry) => { + curMaxThread.max(fetcherLagStatsEntry._2.lag) + }).max(curMaxAll) + }) + } + ) + + newGauge( + metricPrefix + "-MinFetchRate", + { + new Gauge[Double] { + // current min fetch rate across all fetchers/topics/partitions + def value = { + val headRate: Double = + fetcherThreadMap.headOption.map(_._2.fetcherStats.requestRate.oneMinuteRate).getOrElse(0) + + fetcherThreadMap.foldLeft(headRate)((curMinAll, fetcherThreadMapEntry) => { + fetcherThreadMapEntry._2.fetcherStats.requestRate.oneMinuteRate.min(curMinAll) + }) + } + } + } + ) + private def getFetcherId(topic: String, partitionId: Int) : Int = { (topic.hashCode() + 31 * partitionId) % numFetchers } diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 162c749..7663fac 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -96,8 +96,8 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke response = simpleConsumer.fetch(fetchRequest) } catch { case t => - warn("Error in fetch %s".format(fetchRequest), t) if (isRunning.get) { + warn("Error in fetch %s".format(fetchRequest), t) partitionMapLock synchronized { partitionsWithError ++= partitionMap.keys } @@ -152,9 +152,11 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke partitionsWithError += topicAndPartition } case _ => - warn("error for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), - ErrorMapping.exceptionFor(partitionData.error)) - partitionsWithError += topicAndPartition + if (isRunning.get) { + warn("error for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), + ErrorMapping.exceptionFor(partitionData.error)) + partitionsWithError += topicAndPartition + } } } } @@ -219,7 +221,7 @@ class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMet class FetcherLagStats(metricId: ClientIdAndBroker) { private val valueFactory = (k: ClientIdBrokerTopicPartition) => new FetcherLagMetrics(k) - private val stats = new Pool[ClientIdBrokerTopicPartition, FetcherLagMetrics](Some(valueFactory)) + val stats = new Pool[ClientIdBrokerTopicPartition, FetcherLagMetrics](Some(valueFactory)) def getFetcherLagStats(topic: String, partitionId: Int): FetcherLagMetrics = { stats.getAndMaybePut(new ClientIdBrokerTopicPartition(metricId.clientId, metricId.brokerInfo, topic, partitionId)) diff --git a/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala b/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala new file mode 100644 index 0000000..e18ecab --- /dev/null +++ b/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala @@ -0,0 +1,121 @@ +/** + * 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.Logging +import kafka.common._ +import java.util.concurrent.locks.ReentrantLock +import java.io._ + +/** + * This class handles the read/write to the highwaterMark checkpoint file. The file stores the high watermark value for + * all topics and partitions that this broker hosts. The format of this file is as follows - + * version + * number of entries + * topic partition highwatermark + */ + +object HighwaterMarkCheckpoint { + val highWatermarkFileName = "replication-offset-checkpoint" + val currentHighwaterMarkFileVersion = 0 +} + +class HighwaterMarkCheckpoint(val path: String) extends Logging { + /* create the highwatermark file handle for all partitions */ + val name = path + File.separator + HighwaterMarkCheckpoint.highWatermarkFileName + private val hwFile = new File(name) + private val hwFileLock = new ReentrantLock() + // recover from previous tmp file, if required + + def write(highwaterMarksPerPartition: Map[TopicAndPartition, Long]) { + hwFileLock.lock() + try { + // write to temp file and then swap with the highwatermark file + val tempHwFile = new File(hwFile + ".tmp") + + val hwFileWriter = new BufferedWriter(new FileWriter(tempHwFile)) + // checkpoint highwatermark for all partitions + // write the current version + hwFileWriter.write(HighwaterMarkCheckpoint.currentHighwaterMarkFileVersion.toString) + hwFileWriter.newLine() + // write the number of entries in the highwatermark file + hwFileWriter.write(highwaterMarksPerPartition.size.toString) + hwFileWriter.newLine() + + highwaterMarksPerPartition.foreach { partitionAndHw => + hwFileWriter.write("%s %s %s".format(partitionAndHw._1.topic, partitionAndHw._1.partition, partitionAndHw._2)) + hwFileWriter.newLine() + } + hwFileWriter.flush() + hwFileWriter.close() + // swap new high watermark file with previous one + if(!tempHwFile.renameTo(hwFile)) { + // renameTo() fails on Windows if the destination file exists. + hwFile.delete() + if(!tempHwFile.renameTo(hwFile)) { + fatal("Attempt to swap the new high watermark file with the old one failed") + System.exit(1) + } + } + }finally { + hwFileLock.unlock() + } + } + + def read(topic: String, partition: Int): Long = { + hwFileLock.lock() + try { + hwFile.length() match { + case 0 => + warn("No highwatermark file is found. Returning 0 as the highwatermark for partition [%s,%d]".format(topic, partition)) + 0L + case _ => + val hwFileReader = new BufferedReader(new FileReader(hwFile)) + val version = hwFileReader.readLine().toShort + version match { + case HighwaterMarkCheckpoint.currentHighwaterMarkFileVersion => + val numberOfHighWatermarks = hwFileReader.readLine().toInt + val partitionHighWatermarks = + for(i <- 0 until numberOfHighWatermarks) yield { + val nextHwEntry = hwFileReader.readLine() + val partitionHwInfo = nextHwEntry.split(" ") + val topic = partitionHwInfo(0) + val partitionId = partitionHwInfo(1).toInt + val highWatermark = partitionHwInfo(2).toLong + (TopicAndPartition(topic, partitionId) -> highWatermark) + } + hwFileReader.close() + val hwOpt = partitionHighWatermarks.toMap.get(TopicAndPartition(topic, partition)) + hwOpt match { + case Some(hw) => + debug("Read hw %d for partition [%s,%d] from highwatermark checkpoint file".format(hw, topic, partition)) + hw + case None => + warn("No previously checkpointed highwatermark value found for topic %s ".format(topic) + + "partition %d. Returning 0 as the highwatermark".format(partition)) + 0L + } + case _ => fatal("Unrecognized version of the highwatermark checkpoint file " + version) + System.exit(1) + -1L + } + } + }finally { + hwFileLock.unlock() + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 358d617..520bbf0 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -33,6 +33,8 @@ import org.I0Itec.zkclient.ZkClient import kafka.common._ import kafka.utils.{ZkUtils, Pool, SystemTime, Logging} import kafka.network.RequestChannel.Response +import kafka.cluster.Broker +import kafka.controller.KafkaController /** @@ -42,14 +44,21 @@ class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, val zkClient: ZkClient, val brokerId: Int, - val config: KafkaConfig) extends Logging { + val config: KafkaConfig, + val controller: KafkaController) extends Logging { private val producerRequestPurgatory = new ProducerRequestPurgatory(replicaManager.config.producerPurgatoryPurgeIntervalRequests) private val fetchRequestPurgatory = new FetchRequestPurgatory(requestChannel, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) private val delayedRequestMetrics = new DelayedRequestMetrics - + /* following 3 data structures are updated by the update metadata request + * and is queried by the topic metadata request. */ + var leaderCache: mutable.Map[TopicAndPartition, PartitionStateInfo] = + new mutable.HashMap[TopicAndPartition, PartitionStateInfo]() +// private var allBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]() + private var aliveBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]() + private val partitionMetadataLock = new Object this.logIdent = "[KafkaApi-%d] ".format(brokerId) /** @@ -65,6 +74,8 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.MetadataKey => handleTopicMetadataRequest(request) case RequestKeys.LeaderAndIsrKey => handleLeaderAndIsrRequest(request) case RequestKeys.StopReplicaKey => handleStopReplicaRequest(request) + case RequestKeys.UpdateMetadataKey => handleUpdateMetadataRequest(request) + case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request) case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request) case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request) case requestId => throw new KafkaException("Unknown api code " + requestId) @@ -90,7 +101,6 @@ class KafkaApis(val requestChannel: RequestChannel, } } - def handleStopReplicaRequest(request: RequestChannel.Request) { val stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest] val (response, error) = replicaManager.stopReplicas(stopReplicaRequest) @@ -99,6 +109,41 @@ class KafkaApis(val requestChannel: RequestChannel, replicaManager.replicaFetcherManager.shutdownIdleFetcherThreads() } + def handleUpdateMetadataRequest(request: RequestChannel.Request) { + val updateMetadataRequest = request.requestObj.asInstanceOf[UpdateMetadataRequest] + val stateChangeLogger = replicaManager.stateChangeLogger + if(updateMetadataRequest.controllerEpoch < replicaManager.controllerEpoch) { + val stateControllerEpochErrorMessage = ("Broker %d received update metadata request with correlation id %d from an " + + "old controller %d with epoch %d. Latest known controller epoch is %d").format(brokerId, + updateMetadataRequest.correlationId, updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, + replicaManager.controllerEpoch) + stateChangeLogger.warn(stateControllerEpochErrorMessage) + throw new ControllerMovedException(stateControllerEpochErrorMessage) + } + partitionMetadataLock synchronized { + replicaManager.controllerEpoch = updateMetadataRequest.controllerEpoch + // cache the list of alive brokers in the cluster + updateMetadataRequest.aliveBrokers.foreach(b => aliveBrokers.put(b.id, b)) + updateMetadataRequest.partitionStateInfos.foreach { partitionState => + leaderCache.put(partitionState._1, partitionState._2) + if(stateChangeLogger.isTraceEnabled) + stateChangeLogger.trace(("Broker %d cached leader info %s for partition %s in response to UpdateMetadata request " + + "sent by controller %d epoch %d with correlation id %d").format(brokerId, partitionState._2, partitionState._1, + updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId)) + } + } + val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(updateMetadataResponse))) + } + + def handleControlledShutdownRequest(request: RequestChannel.Request) { + val controlledShutdownRequest = request.requestObj.asInstanceOf[ControlledShutdownRequest] + val partitionsRemaining = controller.shutdownBroker(controlledShutdownRequest.brokerId) + val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId, + ErrorMapping.NoError, partitionsRemaining) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse))) + } + /** * Check if a partitionData from a produce request can unblock any * DelayedFetch requests. @@ -218,16 +263,18 @@ class KafkaApis(val requestChannel: RequestChannel, Runtime.getRuntime.halt(1) null case utpe: UnknownTopicOrPartitionException => - warn("Produce request: " + utpe.getMessage) + warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( + producerRequest.correlationId, producerRequest.clientId, topicAndPartition, utpe.getMessage)) new ProduceResult(topicAndPartition, utpe) case nle: NotLeaderForPartitionException => - warn("Produce request: " + nle.getMessage) + warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( + producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nle.getMessage)) new ProduceResult(topicAndPartition, nle) case e => BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() - error("Error processing ProducerRequest with correlation id %d from client %s on %s:%d" - .format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition.topic, topicAndPartition.partition), e) + error("Error processing ProducerRequest with correlation id %d from client %s on partition %s" + .format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition), e) new ProduceResult(topicAndPartition, e) } } @@ -305,10 +352,12 @@ class KafkaApis(val requestChannel: RequestChannel, // since failed fetch requests metric is supposed to indicate failure of a broker in handling a fetch request // for a partition it is the leader for case utpe: UnknownTopicOrPartitionException => - warn("Fetch request: " + utpe.getMessage) + warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( + fetchRequest.correlationId, fetchRequest.clientId, topic, partition, utpe.getMessage)) new FetchResponsePartitionData(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) case nle: NotLeaderForPartitionException => - warn("Fetch request: " + nle.getMessage) + warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( + fetchRequest.correlationId, fetchRequest.clientId, topic, partition, nle.getMessage)) new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) case t => BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() @@ -384,10 +433,12 @@ class KafkaApis(val requestChannel: RequestChannel, // NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages // are typically transient and there is no value in logging the entire stack trace for the same case utpe: UnknownTopicOrPartitionException => - warn(utpe.getMessage) + warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( + offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition, utpe.getMessage)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case nle: NotLeaderForPartitionException => - warn(nle.getMessage) + warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( + offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition,nle.getMessage)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case e => warn("Error while responding to offset request", e) @@ -458,46 +509,79 @@ class KafkaApis(val requestChannel: RequestChannel, val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] val topicsMetadata = new mutable.ArrayBuffer[TopicMetadata]() val config = replicaManager.config - val uniqueTopics = { + var uniqueTopics = Set.empty[String] + uniqueTopics = { if(metadataRequest.topics.size > 0) metadataRequest.topics.toSet else ZkUtils.getAllTopics(zkClient).toSet } - val topicMetadataList = AdminUtils.fetchTopicMetadataFromZk(uniqueTopics, zkClient) - topicMetadataList.foreach( - topicAndMetadata => { - topicAndMetadata.errorCode match { - case ErrorMapping.NoError => topicsMetadata += topicAndMetadata - case ErrorMapping.UnknownTopicOrPartitionCode => - try { - /* check if auto creation of topics is turned on */ - if (config.autoCreateTopicsEnable) { - try { - AdminUtils.createTopic(zkClient, topicAndMetadata.topic, config.numPartitions, config.defaultReplicationFactor) - info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" - .format(topicAndMetadata.topic, config.numPartitions, config.defaultReplicationFactor)) - } catch { - case e: TopicExistsException => // let it go, possibly another broker created this topic - } - val newTopicMetadata = AdminUtils.fetchTopicMetadataFromZk(topicAndMetadata.topic, zkClient) - topicsMetadata += newTopicMetadata - newTopicMetadata.errorCode match { - case ErrorMapping.NoError => - case _ => throw new KafkaException("Topic metadata for automatically created topic %s does not exist".format(topicAndMetadata.topic)) - } + val topicMetadataList = + partitionMetadataLock synchronized { + uniqueTopics.map { topic => + if(leaderCache.keySet.map(_.topic).contains(topic)) { + val partitionStateInfo = leaderCache.filter(p => p._1.topic.equals(topic)) + val sortedPartitions = partitionStateInfo.toList.sortWith((m1,m2) => m1._1.partition < m2._1.partition) + val partitionMetadata = sortedPartitions.map { case(topicAndPartition, partitionState) => + val replicas = leaderCache(topicAndPartition).allReplicas + var replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq + var leaderInfo: Option[Broker] = None + var isrInfo: Seq[Broker] = Nil + val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch + val leader = leaderIsrAndEpoch.leaderAndIsr.leader + val isr = leaderIsrAndEpoch.leaderAndIsr.isr + debug("%s".format(topicAndPartition) + ";replicas = " + replicas + ", in sync replicas = " + isr + ", leader = " + leader) + try { + if(aliveBrokers.keySet.contains(leader)) + leaderInfo = Some(aliveBrokers(leader)) + else throw new LeaderNotAvailableException("Leader not available for partition %s".format(topicAndPartition)) + isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null) + if(replicaInfo.size < replicas.size) + throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + + replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) + if(isrInfo.size < isr.size) + throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + + isr.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) + new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) + } catch { + case e => + error("Error while fetching metadata for partition %s".format(topicAndPartition), e) + new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, + ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } - } catch { - case e => error("Error while retrieving topic metadata", e) } - case _ => - error("Error while fetching topic metadata for topic " + topicAndMetadata.topic, - ErrorMapping.exceptionFor(topicAndMetadata.errorCode).getCause) - topicsMetadata += topicAndMetadata + new TopicMetadata(topic, partitionMetadata) + } else { + // topic doesn't exist, send appropriate error code + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode) + } } - }) - trace("Sending topic metadata for correlation id %d to client %s".format(metadataRequest.correlationId, metadataRequest.clientId)) - topicsMetadata.foreach(metadata => trace("Sending topic metadata " + metadata.toString)) + } + + // handle auto create topics + topicMetadataList.foreach { topicMetadata => + topicMetadata.errorCode match { + case ErrorMapping.NoError => topicsMetadata += topicMetadata + case ErrorMapping.UnknownTopicOrPartitionCode => + if (config.autoCreateTopicsEnable) { + try { + AdminUtils.createTopic(zkClient, topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor) + info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" + .format(topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor)) + } catch { + case e: TopicExistsException => // let it go, possibly another broker created this topic + } + topicsMetadata += new TopicMetadata(topicMetadata.topic, topicMetadata.partitionsMetadata, ErrorMapping.LeaderNotAvailableCode) + } else { + topicsMetadata += topicMetadata + } + case _ => + debug("Error while fetching topic metadata for topic %s due to %s ".format(topicMetadata.topic, + ErrorMapping.exceptionFor(topicMetadata.errorCode).getClass.getName)) + topicsMetadata += topicMetadata + } + } + trace("Sending topic metadata %s for correlation id %d to client %s".format(topicsMetadata.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) val response = new TopicMetadataResponse(topicsMetadata.toSeq, metadataRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 96cbd62..21f4199 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -207,9 +207,20 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the purge interval (in number of requests) of the producer request purgatory */ val producerPurgatoryPurgeIntervalRequests = props.getInt("producer.purgatory.purge.interval.requests", 10000) + /*********** Controlled shutdown configuration ***********/ + + /** Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens */ + val controlledShutdownMaxRetries = props.getInt("controlled.shutdown.max.retries", 3) + + /** Before each retry, the system needs time to recover from the state that caused the previous failure (Controller + * fail over, replica lag etc). This config determines the amount of time to wait before retrying. */ + val controlledShutdownRetryBackoffMs = props.getInt("controlled.shutdown.retry.backoff.ms", 5000) + + /* enable controlled shutdown of the server */ + val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", false) + /*********** Misc configuration ***********/ /* the maximum size for a metadata entry associated with an offset commit */ val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", 1024) - - } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index f0949c2..34342f5 100644 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -33,14 +33,15 @@ class KafkaRequestHandler(id: Int, brokerId: Int, val requestChannel: RequestCha try { val req = requestChannel.receiveRequest() if(req eq RequestChannel.AllDone) { - trace("receives shut down command, shut down".format(brokerId, id)) + debug("Kafka request handler %d on broker %d received shut down command".format( + id, brokerId)) return } req.dequeueTimeMs = SystemTime.milliseconds - debug("handles request " + req) + trace("Kafka request handler %d on broker %d handling request %s".format(id, brokerId, req)) apis.handle(req) } catch { - case e: Throwable => error("exception when handling request", e) + case e: Throwable => error("Exception when handling request") } } } @@ -55,12 +56,12 @@ class KafkaRequestHandlerPool(val brokerId: Int, this.logIdent = "[Kafka Request Handler on Broker " + brokerId + "], " val threads = new Array[Thread](numThreads) val runnables = new Array[KafkaRequestHandler](numThreads) - for(i <- 0 until numThreads) { + for(i <- 0 until numThreads) { runnables(i) = new KafkaRequestHandler(i, brokerId, requestChannel, apis) threads(i) = Utils.daemonThread("kafka-request-handler-" + i, runnables(i)) threads(i).start() } - + def shutdown() { info("shutting down") for(handler <- runnables) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 5f3b92c..562aa72 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -17,17 +17,20 @@ package kafka.server -import kafka.network.SocketServer import kafka.admin._ import kafka.log.LogConfig import kafka.log.CleanerConfig import kafka.log.LogManager import kafka.utils._ import java.util.concurrent._ +import atomic.{AtomicInteger, AtomicBoolean} import java.io.File -import atomic.AtomicBoolean 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.network.{Receive, BlockingChannel, SocketServer} /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required @@ -37,6 +40,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg this.logIdent = "[Kafka Server " + config.brokerId + "], " private var isShuttingDown = new AtomicBoolean(false) private var shutdownLatch = new CountDownLatch(1) + private var startupComplete = new AtomicBoolean(false); + val correlationId: AtomicInteger = new AtomicInteger(0) var socketServer: SocketServer = null var requestHandlerPool: KafkaRequestHandlerPool = null var logManager: LogManager = null @@ -77,11 +82,11 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.socketRequestMaxBytes) socketServer.startup() - replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager) + replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) kafkaController = new KafkaController(config, zkClient) /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, zkClient, config.brokerId, config) + apis = new KafkaApis(socketServer.requestChannel, replicaManager, zkClient, config.brokerId, config, kafkaController) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) Mx4jLoader.maybeLoad() @@ -99,7 +104,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg registerStats() - + startupComplete.set(true); info("started") } @@ -120,6 +125,91 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg } /** + * Performs controlled shutdown + */ + private def controlledShutdown() { + if (startupComplete.get() && config.controlledShutdownEnable) { + // We request the controller to do a controlled shutdown. On failure, we backoff for a configured period + // of time and try again for a configured number of retries. If all the attempt fails, we simply force + // the shutdown. + var remainingRetries = config.controlledShutdownMaxRetries + info("Starting controlled shutdown") + var channel : BlockingChannel = null; + var prevController : Broker = null + var shutdownSuceeded : Boolean =false + try { + while (!shutdownSuceeded && remainingRetries > 0) { + remainingRetries = remainingRetries - 1 + + // 1. Find the controller and establish a connection to it. + + // Get the current controller info. This is to ensure we use the most recent info to issue the + // controlled shutdown request + val controllerId = ZkUtils.getController(zkClient) + ZkUtils.getBrokerInfo(zkClient, controllerId) match { + case Some(broker) => + if (channel == null || prevController == null || !prevController.equals(broker)) { + // if this is the first attempt or if the controller has changed, create a channel to the most recent + // controller + if (channel != null) { + channel.disconnect() + } + channel = new BlockingChannel(broker.host, broker.port, + BlockingChannel.UseDefaultBufferSize, + BlockingChannel.UseDefaultBufferSize, + config.controllerSocketTimeoutMs) + channel.connect() + prevController = broker + } + case None=> + //ignore and try again + } + + // 2. issue a controlled shutdown to the controller + if (channel != null) { + var response: Receive = null + try { + // send the controlled shutdown request + val request = new ControlledShutdownRequest(correlationId.getAndIncrement, config.brokerId) + channel.send(request) + response = channel.receive() + val shutdownResponse = ControlledShutdownResponse.readFrom(response.buffer) + if (shutdownResponse.errorCode == ErrorMapping.NoError && shutdownResponse.partitionsRemaining != null && + shutdownResponse.partitionsRemaining.size == 0) { + shutdownSuceeded = true + info ("Controlled shutdown succeeded") + } + else { + info("Remaining partitions to move: %s".format(shutdownResponse.partitionsRemaining.mkString(","))) + info("Error code from controller: %d".format(shutdownResponse.errorCode)) + } + } + catch { + case ioe: java.io.IOException => + channel.disconnect() + channel = null + // ignore and try again + } + } + if (!shutdownSuceeded) { + Thread.sleep(config.controlledShutdownRetryBackoffMs) + warn("Retrying controlled shutdown after the previous attempt failed...") + } + } + } + finally { + if (channel != null) { + channel.disconnect() + channel = null + } + } + if (!shutdownSuceeded) { + warn("Proceeding to do an unclean shutdown as all the controlled shutdown attempts failed") + } + } + } + + /** * Shutdown API for shutting down a single instance of the Kafka server. * Shuts down the LogManager, the SocketServer and the log cleaner scheduler thread */ @@ -127,6 +217,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg info("shutting down") val canShutdown = isShuttingDown.compareAndSet(false, true); if (canShutdown) { + Utils.swallow(controlledShutdown()) if(socketServer != null) Utils.swallow(socketServer.shutdown()) if(requestHandlerPool != null) @@ -144,6 +235,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg Utils.swallow(zkClient.close()) shutdownLatch.countDown() + startupComplete.set(false); info("shut down completed") } } diff --git a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala index 79f29df..815c90d 100644 --- a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala +++ b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala @@ -53,8 +53,14 @@ class OffsetCheckpoint(val file: File) extends Logging { // flush and overwrite old file writer.flush() - if(!temp.renameTo(file)) - throw new IOException("File rename from %s to %s failed.".format(temp.getAbsolutePath, file.getAbsolutePath)) + // swap new offset checkpoint file with previous one + if(!temp.renameTo(file)) { + // renameTo() fails on Windows if the destination file exists. + file.delete() + if(!temp.renameTo(file)) { + throw new IOException("File rename from %s to %s failed.".format(temp.getAbsolutePath, file.getAbsolutePath)) + } + } } finally { writer.close() } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala index 7f775ec..351dbba 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala @@ -20,7 +20,8 @@ package kafka.server import kafka.cluster.Broker class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val replicaMgr: ReplicaManager) - extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId, brokerConfig.numReplicaFetchers) { + extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId, + "Replica", brokerConfig.numReplicaFetchers) { override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { new ReplicaFetcherThread("ReplicaFetcherThread-%d-%d".format(fetcherId, sourceBroker.id), sourceBroker, brokerConfig, replicaMgr) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 018c76f..715845b 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -19,8 +19,8 @@ package kafka.server import kafka.cluster.Broker import kafka.message.ByteBufferMessageSet -import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest, FetchResponsePartitionData} -import kafka.common.{KafkaStorageException, TopicAndPartition, ErrorMapping} +import kafka.api.{OffsetRequest, FetchResponsePartitionData} +import kafka.common.{KafkaStorageException, TopicAndPartition} class ReplicaFetcherThread(name:String, sourceBroker: Broker, diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index a7b2146..73c87c6 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -42,7 +42,8 @@ class ReplicaManager(val config: KafkaConfig, time: Time, val zkClient: ZkClient, scheduler: Scheduler, - val logManager: LogManager) extends Logging with KafkaMetricsGroup { + val logManager: LogManager, + val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup { /* epoch of the controller that last changed the leader */ @volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1 private val localBrokerId = config.brokerId @@ -54,7 +55,7 @@ class ReplicaManager(val config: KafkaConfig, val highWatermarkCheckpoints = config.logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap private var hwThreadInitialized = false this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " - private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) + val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) newGauge( "LeaderCount", @@ -283,9 +284,11 @@ class ReplicaManager(val config: KafkaConfig, private def maybeShrinkIsr(): Unit = { trace("Evaluating ISR list of partitions to see which replicas can be removed from the ISR") + var curLeaderPartitions: List[Partition] = null leaderPartitionsLock synchronized { - leaderPartitions.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages)) + curLeaderPartitions = leaderPartitions.toList } + curLeaderPartitions.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages)) } def recordFollowerPosition(topic: String, partitionId: Int, replicaId: Int, offset: Long) = { diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index 0e6d9b8..33d7c2c 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -74,7 +74,6 @@ object ConsumerOffsetChecker extends Logging { val lag = logSize - offset println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format(group, topic, pid, offset, logSize, lag, owner match {case Some(ownerStr) => ownerStr case None => "none"})) - consumer.close() case None => // ignore } case None => @@ -157,6 +156,11 @@ object ConsumerOffsetChecker extends Logging { if (options.has("broker-info")) printBrokerInfo(); + for ((_, consumerOpt) <- consumerMap) + consumerOpt match { + case Some(consumer) => consumer.close() + case None => // ignore + } } finally { for (consumerOpt <- consumerMap.values) { diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 3d22dc7..2d93947 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -28,7 +28,6 @@ import collection.mutable.ListBuffer import kafka.tools.KafkaMigrationTool.{ProducerThread, ProducerDataChannel} import kafka.javaapi - object MirrorMaker extends Logging { def main(args: Array[String]) { @@ -114,23 +113,33 @@ object MirrorMaker extends Logging { else new Blacklist(options.valueOf(blacklistOpt)) - val streams = - connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue(), new DefaultDecoder(), new DefaultDecoder())) + var streams: Seq[KafkaStream[Array[Byte], Array[Byte]]] = Nil + try { + streams = connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue(), new DefaultDecoder(), new DefaultDecoder())).flatten + } catch { + case t => + fatal("Unable to create stream - shutting down mirror maker.") + connectors.foreach(_.shutdown) + } val producerDataChannel = new ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]](bufferSize); val consumerThreads = - streams.flatten.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producerDataChannel, streamAndIndex._2)) + streams.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producerDataChannel, streamAndIndex._2)) val producerThreads = new ListBuffer[ProducerThread]() + def cleanShutdown() { + connectors.foreach(_.shutdown) + consumerThreads.foreach(_.awaitShutdown) + producerThreads.foreach(_.shutdown) + producerThreads.foreach(_.awaitShutdown) + info("Kafka mirror maker shutdown successfully") + } + Runtime.getRuntime.addShutdownHook(new Thread() { override def run() { - connectors.foreach(_.shutdown) - consumerThreads.foreach(_.awaitShutdown) - producerThreads.foreach(_.shutdown) - producerThreads.foreach(_.awaitShutdown) - logger.info("Kafka migration tool shutdown successfully"); + cleanShutdown() } }) @@ -145,6 +154,10 @@ object MirrorMaker extends Logging { consumerThreads.foreach(_.start) producerThreads.foreach(_.start) + + // in case the consumer threads hit a timeout/other exception + consumerThreads.foreach(_.awaitShutdown) + cleanShutdown() } class MirrorMakerThread(stream: KafkaStream[Array[Byte], Array[Byte]], @@ -158,6 +171,7 @@ object MirrorMaker extends Logging { this.setName(threadName) override def run() { + info("Starting mirror maker thread " + threadName) try { for (msgAndMetadata <- stream) { val pd = new KeyedMessage[Array[Byte], Array[Byte]](msgAndMetadata.topic, msgAndMetadata.message) diff --git a/core/src/main/scala/kafka/utils/CommandLineUtils.scala b/core/src/main/scala/kafka/utils/CommandLineUtils.scala index 29f1209..5f563ca 100644 --- a/core/src/main/scala/kafka/utils/CommandLineUtils.scala +++ b/core/src/main/scala/kafka/utils/CommandLineUtils.scala @@ -1,4 +1,20 @@ -package kafka.utils +/** + * 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 joptsimple.{OptionSpec, OptionSet, OptionParser} diff --git a/core/src/main/scala/kafka/utils/DelayedItem.scala b/core/src/main/scala/kafka/utils/DelayedItem.scala index 93c4482..d727649 100644 --- a/core/src/main/scala/kafka/utils/DelayedItem.scala +++ b/core/src/main/scala/kafka/utils/DelayedItem.scala @@ -37,7 +37,7 @@ class DelayedItem[T](val item: T, delay: Long, unit: TimeUnit) extends Delayed w */ def getDelay(unit: TimeUnit): Long = { val elapsedMs = (SystemTime.milliseconds - createdMs) - unit.convert(max(delayMs - elapsedMs, 0), unit) + unit.convert(max(delayMs - elapsedMs, 0), TimeUnit.MILLISECONDS) } def compareTo(d: Delayed): Int = { diff --git a/core/src/main/scala/kafka/utils/FileLock.scala b/core/src/main/scala/kafka/utils/FileLock.scala index a5761b9..b43b4b1 100644 --- a/core/src/main/scala/kafka/utils/FileLock.scala +++ b/core/src/main/scala/kafka/utils/FileLock.scala @@ -1,4 +1,20 @@ -package kafka.utils +/** + * 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 java.io._ import java.nio.channels._ diff --git a/core/src/main/scala/kafka/utils/Json.scala b/core/src/main/scala/kafka/utils/Json.scala index e300f60..3f1252c 100644 --- a/core/src/main/scala/kafka/utils/Json.scala +++ b/core/src/main/scala/kafka/utils/Json.scala @@ -1,4 +1,20 @@ -package kafka.utils +/** + * 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.common._ import scala.collection._ diff --git a/core/src/main/scala/kafka/utils/Topic.scala b/core/src/main/scala/kafka/utils/Topic.scala index e69de29..487329a 100644 --- a/core/src/main/scala/kafka/utils/Topic.scala +++ b/core/src/main/scala/kafka/utils/Topic.scala @@ -0,0 +1,16 @@ +/** + * 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. + */ \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index c230b65..84744eb 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -305,24 +305,21 @@ object ZkUtils extends Logging { * create parrent directory if necessary. Never throw NodeExistException. * Return the updated path zkVersion */ - def updatePersistentPath(client: ZkClient, path: String, data: String): Int = { - var stat: Stat = null + def updatePersistentPath(client: ZkClient, path: String, data: String) = { try { - stat = client.writeData(path, data) - return stat.getVersion + client.writeData(path, data) } catch { case e: ZkNoNodeException => { createParentPath(client, path) try { client.createPersistent(path, data) - // When the new path is created, its zkVersion always starts from 0 - return 0 } catch { case e: ZkNodeExistsException => - stat = client.writeData(path, data) - return stat.getVersion + client.writeData(path, data) + case e2 => throw e2 } } + case e2 => throw e2 } } @@ -332,14 +329,14 @@ object ZkUtils extends Logging { */ def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = { try { - val stat = client.writeData(path, data, expectVersion) + val stat = client.writeDataReturnStat(path, data, expectVersion) debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d" .format(path, data, expectVersion, stat.getVersion)) (true, stat.getVersion) } catch { case e: Exception => - error("Conditional update of path %s with data %s and expected version %d failed".format(path, data, - expectVersion), e) + error("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, + expectVersion, e.getMessage)) (false, -1) } } @@ -350,15 +347,15 @@ object ZkUtils extends Logging { */ def conditionalUpdatePersistentPathIfExists(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = { try { - val stat = client.writeData(path, data, expectVersion) + val stat = client.writeDataReturnStat(path, data, expectVersion) debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d" .format(path, data, expectVersion, stat.getVersion)) (true, stat.getVersion) } catch { case nne: ZkNoNodeException => throw nne case e: Exception => - error("Conditional update of path %s with data %s and expected version %d failed".format(path, data, - expectVersion), e) + error("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, + expectVersion, e.getMessage)) (false, -1) } } @@ -466,15 +463,13 @@ object ZkUtils extends Logging { cluster } - def getPartitionLeaderAndIsrForTopics(zkClient: ZkClient, topics: Seq[String]): mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = { + def getPartitionLeaderAndIsrForTopics(zkClient: ZkClient, topicAndPartitions: Set[TopicAndPartition]) + : mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = { val ret = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch] - val partitionsForTopics = getPartitionsForTopics(zkClient, topics) - for((topic, partitions) <- partitionsForTopics) { - for(partition <- partitions) { - ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition.toInt) match { - case Some(leaderIsrAndControllerEpoch) => ret.put(TopicAndPartition(topic, partition.toInt), leaderIsrAndControllerEpoch) - case None => - } + for(topicAndPartition <- topicAndPartitions) { + ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition) match { + case Some(leaderIsrAndControllerEpoch) => ret.put(topicAndPartition, leaderIsrAndControllerEpoch) + case None => } } ret @@ -777,7 +772,7 @@ class ZKGroupTopicDirs(group: String, topic: String) extends ZKGroupDirs(group) class ZKConfig(props: VerifiableProperties) { /** ZK host string */ - val zkConnect = props.getString("zookeeper.connect", null) + val zkConnect = props.getString("zookeeper.connect") /** zookeeper session timeout */ val zkSessionTimeoutMs = props.getInt("zookeeper.session.timeout.ms", 6000) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index 1754c25..cc394a3 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -121,8 +121,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) // create leaders for all partitions TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1) - val actualReplicaAssignment = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata.map(p => p.replicas) - val actualReplicaList = actualReplicaAssignment.map(r => r.map(b => b.id).toList).toList + val actualReplicaList = leaderForPartitionMap.keys.toArray.map(p => (p -> ZkUtils.getReplicasForPartition(zkClient, topic, p))).toMap assertEquals(expectedReplicaAssignment.size, actualReplicaList.size) for(i <- 0 until actualReplicaList.size) assertEquals(expectedReplicaAssignment.get(i).get, actualReplicaList(i)) @@ -134,37 +133,6 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { } @Test - def testGetTopicMetadata() { - val expectedReplicaAssignment = Map( - 0 -> List(0, 1, 2), - 1 -> List(1, 2, 3)) - val leaderForPartitionMap = Map( - 0 -> 0, - 1 -> 1) - val topic = "auto-topic" - TestUtils.createBrokersInZk(zkClient, List(0, 1, 2, 3)) - AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) - // create leaders for all partitions - TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1) - - val newTopicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient) - newTopicMetadata.errorCode match { - case ErrorMapping.UnknownTopicOrPartitionCode => - fail("Topic " + topic + " should've been automatically created") - case _ => - assertEquals(topic, newTopicMetadata.topic) - assertNotNull("partition metadata list cannot be null", newTopicMetadata.partitionsMetadata) - assertEquals("partition metadata list length should be 2", 2, newTopicMetadata.partitionsMetadata.size) - val actualReplicaAssignment = newTopicMetadata.partitionsMetadata.map(p => p.replicas) - val actualReplicaList = actualReplicaAssignment.map(r => r.map(b => b.id).toList).toList - assertEquals(expectedReplicaAssignment.size, actualReplicaList.size) - for(i <- 0 until actualReplicaList.size) { - assertEquals(expectedReplicaAssignment(i), actualReplicaList(i)) - } - } - } - - @Test def testPartitionReassignmentWithLeaderInNewReplicas() { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" @@ -234,7 +202,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); CheckReassignmentStatus.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; - }, 1000) + }, 2000) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 2, 3", newReplicas, assignedReplicas) // leader should be 2 @@ -316,48 +284,45 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { @Test def testShutdownBroker() { - info("inside testShutdownBroker") val expectedReplicaAssignment = Map(1 -> List(0, 1, 2)) val topic = "test" val partition = 1 // create brokers val serverConfigs = TestUtils.createBrokerConfigs(3).map(new KafkaConfig(_)) + val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) // create the topic AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) - val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, partition, 1000) - // broker 2 should be the leader since it was started first - var leaderBeforeShutdown = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, 1000, None).get - var controllerId = ZkUtils.getController(zkClient) - var controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController + val controllerId = ZkUtils.getController(zkClient) + val controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController var partitionsRemaining = controller.shutdownBroker(2) + var activeServers = servers.filter(s => s.config.brokerId != 2) try { - assertEquals(0, partitionsRemaining) - var topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient) - var leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id - assertTrue(leaderAfterShutdown != leaderBeforeShutdown) - assertEquals(2, controller.controllerContext.partitionLeadershipInfo(TopicAndPartition("test", 1)).leaderAndIsr.isr.size) + // wait for the update metadata request to trickle to the brokers + assertTrue("Topic test not created after timeout", TestUtils.waitUntilTrue(() => + activeServers.foldLeft(true)(_ && _.apis.leaderCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.isr.size != 3), 1000)) + assertEquals(0, partitionsRemaining.size) + var partitionStateInfo = activeServers.head.apis.leaderCache(TopicAndPartition(topic, partition)) + var leaderAfterShutdown = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader + assertEquals(0, leaderAfterShutdown) + assertEquals(2, partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.isr.size) + assertEquals(List(0,1), partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.isr) - leaderBeforeShutdown = leaderAfterShutdown - controllerId = ZkUtils.getController(zkClient) - controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController partitionsRemaining = controller.shutdownBroker(1) - assertEquals(0, partitionsRemaining) - topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient) - leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id - assertTrue(leaderAfterShutdown != leaderBeforeShutdown) - assertEquals(1, controller.controllerContext.partitionLeadershipInfo(TopicAndPartition("test", 1)).leaderAndIsr.isr.size) + assertEquals(0, partitionsRemaining.size) + activeServers = servers.filter(s => s.config.brokerId == 0) + partitionStateInfo = activeServers.head.apis.leaderCache(TopicAndPartition(topic, partition)) + leaderAfterShutdown = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader + assertEquals(0, leaderAfterShutdown) - leaderBeforeShutdown = leaderAfterShutdown - controllerId = ZkUtils.getController(zkClient) - controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController + assertTrue(servers.foldLeft(true)(_ && _.apis.leaderCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) partitionsRemaining = controller.shutdownBroker(0) - assertEquals(1, partitionsRemaining) - topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient) - leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id - assertTrue(leaderAfterShutdown == leaderBeforeShutdown) - assertEquals(1, controller.controllerContext.partitionLeadershipInfo(TopicAndPartition("test", 1)).leaderAndIsr.isr.size) - } finally { + assertEquals(1, partitionsRemaining.size) + // leader doesn't change since all the replicas are shut down + assertTrue(servers.foldLeft(true)(_ && _.apis.leaderCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) + } + finally { servers.foreach(_.shutdown()) } } diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 045cd06..bc415e3 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -87,8 +87,8 @@ object SerializationTestUtils{ def createTestLeaderAndIsrRequest() : LeaderAndIsrRequest = { val leaderAndIsr1 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader1, 1, isr1, 1), 1) val leaderAndIsr2 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader2, 1, isr2, 2), 1) - val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, 3)), - ((topic2, 0), PartitionStateInfo(leaderAndIsr2, 3))) + val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, isr1.toSet)), + ((topic2, 0), PartitionStateInfo(leaderAndIsr2, isr2.toSet))) new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 0, 1, 0, "") } diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 778556e..121b6c5 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -23,7 +23,6 @@ import kafka.integration.KafkaServerTestHarness import kafka.server._ import scala.collection._ import org.scalatest.junit.JUnit3Suite -import org.apache.log4j.{Level, Logger} import kafka.message._ import kafka.serializer._ import kafka.admin.AdminUtils @@ -31,6 +30,7 @@ import org.I0Itec.zkclient.ZkClient import kafka.utils._ import kafka.producer.{ProducerConfig, KeyedMessage, Producer} import java.util.{Collections, Properties} +import org.apache.log4j.{Logger, Level} import kafka.utils.TestUtils._ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging { @@ -97,6 +97,9 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000) + // create a consumer val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) @@ -142,7 +145,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) // send some messages to each broker - val sentMessages3 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++ + val sentMessages3 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++ sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) @@ -167,12 +170,15 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.FATAL) // send some messages to each broker - val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++ + val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++ sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000) + // create a consumer val consumerConfig1 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer1)) @@ -240,9 +246,12 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar def testCompressionSetConsumption() { // send some messages to each broker - val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, 200, DefaultCompressionCodec) ++ + val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, 200, DefaultCompressionCodec) ++ sendMessagesToBrokerPartition(configs.last, topic, 1, 200, DefaultCompressionCodec) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000) + val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) @@ -263,9 +272,12 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.FATAL) // send some messages to each broker - val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, NoCompressionCodec) ++ + val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, NoCompressionCodec) ++ sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, NoCompressionCodec) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000) + val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) @@ -303,6 +315,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar // send some messages to each broker val sentMessages1 = sendMessages(configs.head, nMessages, "batch1", NoCompressionCodec, 1) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + // create a consumer val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) @@ -321,12 +335,14 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val receivedMessages1 = getMessages(nMessages, topicMessageStreams1) assertEquals(sentMessages1, receivedMessages1) + zkConsumerConnector1.shutdown() + zkClient.close() } - def sendMessagesToBrokerPartition(config: KafkaConfig, - topic: String, - partition: Int, - numMessages: Int, + def sendMessagesToBrokerPartition(config: KafkaConfig, + topic: String, + partition: Int, + numMessages: Int, compression: CompressionCodec = NoCompressionCodec): List[String] = { val header = "test-%d-%d".format(config.brokerId, partition) val props = new Properties() diff --git a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala index c046a39..e5703bc 100644 --- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala +++ b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala @@ -56,7 +56,7 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L def testResetToEarliestWhenOffsetTooLow() = assertEquals(NumMessages, resetAndConsume(NumMessages, "smallest", SmallOffset)) - + def testResetToLatestWhenOffsetTooHigh() = assertEquals(0, resetAndConsume(NumMessages, "largest", LargeOffset)) @@ -68,12 +68,16 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L * Returns the count of messages received. */ def resetAndConsume(numMessages: Int, resetTo: String, offset: Long): Int = { + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + val producer: Producer[String, Array[Byte]] = TestUtils.createProducer(TestUtils.getBrokerListStrFromConfigs(configs), new DefaultEncoder(), new StringEncoder()) for(i <- 0 until numMessages) producer.send(new KeyedMessage[String, Array[Byte]](topic, topic, "test".getBytes)) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + // update offset in zookeeper for consumer to jump "forward" in time val dirs = new ZKGroupTopicDirs(group, topic) var consumerProps = TestUtils.createConsumerProperties(zkConnect, group, testConsumer) @@ -98,8 +102,10 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L } catch { case e: ConsumerTimeoutException => info("consumer timed out after receiving " + received + " messages.") + } finally { + producer.close() + consumerConnector.shutdown } - consumerConnector.shutdown received } diff --git a/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala b/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala index c4866eb..c3c7631 100644 --- a/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala +++ b/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala @@ -21,12 +21,12 @@ import kafka.api.FetchRequestBuilder import kafka.message.ByteBufferMessageSet import kafka.server.{KafkaRequestHandler, KafkaConfig} import org.apache.log4j.{Level, Logger} -import org.junit.Assert._ import org.scalatest.junit.JUnit3Suite import scala.collection._ -import kafka.producer.KeyedMessage import kafka.utils._ import kafka.common.{ErrorMapping, KafkaException, OffsetOutOfRangeException} +import kafka.producer.KeyedMessage +import org.junit.Assert.assertEquals /** * End to end tests of the primitive apis against a local server @@ -63,6 +63,8 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness producer.send(producerData:_*) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + var fetchedMessage: ByteBufferMessageSet = null while(fetchedMessage == null || fetchedMessage.validBytes == 0) { val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) @@ -90,6 +92,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness val producedData = List("a_" + topic, "b_" + topic) messages += topic -> producedData producer.send(producedData.map(m => new KeyedMessage[String, String](topic, topic, m)):_*) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) builder.addFetch(topic, offset, 0, 10000) } @@ -132,6 +135,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness builder.addFetch(topic, 0, 0, 10000) } producer.send(produceList: _*) + topics.foreach(topic => TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)) // wait a bit for produced message to be available val request = builder.build() @@ -155,6 +159,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness builder.addFetch(topic, 0, 0, 10000) } producer.send(produceList: _*) + topics.foreach(topic => TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)) producer.send(produceList: _*) // wait a bit for produced message to be available diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index 1c6a01b..5f331d2 100644 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -35,7 +35,7 @@ import kafka.utils.{TestUtils, Utils} * End to end tests of the primitive apis against a local server */ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with ZooKeeperTestHarness { - + val port = TestUtils.choosePort val props = TestUtils.createBrokerConfig(0, port) val config = new KafkaConfig(props) @@ -287,8 +287,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with def testConsumerEmptyTopic() { val newTopic = "new-topic" AdminUtils.createTopic(zkClient, newTopic, 1, 1) - assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() => - AdminUtils.fetchTopicMetadataFromZk(newTopic, zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime)) + TestUtils.waitUntilMetadataIsPropagated(servers, newTopic, 0, 1000) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, newTopic, 0, 500) val fetchResponse = consumer.fetch(new FetchRequestBuilder().addFetch(newTopic, 0, 0, 10000).build()) assertFalse(fetchResponse.messageSet(newTopic, 0).iterator.hasNext) diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala new file mode 100644 index 0000000..cb0e349 --- /dev/null +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -0,0 +1,139 @@ +/** + * 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 org.scalatest.junit.JUnit3Suite +import kafka.zk.ZooKeeperTestHarness +import kafka.utils.TestUtils._ +import junit.framework.Assert._ +import kafka.utils.{ZkUtils, Utils, TestUtils} +import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} +import kafka.cluster.Broker +import kafka.common.ErrorMapping +import kafka.api._ +import kafka.admin.AdminUtils + +class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { + val brokerId1 = 0 + val brokerId2 = 1 + val brokerId3 = 2 + val brokerId4 = 3 + + val port1 = TestUtils.choosePort() + val port2 = TestUtils.choosePort() + val port3 = TestUtils.choosePort() + val port4 = TestUtils.choosePort() + + val enableShutdown = true + val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) + configProps1.put("controlled.shutdown.enable", "true") + val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) + configProps2.put("controlled.shutdown.enable", "true") + val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3) + configProps3.put("controlled.shutdown.enable", "true") + val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4) + configProps4.put("controlled.shutdown.enable", "true") + configProps4.put("controlled.shutdown.retry.backoff.ms", "100") + + var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] + + val partitionId = 0 + + override def setUp() { + super.setUp() + // start all the servers + val server1 = TestUtils.createServer(new KafkaConfig(configProps1)) + val server2 = TestUtils.createServer(new KafkaConfig(configProps2)) + val server3 = TestUtils.createServer(new KafkaConfig(configProps3)) + val server4 = TestUtils.createServer(new KafkaConfig(configProps4)) + + servers ++= List(server1, server2, server3, server4) + } + + override def tearDown() { + servers.map(server => server.shutdown()) + servers.map(server => Utils.rm(server.config.logDirs)) + super.tearDown() + } + + def testRollingBounce { + // start all the brokers + val topic1 = "new-topic1" + val topic2 = "new-topic2" + val topic3 = "new-topic3" + val topic4 = "new-topic4" + + // create topics with 1 partition, 2 replicas, one on each broker + AdminUtils.createTopicWithAssignment(zkClient, topic1, Map(0->Seq(0,1))) + AdminUtils.createTopicWithAssignment(zkClient, topic2, Map(0->Seq(1,2))) + AdminUtils.createTopicWithAssignment(zkClient, topic3, Map(0->Seq(2,3))) + AdminUtils.createTopicWithAssignment(zkClient, topic4, Map(0->Seq(0,3))) + + // wait until leader is elected + var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partitionId, 500) + var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, partitionId, 500) + var leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, partitionId, 500) + var leader4 = waitUntilLeaderIsElectedOrChanged(zkClient, topic4, partitionId, 500) + + debug("Leader for " + topic1 + " is elected to be: %s".format(leader1.getOrElse(-1))) + debug("Leader for " + topic2 + " is elected to be: %s".format(leader1.getOrElse(-1))) + debug("Leader for " + topic3 + "is elected to be: %s".format(leader1.getOrElse(-1))) + debug("Leader for " + topic4 + "is elected to be: %s".format(leader1.getOrElse(-1))) + + assertTrue("Leader should get elected", leader1.isDefined) + assertTrue("Leader should get elected", leader2.isDefined) + assertTrue("Leader should get elected", leader3.isDefined) + assertTrue("Leader should get elected", leader4.isDefined) + + assertTrue("Leader could be broker 0 or broker 1 for " + topic1, (leader1.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 1)) + assertTrue("Leader could be broker 1 or broker 2 for " + topic2, (leader2.getOrElse(-1) == 1) || (leader1.getOrElse(-1) == 2)) + assertTrue("Leader could be broker 2 or broker 3 for " + topic3, (leader3.getOrElse(-1) == 2) || (leader1.getOrElse(-1) == 3)) + assertTrue("Leader could be broker 3 or broker 4 for " + topic4, (leader4.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 3)) + + // Do a rolling bounce and check if leader transitions happen correctly + + // Bring down the leader for the first topic + bounceServer(topic1, 0) + + // Bring down the leader for the second topic + bounceServer(topic2, 1) + + // Bring down the leader for the third topic + bounceServer(topic3, 2) + + // Bring down the leader for the fourth topic + bounceServer(topic4, 3) + } + + private def bounceServer(topic: String, startIndex: Int) { + var prevLeader = 0 + if (isLeaderLocalOnBroker(topic, partitionId, servers(startIndex))) { + servers(startIndex).shutdown() + prevLeader = startIndex + } + else { + servers((startIndex + 1) % 4).shutdown() + prevLeader = (startIndex + 1) % 4 + } + var newleader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 1500) + // Ensure the new leader is different from the old + assertTrue("Leader transition did not happen for " + topic, newleader.getOrElse(-1) != -1 && (newleader.getOrElse(-1) != prevLeader)) + // Start the server back up again + servers(prevLeader).startup() + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index be94254..9998a11 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -22,26 +22,27 @@ import kafka.zk.ZooKeeperTestHarness import kafka.admin.AdminUtils import java.nio.ByteBuffer import junit.framework.Assert._ -import org.easymock.EasyMock -import kafka.network._ import kafka.cluster.Broker import kafka.utils.TestUtils import kafka.utils.TestUtils._ -import kafka.server.{ReplicaManager, KafkaApis, KafkaConfig} +import kafka.server.{KafkaServer, KafkaConfig} +import kafka.api.TopicMetadataRequest import kafka.common.ErrorMapping -import kafka.api.{RequestKeys, TopicMetadata, TopicMetadataResponse, TopicMetadataRequest} +import kafka.client.ClientUtils class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { val props = createBrokerConfigs(1) val configs = props.map(p => new KafkaConfig(p)) - var brokers: Seq[Broker] = null + private var server1: KafkaServer = null + val brokers = configs.map(c => new Broker(c.brokerId,c.hostName,c.port)) override def setUp() { super.setUp() - brokers = TestUtils.createBrokersInZk(zkClient, configs.map(config => config.brokerId)) + server1 = TestUtils.createServer(configs.head) } override def tearDown() { + server1.shutdown() super.tearDown() } @@ -65,16 +66,15 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { // create topic val topic = "test" AdminUtils.createTopic(zkClient, topic, 1, 1) - // set up leader for topic partition 0 - val leaderForPartitionMap = Map( - 0 -> configs.head.brokerId - ) - TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1) - val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0) - val topicMetadata = mockLogManagerAndTestTopic(topicMetadataRequest) - assertEquals("Expecting metadata only for 1 topic", 1, topicMetadata.size) - assertEquals("Expecting metadata for the test topic", "test", topicMetadata.head.topic) - val partitionMetadata = topicMetadata.head.partitionsMetadata + TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0, 1000) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", + 2000,0).topicsMetadata + assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) + assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) + assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) + assertEquals("Expecting metadata for the test topic", "test", topicsMetadata.head.topic) + var partitionMetadata = topicsMetadata.head.partitionsMetadata assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size) assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId) assertEquals(1, partitionMetadata.head.replicas.size) @@ -82,60 +82,55 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { def testGetAllTopicMetadata { // create topic - val topic = "test" - AdminUtils.createTopic(zkClient, topic, 1, 1) - // set up leader for topic partition 0 - val leaderForPartitionMap = Map( - 0 -> configs.head.brokerId - ) - TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1) - val topicMetadataRequest = new TopicMetadataRequest(List(), 0) - val topicMetadata = mockLogManagerAndTestTopic(topicMetadataRequest) - assertEquals("Expecting metadata only for 1 topic", 1, topicMetadata.size) - assertEquals("Expecting metadata for the test topic", "test", topicMetadata.head.topic) - val partitionMetadata = topicMetadata.head.partitionsMetadata - assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size) - assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId) - assertEquals(1, partitionMetadata.head.replicas.size) + val topic1 = "testGetAllTopicMetadata1" + val topic2 = "testGetAllTopicMetadata2" + AdminUtils.createTopic(zkClient, topic1, 1, 1) + AdminUtils.createTopic(zkClient, topic2, 1, 1) + + // wait for leader to be elected for both topics + TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic1, 0, 1000) + TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic2, 0, 1000) + + // issue metadata request with empty list of topics + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokers, "TopicMetadataTest-testGetAllTopicMetadata", + 2000, 0).topicsMetadata + assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) + assertEquals(2, topicsMetadata.size) + assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) + assertEquals(ErrorMapping.NoError, topicsMetadata.last.partitionsMetadata.head.errorCode) + val partitionMetadataTopic1 = topicsMetadata.head.partitionsMetadata + val partitionMetadataTopic2 = topicsMetadata.last.partitionsMetadata + assertEquals("Expecting metadata for 1 partition", 1, partitionMetadataTopic1.size) + assertEquals("Expecting partition id to be 0", 0, partitionMetadataTopic1.head.partitionId) + assertEquals(1, partitionMetadataTopic1.head.replicas.size) + assertEquals("Expecting metadata for 1 partition", 1, partitionMetadataTopic2.size) + assertEquals("Expecting partition id to be 0", 0, partitionMetadataTopic2.head.partitionId) + assertEquals(1, partitionMetadataTopic2.head.replicas.size) } def testAutoCreateTopic { // auto create topic - val topic = "test" - - val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0) - val topicMetadata = mockLogManagerAndTestTopic(topicMetadataRequest) - assertEquals("Expecting metadata only for 1 topic", 1, topicMetadata.size) - assertEquals("Expecting metadata for the test topic", "test", topicMetadata.head.topic) - val partitionMetadata = topicMetadata.head.partitionsMetadata + val topic = "testAutoCreateTopic" + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testAutoCreateTopic", + 2000,0).topicsMetadata + assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode) + assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) + assertEquals("Expecting metadata for the test topic", topic, topicsMetadata.head.topic) + assertEquals(0, topicsMetadata.head.partitionsMetadata.size) + + // wait for leader to be elected + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0, 1000) + + // retry the metadata for the auto created topic + topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", + 2000,0).topicsMetadata + assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) + assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) + var partitionMetadata = topicsMetadata.head.partitionsMetadata assertEquals("Expecting metadata for 1 partition", 1, partitionMetadata.size) assertEquals("Expecting partition id to be 0", 0, partitionMetadata.head.partitionId) - assertEquals(0, partitionMetadata.head.replicas.size) - assertEquals(None, partitionMetadata.head.leader) - assertEquals(ErrorMapping.LeaderNotAvailableCode, partitionMetadata.head.errorCode) - } - - private def mockLogManagerAndTestTopic(request: TopicMetadataRequest): Seq[TopicMetadata] = { - // topic metadata request only requires 1 call from the replica manager - val replicaManager = EasyMock.createMock(classOf[ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.replay(replicaManager) - - - val serializedMetadataRequest = TestUtils.createRequestByteBuffer(request) - - // create the kafka request handler - val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, zkClient, 1, configs.head) - - // call the API (to be tested) to get metadata - apis.handleTopicMetadataRequest(new RequestChannel.Request - (processor=0, requestKey=RequestKeys.MetadataKey, buffer=serializedMetadataRequest, startTimeMs=1)) - val metadataResponse = requestChannel.receiveResponse(0).responseSend.asInstanceOf[BoundedByteBufferSend].buffer - - // check assertions - val topicMetadata = TopicMetadataResponse.readFrom(metadataResponse).topicsMetadata - - topicMetadata + assertEquals(1, partitionMetadata.head.replicas.size) + assertTrue(partitionMetadata.head.leader.isDefined) } } diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index a185ce4..3a4b41b 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -1,4 +1,20 @@ -package kafka.log +/** + * 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.log import junit.framework.Assert._ import java.util.concurrent.atomic._ diff --git a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala index 3b2c069..9213a5d 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala @@ -25,6 +25,7 @@ import org.scalatest.junit.JUnitSuite import scala.collection._ import scala.util.Random import kafka.utils.TestUtils +import kafka.common.InvalidOffsetException class OffsetIndexTest extends JUnitSuite { @@ -89,7 +90,7 @@ class OffsetIndexTest extends JUnitSuite { assertWriteFails("Append should fail on a full index", idx, idx.maxEntries + 1, classOf[IllegalArgumentException]) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[InvalidOffsetException]) def appendOutOfOrder() { idx.append(51, 0) idx.append(50, 1) diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index ba035b3..f2f91e8 100644 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -260,10 +260,11 @@ class AsyncProducerTest extends JUnit3Suite { topicPartitionInfos = topicPartitionInfos) try { handler.partitionAndCollate(producerDataList) - fail("Should fail with UnknownTopicOrPartitionException") } catch { - case e: UnknownTopicOrPartitionException => // expected, do nothing + // should not throw any exception + case e => fail("Should not throw any exception") + } } @@ -291,10 +292,10 @@ class AsyncProducerTest extends JUnit3Suite { topicPartitionInfos = topicPartitionInfos) try { handler.handle(producerDataList) - fail("Should fail with NoBrokersForPartitionException") + fail("Should fail with FailedToSendMessageException") } catch { - case e: NoBrokersForPartitionException => // expected, do nothing + case e: FailedToSendMessageException => // we retry on any exception now } } @@ -419,6 +420,8 @@ class AsyncProducerTest extends JUnit3Suite { val response2 = ProducerResponse(0, Map((TopicAndPartition("topic1", 0), ProducerResponseStatus(ErrorMapping.NoError, 0L)))) val mockSyncProducer = EasyMock.createMock(classOf[SyncProducer]) + // don't care about config mock + EasyMock.expect(mockSyncProducer.config).andReturn(EasyMock.anyObject()).anyTimes() EasyMock.expect(mockSyncProducer.send(request1)).andThrow(new RuntimeException) // simulate SocketTimeoutException EasyMock.expect(mockSyncProducer.send(request2)).andReturn(response1) EasyMock.expect(mockSyncProducer.send(request3)).andReturn(response2) diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 1d82598..72eccc1 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -23,14 +23,16 @@ import kafka.message.Message import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer} import kafka.zk.ZooKeeperTestHarness import org.apache.log4j.{Level, Logger} -import org.junit.Assert._ import org.junit.Test import kafka.utils._ import java.util import kafka.admin.AdminUtils import util.Properties import kafka.api.FetchRequestBuilder -import kafka.common.{KafkaException, ErrorMapping, FailedToSendMessageException} +import org.junit.Assert.assertTrue +import org.junit.Assert.assertFalse +import org.junit.Assert.assertEquals +import kafka.common.{ErrorMapping, FailedToSendMessageException} class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ @@ -43,6 +45,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ private var consumer1: SimpleConsumer = null private var consumer2: SimpleConsumer = null private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) + private var servers = List.empty[KafkaServer] private val props1 = TestUtils.createBrokerConfig(brokerId1, port1) private val config1 = new KafkaConfig(props1) { @@ -60,6 +63,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ // set up 2 brokers with 4 partitions each server1 = TestUtils.createServer(config1) server2 = TestUtils.createServer(config2) + servers = List(server1,server2) val props = new Properties() props.put("host", "localhost") @@ -68,7 +72,6 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ consumer1 = new SimpleConsumer("localhost", port1, 1000000, 64*1024, "") consumer2 = new SimpleConsumer("localhost", port2, 100, 64*1024, "") - // temporarily set request handler logger to a higher level requestHandlerLogger.setLevel(Level.FATAL) } @@ -85,10 +88,11 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ @Test def testUpdateBrokerPartitionInfo() { - AdminUtils.createTopic(zkClient, "new-topic", 1, 2) - assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() => - AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime)) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500) + val topic = "new-topic" + AdminUtils.createTopic(zkClient, topic, 1, 2) + // wait until the update metadata request for new topic reaches all servers + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 500) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) val props1 = new util.Properties() props1.put("metadata.broker.list", "localhost:80,localhost:81") @@ -96,10 +100,10 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val producerConfig1 = new ProducerConfig(props1) val producer1 = new Producer[String, String](producerConfig1) try{ - producer1.send(new KeyedMessage[String, String]("new-topic", "test", "test1")) + producer1.send(new KeyedMessage[String, String](topic, "test", "test1")) fail("Test should fail because the broker list provided are not valid") } catch { - case e: KafkaException => + case e: FailedToSendMessageException => case oe => fail("fails with exception", oe) } finally { producer1.close() @@ -111,7 +115,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val producerConfig2= new ProducerConfig(props2) val producer2 = new Producer[String, String](producerConfig2) try{ - producer2.send(new KeyedMessage[String, String]("new-topic", "test", "test1")) + producer2.send(new KeyedMessage[String, String](topic, "test", "test1")) } catch { case e => fail("Should succeed sending the message", e) } finally { @@ -124,7 +128,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val producerConfig3 = new ProducerConfig(props3) val producer3 = new Producer[String, String](producerConfig3) try{ - producer3.send(new KeyedMessage[String, String]("new-topic", "test", "test1")) + producer3.send(new KeyedMessage[String, String](topic, "test", "test1")) } catch { case e => fail("Should succeed sending the message", e) } finally { @@ -149,27 +153,27 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val producerConfig1 = new ProducerConfig(props1) val producerConfig2 = new ProducerConfig(props2) + val topic = "new-topic" // create topic with 1 partition and await leadership - AdminUtils.createTopic(zkClient, "new-topic", 1, 2) - assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() => - AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime)) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500) + AdminUtils.createTopic(zkClient, topic, 1, 2) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) val producer1 = new Producer[String, String](producerConfig1) val producer2 = new Producer[String, String](producerConfig2) // Available partition ids should be 0. - producer1.send(new KeyedMessage[String, String]("new-topic", "test", "test1")) - producer1.send(new KeyedMessage[String, String]("new-topic", "test", "test2")) + producer1.send(new KeyedMessage[String, String](topic, "test", "test1")) + producer1.send(new KeyedMessage[String, String](topic, "test", "test2")) // get the leader - val leaderOpt = ZkUtils.getLeaderForPartition(zkClient, "new-topic", 0) + val leaderOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) assertTrue("Leader for topic new-topic partition 0 should exist", leaderOpt.isDefined) val leader = leaderOpt.get val messageSet = if(leader == server1.config.brokerId) { - val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) + val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) response1.messageSet("new-topic", 0).iterator.toBuffer }else { - val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) + val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) response2.messageSet("new-topic", 0).iterator.toBuffer } assertEquals("Should have fetched 2 messages", 2, messageSet.size) @@ -178,7 +182,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ producer1.close() try { - producer2.send(new KeyedMessage[String, String]("new-topic", "test", "test2")) + producer2.send(new KeyedMessage[String, String](topic, "test", "test2")) fail("Should have timed out for 3 acks.") } catch { @@ -200,19 +204,25 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ props.put("request.required.acks", "1") props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + val topic = "new-topic" // create topic - AdminUtils.createTopicWithAssignment(zkClient, "new-topic", Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0))) - assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() => - AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime)) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 1, 500) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 2, 500) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 3, 500) + AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0))) + // waiting for 1 partition is enough + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 2, 500) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 3, 500) val config = new ProducerConfig(props) val producer = new Producer[String, String](config) - // Available partition ids should be 0, 1, 2 and 3, all lead and hosted only on broker 0 - producer.send(new KeyedMessage[String, String]("new-topic", "test", "test1")) + try { + // Available partition ids should be 0, 1, 2 and 3, all lead and hosted only + // on broker 0 + producer.send(new KeyedMessage[String, String](topic, "test", "test1")) + } catch { + case e => fail("Unexpected exception: " + e) + } // kill the broker server1.shutdown @@ -220,7 +230,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ try { // These sends should fail since there are no available brokers - producer.send(new KeyedMessage[String, String]("new-topic", "test", "test1")) + producer.send(new KeyedMessage[String, String](topic, "test", "test1")) fail("Should fail since no leader exists for the partition.") } catch { case e => // success @@ -228,12 +238,12 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ // restart server 1 server1.startup() - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) try { // cross check if broker 1 got the messages - val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) - val messageSet1 = response1.messageSet("new-topic", 0).iterator + val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) + val messageSet1 = response1.messageSet(topic, 0).iterator assertTrue("Message set should have 1 message", messageSet1.hasNext) assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes), messageSet1.next.message) assertFalse("Message set should have another message", messageSet1.hasNext) @@ -252,22 +262,22 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ props.put("request.timeout.ms", String.valueOf(timeoutMs)) props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) props.put("request.required.acks", "1") - + props.put("client.id","ProducerTest-testAsyncSendCanCorrectlyFailWithTimeout") val config = new ProducerConfig(props) val producer = new Producer[String, String](config) + val topic = "new-topic" // create topics in ZK - AdminUtils.createTopicWithAssignment(zkClient, "new-topic", Map(0->Seq(0,1))) - assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() => - AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime)) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500) + AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(0,1))) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) // do a simple test to make sure plumbing is okay try { // this message should be assigned to partition 0 whose leader is on broker 0 - producer.send(new KeyedMessage[String, String]("new-topic", "test", "test")) + producer.send(new KeyedMessage[String, String](topic, "test", "test")) // cross check if brokers got the messages - val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) + val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) val messageSet1 = response1.messageSet("new-topic", 0).iterator assertTrue("Message set should have 1 message", messageSet1.hasNext) assertEquals(new Message("test".getBytes), messageSet1.next.message) @@ -283,7 +293,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ try { // this message should be assigned to partition 0 whose leader is on broker 0, but // broker 0 will not response within timeoutMs millis. - producer.send(new KeyedMessage[String, String]("new-topic", "test", "test")) + producer.send(new KeyedMessage[String, String](topic, "test", "test")) } catch { case e: FailedToSendMessageException => /* success */ case e: Exception => fail("Not expected", e) @@ -320,4 +330,3 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ } } } - diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 9963502..456e538 100644 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -25,7 +25,8 @@ import org.junit._ import org.junit.Assert._ import kafka.common._ import kafka.cluster.Replica -import kafka.utils._ +import kafka.utils.{SystemTime, KafkaScheduler, TestUtils, MockTime, Utils} +import java.util.concurrent.atomic.AtomicBoolean class HighwatermarkPersistenceTest extends JUnit3Suite { @@ -55,7 +56,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { val scheduler = new KafkaScheduler(2) scheduler.startup // create replica manager - val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0)) + val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0), new AtomicBoolean(false)) replicaManager.startup() replicaManager.checkpointHighWatermarks() var fooPartition0Hw = hwmFor(replicaManager, topic, 0) @@ -94,7 +95,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { val scheduler = new KafkaScheduler(2) scheduler.startup // create replica manager - val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0)) + val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, logManagers(0), new AtomicBoolean(false)) replicaManager.startup() replicaManager.checkpointHighWatermarks() var topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index 6184f42..7026432 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -24,6 +24,7 @@ import org.easymock.EasyMock import kafka.log.Log import org.junit.Assert._ import kafka.utils._ +import java.util.concurrent.atomic.AtomicBoolean class IsrExpirationTest extends JUnit3Suite { @@ -80,7 +81,7 @@ class IsrExpirationTest extends JUnit3Suite { private def getPartitionWithAllReplicasInIsr(topic: String, partitionId: Int, time: Time, config: KafkaConfig, localLog: Log): Partition = { val leaderId=config.brokerId - val replicaManager = new ReplicaManager(config, time, null, null, null) + val replicaManager = new ReplicaManager(config, time, null, null, null, new AtomicBoolean(false)) val partition = replicaManager.getOrCreatePartition(topic, partitionId, 1) val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog)) diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 48c63ad..bb6fbdf 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -132,7 +132,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val leaderAndIsr = new collection.mutable.HashMap[(String, Int), LeaderIsrAndControllerEpoch] leaderAndIsr.put((topic, partitionId), new LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId2, List(brokerId1, brokerId2)), 2)) - val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, 1)).toMap + val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, Set(0,1))).toMap val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, controllerId, staleControllerEpoch, 0, "") diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index c5f39cb..298ba71 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -50,6 +50,8 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { // create topic AdminUtils.createTopic(zkClient, topic, 1, 1) + TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) + // send some messages producer.send(sent1.map(m => new KeyedMessage[Int, String](topic, 0, m)):_*) @@ -65,11 +67,12 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { server = new KafkaServer(config) server.startup() + // wait for the broker to receive the update metadata request after startup + TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) + producer = new Producer[Int, String](new ProducerConfig(producerConfig)) val consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "") - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) - var fetchedMessage: ByteBufferMessageSet = null while(fetchedMessage == null || fetchedMessage.validBytes == 0) { val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).maxWait(0).build()) diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 1c6f615..bab436d 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -20,7 +20,7 @@ import kafka.cluster.{Partition, Replica} import kafka.log.Log import kafka.message.{ByteBufferMessageSet, Message} import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.utils.{Time, TestUtils, MockTime} +import kafka.utils.{ZkUtils, Time, TestUtils, MockTime} import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient import org.scalatest.junit.JUnit3Suite @@ -57,7 +57,9 @@ class SimpleFetchTest extends JUnit3Suite { val fetchSize = 100 val messages = new Message("test-message".getBytes()) - val zkClient = EasyMock.createMock(classOf[ZkClient]) + // create nice mock since we don't particularly care about zkclient calls + val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) + EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) EasyMock.replay(zkClient) val log = EasyMock.createMock(classOf[kafka.log.Log]) @@ -85,10 +87,12 @@ class SimpleFetchTest extends JUnit3Suite { EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() EasyMock.replay(replicaManager) + val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) + // start a request channel with 2 processors and a queue size of 5 (this is more or less arbitrary) // don't provide replica or leader callbacks since they will not be tested here val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head) + val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) // This request (from a follower) wants to read up to 2*HW but should only get back up to HW bytes into the log val goodFetch = new FetchRequestBuilder() @@ -124,7 +128,8 @@ class SimpleFetchTest extends JUnit3Suite { val followerReplicaId = configs(1).brokerId val followerLEO = 15 - val zkClient = EasyMock.createMock(classOf[ZkClient]) + val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) + EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) EasyMock.replay(zkClient) val log = EasyMock.createMock(classOf[kafka.log.Log]) @@ -153,8 +158,10 @@ class SimpleFetchTest extends JUnit3Suite { EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() EasyMock.replay(replicaManager) + val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) + val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head) + val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) /** * This fetch, coming from a replica, requests all data at offset "15". Because the request is coming diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 00ea98f..6b343e3 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -38,6 +38,7 @@ import kafka.api._ import collection.mutable.Map import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} import kafka.common.TopicAndPartition +import junit.framework.Assert /** @@ -148,6 +149,7 @@ object TestUtils extends Logging { props.put("zookeeper.sync.time.ms", "200") props.put("auto.commit.interval.ms", "1000") props.put("rebalance.max.retries", "4") + props.put("auto.offset.reset", "smallest") props } @@ -506,6 +508,12 @@ object TestUtils extends Logging { byteBuffer.rewind() byteBuffer } + + def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int, timeout: Long) = { + Assert.assertTrue("Partition [%s,%d] metadata not propagated after timeout".format(topic, partition), + TestUtils.waitUntilTrue(() => + servers.foldLeft(true)(_ && _.apis.leaderCache.keySet.contains(TopicAndPartition(topic, partition))), timeout)) + } } diff --git a/project/Build.scala b/project/Build.scala index f177215..bad93db 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -26,11 +26,26 @@ object KafkaBuild extends Build { val buildNumber = SettingKey[String]("build-number", "Build number defaults to $BUILD_NUMBER environment variable") val releaseName = SettingKey[String]("release-name", "the full name of this release") val commonSettings = Seq( - organization := "org.apache", + organization := "org.apache.kafka", + pomExtra := + + org.apache + apache + 10 + + + + Apache 2 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + +, scalacOptions ++= Seq("-deprecation", "-unchecked", "-g:none"), crossScalaVersions := Seq("2.8.0","2.8.2", "2.9.1", "2.9.2"), scalaVersion := "2.8.0", - version := "0.8.0-SNAPSHOT", + version := "0.8.0-beta1", + publishTo := Some("Apache Maven Repo" at "https://repository.apache.org/service/local/staging/deploy/maven2"), + credentials += Credentials(Path.userHome / ".m2" / ".credentials"), buildNumber := System.getProperty("build.number", ""), version <<= (buildNumber, version) { (build, version) => if (build == "") version else version + "+" + build}, releaseName <<= (name, version, scalaVersion) {(name, version, scalaVersion) => name + "_" + scalaVersion + "-" + version}, diff --git a/project/plugins.sbt b/project/plugins.sbt index e8c3e53..7abe703 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -3,3 +3,7 @@ resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline. addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.8") addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") + +resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) + +addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6") \ No newline at end of file diff --git a/system_test/replication_testsuite/testcase_0119/testcase_0119_properties.json b/system_test/replication_testsuite/testcase_0119/testcase_0119_properties.json index ceb06aa..442a481 100644 --- a/system_test/replication_testsuite/testcase_0119/testcase_0119_properties.json +++ b/system_test/replication_testsuite/testcase_0119/testcase_0119_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties"