diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index e055d67..978447d 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -22,42 +22,26 @@ fi base_dir=$(dirname $0)/.. +SCALA_VERSION=2.8.0 -USER_HOME=$(eval echo ~${USER}) -ivyPath=$(echo "$USER_HOME/.ivy2/cache") - -snappy=$(echo "$ivyPath/org.xerial.snappy/snappy-java/bundles/snappy-java-1.0.4.1.jar") -CLASSPATH=$CLASSPATH:$snappy - -library=$(echo "$ivyPath/org.scala-lang/scala-library/jars/scala-library-2.8.0.jar") -CLASSPATH=$CLASSPATH:$library - -compiler=~$(echo "$ivyPath/org.scala-lang/scala-compiler/jars/scala-compiler-2.8.0.jar") -CLASSPATH=$CLASSPATH:$compiler - -log4j=$(echo "$ivyPath/log4j/log4j/jars/log4j-1.2.15.jar") -CLASSPATH=$CLASSPATH:$log4j - -slf=$(echo "$ivyPath/org.slf4j/slf4j-api/jars/slf4j-api-1.6.4.jar") -CLASSPATH=$CLASSPATH:$slf - -zookeeper=$(echo "$ivyPath/org.apache.zookeeper/zookeeper/jars/zookeeper-3.3.4.jar") -CLASSPATH=$CLASSPATH:$zookeeper - -jopt=$(echo "$ivyPath/net.sf.jopt-simple/jopt-simple/jars/jopt-simple-3.2.jar") -CLASSPATH=$CLASSPATH:$jopt - +# assume all dependencies have been packaged into one jar with sbt-assembly's task "assembly-package-dependency" for file in $base_dir/core/target/scala-2.8.0/*.jar; do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/core/lib/*.jar; +for file in $base_dir/perf/target/scala-${SCALA_VERSION}/kafka*.jar; +do + CLASSPATH=$CLASSPATH:$file +done + +# classpath addition for release +for file in $base_dir/libs/*.jar; do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/perf/target/scala-2.8.0/kafka*.jar; +for file in $base_dir/kafka*.jar; do CLASSPATH=$CLASSPATH:$file done diff --git a/config/consumer.properties b/config/consumer.properties index 9dbd583..7343cbc 100644 --- a/config/consumer.properties +++ b/config/consumer.properties @@ -14,13 +14,13 @@ # limitations under the License. # see kafka.consumer.ConsumerConfig for more details -# zk connection string +# Zookeeper connection string # comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=127.0.0.1:2181 +zookeeper.connect=127.0.0.1:2181 # timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 +zookeeper.connection.timeout.ms=1000000 #consumer group id group.id=test-consumer-group diff --git a/config/producer.properties b/config/producer.properties index a1c8cb2..162b8a6 100644 --- a/config/producer.properties +++ b/config/producer.properties @@ -18,7 +18,7 @@ # list of brokers used for bootstrapping # format: host1:port1,host2:port2 ... -broker.list=localhost:9092 +metadata.broker.list=localhost:9092 # name of the partitioner class for partitioning events; default partition spreads data randomly #partitioner.class= @@ -31,7 +31,7 @@ producer.type=sync compression.codec=none # message encoder -serializer.class=kafka.serializer.StringEncoder +serializer.class=kafka.serializer.DefaultEncoder # allow topic level compression #compressed.topics= diff --git a/config/server.properties b/config/server.properties index 5a16caf..f9d54bc 100644 --- a/config/server.properties +++ b/config/server.properties @@ -97,15 +97,15 @@ log.cleanup.interval.mins=1 ############################# Zookeeper ############################# -# Zk connection string (see zk docs for details). +# Zookeeper connection string (see zookeeper docs for details). # This is a comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". # You can also append an optional chroot string to the urls to specify the # root directory for all kafka znodes. -zk.connect=localhost:2181 +zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 +zookeeper.connection.timeout.ms=1000000 # metrics reporter properties kafka.metrics.polling.interval.secs=5 diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java index df17978..4b1d117 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java @@ -70,7 +70,7 @@ public class DataGenerator { System.out.println("server uri:" + _uri.toString()); Properties producerProps = new Properties(); - producerProps.put("broker.list", String.format("%s:%d", _uri.getHost(), _uri.getPort())); + producerProps.put("metadata.broker.list", String.format("%s:%d", _uri.getHost(), _uri.getPort())); producerProps.put("send.buffer.bytes", String.valueOf(TCP_BUFFER_SIZE)); producerProps.put("connect.timeout.ms", String.valueOf(CONNECT_TIMEOUT)); producerProps.put("reconnect.interval", String.valueOf(RECONNECT_INTERVAL)); diff --git a/contrib/hadoop-producer/README.md b/contrib/hadoop-producer/README.md index 1bd3721..547c1ef 100644 --- a/contrib/hadoop-producer/README.md +++ b/contrib/hadoop-producer/README.md @@ -87,7 +87,7 @@ compression codec, one would add the "kafka.output.compression.codec" parameter compression). For easier debugging, the above values as well as the Kafka broker information -(kafka.broker.list), the topic (kafka.output.topic), and the schema +(kafka.metadata.broker.list), the topic (kafka.output.topic), and the schema (kafka.output.schema) are injected into the job's configuration. By default, the Hadoop producer uses Kafka's sync producer as asynchronous operation doesn't make sense in the batch Hadoop case. 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 aa1f944..32f096c 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 @@ -124,8 +124,8 @@ public class KafkaOutputFormat extends OutputFormat // URL: kafka:/// // e.g. kafka://kafka-server:9000,kafka-server2:9000/foobar String brokerList = uri.getAuthority(); - props.setProperty("broker.list", brokerList); - job.set(KAFKA_CONFIG_PREFIX + ".broker.list", brokerList); + props.setProperty("metadata.broker.list", brokerList); + job.set(KAFKA_CONFIG_PREFIX + ".metadata.broker.list", brokerList); if (uri.getPath() == null || uri.getPath().length() <= 1) throw new KafkaException("no topic specified in kafka uri"); diff --git a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java index a381ccd..6eea635 100644 --- a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java +++ b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java @@ -17,6 +17,7 @@ package kafka.bridge.hadoop; import java.io.IOException; +import java.util.Arrays; import java.util.LinkedList; import java.util.List; import kafka.javaapi.producer.Producer; @@ -62,7 +63,9 @@ public class KafkaRecordWriter extends RecordWriter if (value instanceof byte[]) valBytes = (byte[]) value; else if (value instanceof BytesWritable) - valBytes = ((BytesWritable) value).getBytes(); + // BytesWritable.getBytes returns its internal buffer, so .length would refer to its capacity, not the + // intended size of the byte array contained. We need to use BytesWritable.getLength for the true size. + valBytes = Arrays.copyOf(((BytesWritable) value).getBytes(), ((BytesWritable) value).getLength()); else throw new IllegalArgumentException("KafkaRecordWriter expects byte array value to publish"); diff --git a/core/build.sbt b/core/build.sbt index 211aaf9..405ea55 100644 --- a/core/build.sbt +++ b/core/build.sbt @@ -1,5 +1,6 @@ import sbt._ import Keys._ +import AssemblyKeys._ name := "kafka" @@ -11,8 +12,10 @@ libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _ ) libraryDependencies ++= Seq( "org.apache.zookeeper" % "zookeeper" % "3.3.4", - "com.github.sgroschupf" % "zkclient" % "0.1", + "com.101tec" % "zkclient" % "0.2", "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", "org.easymock" % "easymock" % "3.0" % "test", "junit" % "junit" % "4.1" % "test" ) @@ -24,4 +27,5 @@ libraryDependencies <<= (scalaVersion, libraryDependencies) { (sv, deps) => }) } +assemblySettings diff --git a/core/lib/metrics-annotation-3.0.0-c0c8be71.jar b/core/lib/metrics-annotation-3.0.0-c0c8be71.jar deleted file mode 100644 index dba9d2b..0000000 Binary files a/core/lib/metrics-annotation-3.0.0-c0c8be71.jar and /dev/null differ diff --git a/core/lib/metrics-core-3.0.0-c0c8be71.jar b/core/lib/metrics-core-3.0.0-c0c8be71.jar deleted file mode 100644 index 529a69b..0000000 Binary files a/core/lib/metrics-core-3.0.0-c0c8be71.jar and /dev/null differ diff --git a/core/lib/zkclient-20120522.jar b/core/lib/zkclient-20120522.jar deleted file mode 100644 index 225e97e..0000000 Binary files a/core/lib/zkclient-20120522.jar and /dev/null differ diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index b896182..a1c485e 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -214,7 +214,7 @@ object AdminUtils extends Logging { try { Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head) } catch { - case e => throw new LeaderNotAvailableException("Leader not available for topic %s partition %d".format(topic, partition), e) + case e => throw new LeaderNotAvailableException("Leader not available for partition [%s,%d]".format(topic, partition), e) } case None => throw new LeaderNotAvailableException("No leader exists for partition " + partition) } @@ -233,7 +233,7 @@ object AdminUtils extends Logging { new PartitionMetadata(partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) } catch { case e => - error("Error while fetching metadata for partition [%s,%d]".format(topic, partition), e) + debug("Error while fetching metadata for partition [%s,%d]".format(topic, partition), e) new PartitionMetadata(partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } diff --git a/core/src/main/scala/kafka/admin/ListTopicCommand.scala b/core/src/main/scala/kafka/admin/ListTopicCommand.scala deleted file mode 100644 index f91eca2..0000000 --- a/core/src/main/scala/kafka/admin/ListTopicCommand.scala +++ /dev/null @@ -1,90 +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.admin - -import joptsimple.OptionParser -import org.I0Itec.zkclient.ZkClient -import kafka.utils.{Utils, ZKStringSerializer, ZkUtils} -import kafka.common.ErrorMapping - -object ListTopicCommand { - - def main(args: Array[String]): Unit = { - val parser = new OptionParser - val topicOpt = parser.accepts("topic", "REQUIRED: The topic to be deleted.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - .defaultsTo("") - val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + - "Multiple URLS can be given to allow fail-over.") - .withRequiredArg - .describedAs("urls") - .ofType(classOf[String]) - - val options = parser.parse(args : _*) - - for(arg <- List(zkConnectOpt)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } - - val topic = options.valueOf(topicOpt) - val zkConnect = options.valueOf(zkConnectOpt) - var zkClient: ZkClient = null - try { - var topicList: Seq[String] = Nil - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) - - if (topic == "") - topicList = ZkUtils.getChildren(zkClient, ZkUtils.BrokerTopicsPath) - else - topicList = List(topic) - - if (topicList.size <= 0) - println("no topics exist!") - - for (t <- topicList) - showTopic(t, zkClient) - } - catch { - case e => - println("list topic failed because of " + e.getMessage) - println(Utils.stackTrace(e)) - } - finally { - if (zkClient != null) - zkClient.close() - } - } - - def showTopic(topic: String, zkClient: ZkClient) { - val topicMetaData = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient) - topicMetaData.errorCode match { - case ErrorMapping.UnknownTopicOrPartitionCode => - println("topic " + topic + " doesn't exist!") - case _ => - println("topic: " + topic) - for (part <- topicMetaData.partitionsMetadata) - println(part.toString) - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala index a2afd16..53fc433 100644 --- a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala +++ b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala @@ -54,7 +54,7 @@ object PreferredReplicaLeaderElectionCommand extends Logging { if (!options.has(jsonFileOpt)) ZkUtils.getAllPartitions(zkClient) else - parsePreferredReplicaJsonData(Utils.readFileAsString(options.valueOf(jsonFileOpt))) + parsePreferredReplicaElectionData(Utils.readFileAsString(options.valueOf(jsonFileOpt))) val preferredReplicaElectionCommand = new PreferredReplicaLeaderElectionCommand(zkClient, partitionsForPreferredReplicaElection) preferredReplicaElectionCommand.moveLeaderToPreferredReplica() @@ -69,7 +69,7 @@ object PreferredReplicaLeaderElectionCommand extends Logging { } } - def parsePreferredReplicaJsonData(jsonString: String): immutable.Set[TopicAndPartition] = { + def parsePreferredReplicaElectionData(jsonString: String): immutable.Set[TopicAndPartition] = { Json.parseFull(jsonString) match { case Some(m) => m.asInstanceOf[Map[String, Any]].get("partitions") match { @@ -102,7 +102,7 @@ object PreferredReplicaLeaderElectionCommand extends Logging { } catch { case nee: ZkNodeExistsException => val partitionsUndergoingPreferredReplicaElection = - PreferredReplicaLeaderElectionCommand.parsePreferredReplicaJsonData(ZkUtils.readData(zkClient, zkPath)._1) + PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(ZkUtils.readData(zkClient, zkPath)._1) throw new AdminOperationException("Preferred replica leader election currently in progress for " + "%s. Aborting operation".format(partitionsUndergoingPreferredReplicaElection)) case e2 => throw new AdminOperationException(e2.toString) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index d364608..e604233 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -104,19 +104,37 @@ object TopicCommand { } def describeTopic(zkClient: ZkClient, opts: TopicCommandOptions) { - CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.topicOpt) - val topics = opts.options.valuesOf(opts.topicOpt) - val metadata = AdminUtils.fetchTopicMetadataFromZk(topics.toSet, zkClient) - for(md <- metadata) { - println(md.topic) - val config = AdminUtils.fetchTopicConfig(zkClient, md.topic) - println("\tconfigs: " + config.map(kv => kv._1 + " = " + kv._2).mkString(", ")) - println("\tpartitions: " + md.partitionsMetadata.size) - for(pd <- md.partitionsMetadata) { - println("\t\tpartition " + pd.partitionId) - println("\t\tleader: " + (if(pd.leader.isDefined) formatBroker(pd.leader.get) else "none")) - println("\t\treplicas: " + pd.replicas.map(formatBroker).mkString(", ")) - println("\t\tisr: " + pd.isr.map(formatBroker).mkString(", ")) + var topics: Seq[String] = opts.options.valuesOf(opts.topicOpt).toSeq.sorted + if (topics.size <= 0) + topics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath).sorted + val reportUnderReplicatedPartitions = if (opts.options.has(opts.reportUnderReplicatedPartitionsOpt)) true else false + val reportUnavailablePartitions = if (opts.options.has(opts.reportUnavailablePartitionsOpt)) true else false + val liveBrokers = ZkUtils.getAllBrokersInCluster(zkClient).map(_.id).toSet + for (topic <- topics) { + ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic) match { + case Some(topicPartitionAssignment) => + val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) + if (!reportUnavailablePartitions && !reportUnderReplicatedPartitions) { + println(topic) + val config = AdminUtils.fetchTopicConfig(zkClient, topic) + println("\tconfigs: " + config.map(kv => kv._1 + " = " + kv._2).mkString(", ")) + println("\tpartitions: " + sortedPartitions.size) + } + for ((partitionId, assignedReplicas) <- sortedPartitions) { + val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionId) + val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitionId) + if ((!reportUnderReplicatedPartitions && !reportUnavailablePartitions) || + (reportUnderReplicatedPartitions && inSyncReplicas.size < assignedReplicas.size) || + (reportUnavailablePartitions && (!leader.isDefined || !liveBrokers.contains(leader.get)))) { + print("\t\ttopic: " + topic) + print("\tpartition: " + partitionId) + print("\tleader: " + (if(leader.isDefined) leader.get else "none")) + print("\treplicas: " + assignedReplicas.mkString(",")) + println("\tisr: " + inSyncReplicas.mkString(",")) + } + } + case None => + println("topic " + topic + " doesn't exist!") } } } @@ -177,7 +195,11 @@ object TopicCommand { .describedAs("broker_id_for_part1_replica1 : broker_id_for_part1_replica2 , " + "broker_id_for_part2_replica1 : broker_id_for_part2_replica2 , ...") .ofType(classOf[String]) - + val reportUnderReplicatedPartitionsOpt = parser.accepts("under-replicated-partitions", + "if set when describing topics, only show under replicated partitions") + val reportUnavailablePartitionsOpt = parser.accepts("unavailable-partitions", + "if set when describing topics, only show partitions whose leader is not available") + val options = parser.parse(args : _*) } diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index b40522d..68e64d6 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -79,11 +79,17 @@ case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndContr 4 /* replication factor */ size } + + override def toString(): String = { + val partitionStateInfo = new StringBuilder + partitionStateInfo.append("(LeaderAndIsrInfo:" + leaderIsrAndControllerEpoch.toString) + partitionStateInfo.append(",ReplicationFactor:" + replicationFactor + ")") + partitionStateInfo.toString() + } } object LeaderAndIsrRequest { val CurrentVersion = 0.shortValue - val DefaultClientId = "" val IsInit: Boolean = true val NotInit: Boolean = false val DefaultAckTimeout: Int = 1000 @@ -122,13 +128,13 @@ case class LeaderAndIsrRequest (versionId: Short, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[(String, Int), PartitionStateInfo], - leaders: Set[Broker]) + aliveLeaders: Set[Broker]) extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey), correlationId) { - def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], liveBrokers: Set[Broker], controllerId: Int, - controllerEpoch: Int, correlationId: Int) = { - this(LeaderAndIsrRequest.CurrentVersion, correlationId, LeaderAndIsrRequest.DefaultClientId, LeaderAndIsrRequest.DefaultAckTimeout, - controllerId, controllerEpoch, partitionStateInfos, liveBrokers) + 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, + controllerId, controllerEpoch, partitionStateInfos, aliveLeaders) } def writeTo(buffer: ByteBuffer) { @@ -144,8 +150,8 @@ case class LeaderAndIsrRequest (versionId: Short, buffer.putInt(key._2) value.writeTo(buffer) } - buffer.putInt(leaders.size) - leaders.foreach(_.writeTo(buffer)) + buffer.putInt(aliveLeaders.size) + aliveLeaders.foreach(_.writeTo(buffer)) } def sizeInBytes(): Int = { @@ -160,22 +166,22 @@ case class LeaderAndIsrRequest (versionId: Short, for((key, value) <- partitionStateInfos) size += (2 + key._1.length) /* topic */ + 4 /* partition */ + value.sizeInBytes /* partition state info */ size += 4 /* number of leader brokers */ - for(broker <- leaders) + for(broker <- aliveLeaders) size += broker.sizeInBytes /* broker info */ size } override def toString(): String = { val leaderAndIsrRequest = new StringBuilder - leaderAndIsrRequest.append("Name: " + this.getClass.getSimpleName) - leaderAndIsrRequest.append("; Version: " + versionId) - leaderAndIsrRequest.append("; Controller: " + controllerId) - leaderAndIsrRequest.append("; ControllerEpoch: " + controllerEpoch) - leaderAndIsrRequest.append("; CorrelationId: " + correlationId) - leaderAndIsrRequest.append("; ClientId: " + clientId) - leaderAndIsrRequest.append("; AckTimeoutMs: " + ackTimeoutMs + " ms") - leaderAndIsrRequest.append("; PartitionStateInfo: " + partitionStateInfos.mkString(",")) - leaderAndIsrRequest.append("; Leaders: " + leaders.mkString(",")) + leaderAndIsrRequest.append("Name:" + this.getClass.getSimpleName) + leaderAndIsrRequest.append(";Version:" + versionId) + leaderAndIsrRequest.append(";Controller:" + controllerId) + 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/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index 409de76..a0d68c5 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -108,6 +108,18 @@ case class PartitionMetadata(partitionId: Int, buffer.putInt(isr.size) isr.foreach(r => buffer.putInt(r.id)) } + + override def toString(): String = { + val partitionMetadataString = new StringBuilder + partitionMetadataString.append("\tpartition " + partitionId) + partitionMetadataString.append("\tleader: " + (if(leader.isDefined) formatBroker(leader.get) else "none")) + partitionMetadataString.append("\treplicas: " + replicas.map(formatBroker).mkString(",")) + partitionMetadataString.append("\tisr: " + isr.map(formatBroker).mkString(",")) + partitionMetadataString.append("\tisUnderReplicated: %s".format(if(isr.size < replicas.size) "true" else "false")) + partitionMetadataString.toString() + } + + private def formatBroker(broker: Broker) = broker.id + " (" + broker.host + ":" + broker.port + ")" } diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index a3d88ea..025d3ab 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -7,6 +7,7 @@ import kafka.producer._ import kafka.common.KafkaException import kafka.utils.{Utils, Logging} import java.util.Properties +import util.Random /** * Helper functions common to clients (producer, consumer, or admin) @@ -16,7 +17,7 @@ object ClientUtils extends Logging{ /** * Used by the producer to send a metadata request since it has access to the ProducerConfig * @param topics The topics for which the metadata needs to be fetched - * @param brokers The brokers in the cluster as configured on the producer through broker.list + * @param brokers The brokers in the cluster as configured on the producer through metadata.broker.list * @param producerConfig The producer's config * @return topic metadata response */ @@ -26,9 +27,12 @@ object ClientUtils extends Logging{ val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, producerConfig.clientId, topics.toSeq) var topicMetadataResponse: TopicMetadataResponse = null var t: Throwable = null - while(i < brokers.size && !fetchMetaDataSucceeded) { - val producer: SyncProducer = ProducerPool.createSyncProducer(producerConfig, brokers(i)) - info("Fetching metadata with correlation id %d for %d topic(s) %s".format(correlationId, topics.size, topics)) + // shuffle the list of brokers before sending metadata requests so that most requests don't get routed to the + // same broker + val shuffledBrokers = Random.shuffle(brokers) + while(i < shuffledBrokers.size && !fetchMetaDataSucceeded) { + val producer: SyncProducer = ProducerPool.createSyncProducer(producerConfig, shuffledBrokers(i)) + info("Fetching metadata from broker %s with correlation id %d for %d topic(s) %s".format(shuffledBrokers(i), correlationId, topics.size, topics)) try { topicMetadataResponse = producer.send(topicMetadataRequest) fetchMetaDataSucceeded = true @@ -36,7 +40,7 @@ object ClientUtils extends Logging{ catch { case e => warn("Fetching topic metadata with correlation id %d for topics [%s] from broker [%s] failed" - .format(correlationId, topics, brokers(i).toString), e) + .format(correlationId, topics, shuffledBrokers(i).toString), e) t = e } finally { i = i + 1 @@ -44,7 +48,7 @@ object ClientUtils extends Logging{ } } if(!fetchMetaDataSucceeded){ - throw new KafkaException("fetching topic metadata for topics [%s] from broker [%s] failed".format(topics, brokers), t) + 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)) } @@ -60,7 +64,7 @@ object ClientUtils extends Logging{ */ def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String, timeoutMs: Int): TopicMetadataResponse = { val props = new Properties() - props.put("broker.list", brokers.map(_.getConnectionString()).mkString(",")) + 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) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index f79a622..ce2a634 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -25,9 +25,10 @@ import kafka.log.LogConfig import kafka.server.ReplicaManager import com.yammer.metrics.core.Gauge import kafka.metrics.KafkaMetricsGroup -import kafka.common._ import kafka.controller.{LeaderIsrAndControllerEpoch, KafkaController} import org.apache.log4j.Logger +import kafka.message.ByteBufferMessageSet +import kafka.common.{TopicAndPartition, NotLeaderForPartitionException, ErrorMapping} /** @@ -62,7 +63,7 @@ class Partition(val topic: String, newGauge( topic + "-" + partitionId + "-UnderReplicated", new Gauge[Int] { - def getValue = { + def value = { if (isUnderReplicated) 1 else 0 } } @@ -82,8 +83,11 @@ class Partition(val topic: String, if (isReplicaLocal(replicaId)) { val config = LogConfig.fromProps(logManager.defaultConfig.toProps, AdminUtils.fetchTopicConfig(zkClient, topic)) val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) - val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParent) - val offset = checkpoint.read.getOrElse(TopicAndPartition(topic, partitionId), 0L).min(log.logEndOffset) + val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParent) + val offsetMap = checkpoint.read + if (!offsetMap.contains(TopicAndPartition(topic, partitionId))) + warn("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId)) + val offset = offsetMap.getOrElse(TopicAndPartition(topic, partitionId), 0L).min(log.logEndOffset) val localReplica = new Replica(replicaId, this, time, offset, Some(log)) addReplicaIfNotExists(localReplica) } else { @@ -169,7 +173,7 @@ class Partition(val topic: String, * 4. start a fetcher to the new leader */ def makeFollower(controllerId: Int, topic: String, partitionId: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, - liveBrokers: Set[Broker], correlationId: Int): Boolean = { + aliveLeaders: Set[Broker], correlationId: Int): Boolean = { leaderIsrUpdateLock synchronized { val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr if (leaderEpoch >= leaderAndIsr.leaderEpoch) { @@ -182,13 +186,16 @@ class Partition(val topic: String, // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch + // make sure local replica exists. This reads the last check pointed high watermark from disk. On startup, it is + // important to ensure that this operation happens for every single partition in a leader and isr request, else + // some high watermark values could be overwritten with 0. This leads to replicas fetching from the earliest offset + // on the leader + val localReplica = getOrCreateReplica() val newLeaderBrokerId: Int = leaderAndIsr.leader - liveBrokers.find(_.id == newLeaderBrokerId) match { + aliveLeaders.find(_.id == newLeaderBrokerId) match { case Some(leaderBroker) => // stop fetcher thread to previous leader replicaFetcherManager.removeFetcher(topic, partitionId) - // make sure local replica exists - val localReplica = getOrCreateReplica() localReplica.log.get.truncateTo(localReplica.highWatermark) inSyncReplicas = Set.empty[Replica] leaderEpoch = leaderAndIsr.leaderEpoch @@ -198,7 +205,7 @@ class Partition(val topic: String, replicaFetcherManager.addFetcher(topic, partitionId, localReplica.logEndOffset, leaderBroker) 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] became unavailable during the state change operation" + " controller %d epoch %d since leader %d for partition [%s,%d] is unavailable during the state change operation" .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, newLeaderBrokerId, topic, partitionId)) } @@ -220,7 +227,8 @@ class Partition(val topic: String, if (!inSyncReplicas.contains(replica) && replica.logEndOffset >= leaderHW) { // expand ISR val newInSyncReplicas = inSyncReplicas + replica - info("Expanding ISR for topic %s partition %d to %s".format(topic, partitionId, newInSyncReplicas.map(_.brokerId).mkString(", "))) + info("Expanding ISR for partition [%s,%d] from %s to %s" + .format(topic, partitionId, inSyncReplicas.map(_.brokerId).mkString(","), newInSyncReplicas.map(_.brokerId).mkString(","))) // update ISR in ZK and cache updateIsr(newInSyncReplicas) replicaManager.isrExpandRate.mark() @@ -258,16 +266,20 @@ class Partition(val topic: String, } } - def maybeIncrementLeaderHW(leaderReplica: Replica) { + /** + * There is no need to acquire the leaderIsrUpdate lock here since all callers of this private API acquire that lock + * @param leaderReplica + */ + private def maybeIncrementLeaderHW(leaderReplica: Replica) { val allLogEndOffsets = inSyncReplicas.map(_.logEndOffset) val newHighWatermark = allLogEndOffsets.min val oldHighWatermark = leaderReplica.highWatermark if(newHighWatermark > oldHighWatermark) { leaderReplica.highWatermark = newHighWatermark - debug("Highwatermark for topic %s partition %d updated to %d".format(topic, partitionId, newHighWatermark)) + debug("Highwatermark for partition [%s,%d] updated to %d".format(topic, partitionId, newHighWatermark)) } else - debug("Old hw for topic %s partition %d is %d. New hw is %d. All leo's are %s" + debug("Old hw for partition [%s,%d] is %d. New hw is %d. All leo's are %s" .format(topic, partitionId, oldHighWatermark, newHighWatermark, allLogEndOffsets.mkString(","))) } @@ -279,7 +291,7 @@ class Partition(val topic: String, if(outOfSyncReplicas.size > 0) { val newInSyncReplicas = inSyncReplicas -- outOfSyncReplicas assert(newInSyncReplicas.size > 0) - info("Shrinking ISR for topic %s partition %d from %s to %s".format(topic, partitionId, + info("Shrinking ISR for partition [%s,%d] from %s to %s".format(topic, partitionId, inSyncReplicas.map(_.brokerId).mkString(","), newInSyncReplicas.map(_.brokerId).mkString(","))) // update ISR in zk and in cache updateIsr(newInSyncReplicas) @@ -304,18 +316,38 @@ class Partition(val topic: String, val candidateReplicas = inSyncReplicas - leaderReplica // Case 1 above val possiblyStuckReplicas = candidateReplicas.filter(r => r.logEndOffset < leaderLogEndOffset) - debug("Possibly stuck replicas for topic %s partition %d are %s".format(topic, partitionId, - possiblyStuckReplicas.map(_.brokerId).mkString(","))) + if(possiblyStuckReplicas.size > 0) + debug("Possibly stuck replicas for partition [%s,%d] are %s".format(topic, partitionId, + possiblyStuckReplicas.map(_.brokerId).mkString(","))) val stuckReplicas = possiblyStuckReplicas.filter(r => r.logEndOffsetUpdateTimeMs < (time.milliseconds - keepInSyncTimeMs)) - debug("Stuck replicas for topic %s partition %d are %s".format(topic, partitionId, stuckReplicas.map(_.brokerId).mkString(","))) + if(stuckReplicas.size > 0) + debug("Stuck replicas for partition [%s,%d] are %s".format(topic, partitionId, stuckReplicas.map(_.brokerId).mkString(","))) // Case 2 above val slowReplicas = candidateReplicas.filter(r => r.logEndOffset >= 0 && (leaderLogEndOffset - r.logEndOffset) > keepInSyncMessages) - debug("Slow replicas for topic %s partition %d are %s".format(topic, partitionId, slowReplicas.map(_.brokerId).mkString(","))) + if(slowReplicas.size > 0) + debug("Slow replicas for partition [%s,%d] are %s".format(topic, partitionId, slowReplicas.map(_.brokerId).mkString(","))) stuckReplicas ++ slowReplicas } + def appendMessagesToLeader(messages: ByteBufferMessageSet) = { + leaderIsrUpdateLock synchronized { + val leaderReplicaOpt = leaderReplicaIfLocal() + leaderReplicaOpt match { + case Some(leaderReplica) => + val log = leaderReplica.log.get + val info = log.append(messages, assignOffsets = true) + // we may need to increment high watermark since ISR could be down to 1 + maybeIncrementLeaderHW(leaderReplica) + info + case None => + throw new NotLeaderForPartitionException("Leader not local for partition [%s,%d] on broker %d" + .format(topic, partitionId, localBrokerId)) + } + } + } + private def updateIsr(newIsr: Set[Replica]) { - info("Updated ISR for topic %s partition %d to %s".format(topic, partitionId, newIsr.mkString(", "))) + debug("Updated ISR for partition [%s,%d] to %s".format(topic, partitionId, newIsr.mkString(","))) val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion) // use the epoch of the controller that made the leadership decision, instead of the current controller epoch val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath(zkClient, diff --git a/core/src/main/scala/kafka/cluster/Replica.scala b/core/src/main/scala/kafka/cluster/Replica.scala index 321ab58..5e659b4 100644 --- a/core/src/main/scala/kafka/cluster/Replica.scala +++ b/core/src/main/scala/kafka/cluster/Replica.scala @@ -40,10 +40,10 @@ class Replica(val brokerId: Int, if (!isLocal) { logEndOffsetValue.set(newLogEndOffset) logEndOffsetUpdateTimeMsValue.set(time.milliseconds) - trace("Setting log end offset for replica %d for topic %s partition %d to %d" + trace("Setting log end offset for replica %d for partition [%s,%d] to %d" .format(brokerId, topic, partitionId, logEndOffsetValue.get())) } else - throw new KafkaException("Shouldn't set logEndOffset for replica %d topic %s partition %d since it's local" + throw new KafkaException("Shouldn't set logEndOffset for replica %d partition [%s,%d] since it's local" .format(brokerId, topic, partitionId)) } @@ -66,11 +66,11 @@ class Replica(val brokerId: Int, def highWatermark_=(newHighWatermark: Long) { if (isLocal) { - trace("Setting hw for replica %d topic %s partition %d on broker %d to %d" + trace("Setting hw for replica %d partition [%s,%d] on broker %d to %d" .format(brokerId, topic, partitionId, brokerId, newHighWatermark)) highWatermarkValue.set(newHighWatermark) } else - throw new KafkaException("Unable to set highwatermark for replica %d topic %s partition %d since it's not local" + throw new KafkaException("Unable to set highwatermark for replica %d partition [%s,%d] since it's not local" .format(brokerId, topic, partitionId)) } @@ -78,7 +78,7 @@ class Replica(val brokerId: Int, if (isLocal) highWatermarkValue.get() else - throw new KafkaException("Unable to get highwatermark for replica %d topic %s partition %d since it's not local" + throw new KafkaException("Unable to get highwatermark for replica %d partition [%s,%d] since it's not local" .format(brokerId, topic, partitionId)) } diff --git a/core/src/main/scala/kafka/common/LeaderElectionNotNeededException.scala b/core/src/main/scala/kafka/common/LeaderElectionNotNeededException.scala new file mode 100644 index 0000000..ca89d25 --- /dev/null +++ b/core/src/main/scala/kafka/common/LeaderElectionNotNeededException.scala @@ -0,0 +1,27 @@ +/** + * 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 + +/** + * This exception is thrown when new leader election is not necessary. + */ +class LeaderElectionNotNeededException(message: String, cause: Throwable) extends RuntimeException(message, cause) { + def this(message: String) = this(message, null) + def this() = this(null, null) +} + + diff --git a/core/src/main/scala/kafka/common/NoReplicaOnlineException.scala b/core/src/main/scala/kafka/common/NoReplicaOnlineException.scala new file mode 100644 index 0000000..a1e1279 --- /dev/null +++ b/core/src/main/scala/kafka/common/NoReplicaOnlineException.scala @@ -0,0 +1,28 @@ +/** + * 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 + + +/** + * This exception is thrown by the leader elector in the controller when leader election fails for a partition since + * all the replicas for a partition are offline + */ +class NoReplicaOnlineException(message: String, cause: Throwable) extends RuntimeException(message, cause) { + def this(message: String) = this(message, null) + def this() = this(null, null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/PartitionOfflineException.scala b/core/src/main/scala/kafka/common/PartitionOfflineException.scala deleted file mode 100644 index 3367708..0000000 --- a/core/src/main/scala/kafka/common/PartitionOfflineException.scala +++ /dev/null @@ -1,28 +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 - - -/** - * This exception is thrown by the leader elector in the controller when leader election fails for a partition since - * all the replicas for a partition are offline - */ -class PartitionOfflineException(message: String, cause: Throwable) extends RuntimeException(message, cause) { - def this(message: String) = this(message, null) - def this() = this(null, 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 d6c4a51..e2b0041 100644 --- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala @@ -158,7 +158,7 @@ object ConsoleConsumer extends Logging { props.put("auto.commit.enable", "true") props.put("auto.commit.interval.ms", options.valueOf(autoCommitIntervalOpt).toString) props.put("auto.offset.reset", if(options.has(resetBeginningOpt)) "smallest" else "largest") - props.put("zk.connect", options.valueOf(zkConnectOpt)) + props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) props.put("consumer.timeout.ms", options.valueOf(consumerTimeoutMsOpt).toString) val config = new ConsumerConfig(props) val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala index d8c23f2..13c3f77 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala @@ -83,7 +83,7 @@ object Consumer extends Logging { * Create a ConsumerConnector * * @param config at the minimum, need to specify the groupid of the consumer and the zookeeper - * connection string zk.connect. + * connection string zookeeper.connect. */ def create(config: ConsumerConfig): ConsumerConnector = { val consumerConnect = new ZookeeperConsumerConnector(config) @@ -94,7 +94,7 @@ object Consumer extends Logging { * Create a ConsumerConnector * * @param config at the minimum, need to specify the groupid of the consumer and the zookeeper - * connection string zk.connect. + * connection string zookeeper.connect. */ def createJavaConsumerConnector(config: ConsumerConfig): kafka.javaapi.consumer.ConsumerConnector = { val consumerConnect = new kafka.javaapi.consumer.ZookeeperConsumerConnector(config) diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index 3aa7b08..c6250dc 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -43,13 +43,17 @@ class ConsumerFetcherManager(private val consumerIdString: String, private val noLeaderPartitionSet = new mutable.HashSet[TopicAndPartition] private val lock = new ReentrantLock private val cond = lock.newCondition() - private val leaderFinderThread = new ShutdownableThread(consumerIdString + "-leader-finder-thread"){ + private var leaderFinderThread: ShutdownableThread = null + + 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() { lock.lock() try { - if (noLeaderPartitionSet.isEmpty) + if (noLeaderPartitionSet.isEmpty) { + trace("No partition for leader election.") cond.await() + } try { trace("Partitions without leader %s".format(noLeaderPartitionSet)) @@ -93,8 +97,6 @@ class ConsumerFetcherManager(private val consumerIdString: String, Thread.sleep(config.refreshLeaderBackoffMs) } } - leaderFinderThread.start() - override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { new ConsumerFetcherThread( @@ -103,8 +105,9 @@ class ConsumerFetcherManager(private val consumerIdString: String, } def startConnections(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster) { - if (!leaderFinderThread.isRunning.get()) - throw new RuntimeException("%s already shutdown".format(name)) + leaderFinderThread = new LeaderFinderThread(consumerIdString + "-leader-finder-thread") + leaderFinderThread.start() + lock.lock() try { partitionMap = topicInfos.map(tpi => (TopicAndPartition(tpi.topic, tpi.partitionId), tpi)).toMap @@ -116,16 +119,21 @@ class ConsumerFetcherManager(private val consumerIdString: String, } } - def stopAllConnections() { - lock.lock() - // first, clear noLeaderPartitionSet so that no more fetchers can be added to leader_finder_thread - noLeaderPartitionSet.clear() - // second, clear partitionMap - partitionMap = null - lock.unlock() + def stopConnections() { + info("Stopping leader finder thread") + if (leaderFinderThread != null) { + leaderFinderThread.shutdown() + leaderFinderThread = null + } - // third, stop all existing fetchers + info("Stopping all fetchers") closeAllFetchers() + + // no need to hold the lock for the following since leaderFindThread and all fetchers have been stopped + partitionMap = null + noLeaderPartitionSet.clear() + + info("All connections stopped") } def addPartitionsWithError(partitionList: Iterable[TopicAndPartition]) { @@ -141,11 +149,4 @@ class ConsumerFetcherManager(private val consumerIdString: String, lock.unlock() } } - - def shutdown() { - info("shutting down") - leaderFinderThread.shutdown() - stopAllConnections() - info("shutdown completed") - } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 80df1b5..5f9c902 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -45,7 +45,7 @@ class ConsumerFetcherThread(name: String, def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: FetchResponsePartitionData) { val pti = partitionMap(topicAndPartition) if (pti.getFetchOffset != fetchOffset) - throw new RuntimeException("Offset doesn't match for topic %s partition: %d pti offset: %d fetch offset: %d" + throw new RuntimeException("Offset doesn't match for partition [%s,%d] pti offset: %d fetch offset: %d" .format(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset, fetchOffset)) pti.enqueue(partitionData.messages.asInstanceOf[ByteBufferMessageSet]) } diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index 7423141..a4227a4 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -109,7 +109,7 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk def clearCurrentChunk() { try { - info("Clearing the current data chunk for this consumer iterator") + debug("Clearing the current data chunk for this consumer iterator") current.set(null) } } diff --git a/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala b/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala index 9792244..64b702b 100644 --- a/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala +++ b/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala @@ -50,12 +50,12 @@ class PartitionTopicInfo(val topic: String, } /** - * Enqueue a message set for processing + * Enqueue a message set for processing. */ def enqueue(messages: ByteBufferMessageSet) { - val size = messages.sizeInBytes + val size = messages.validBytes if(size > 0) { - val next = nextOffset(messages) + val next = messages.shallowIterator.toSeq.last.nextOffset trace("Updating fetch offset = " + fetchedOffset.get + " to " + next) chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get)) fetchedOffset.set(next) @@ -65,17 +65,6 @@ class PartitionTopicInfo(val topic: String, } } - /** - * Get the next fetch offset after this message set - */ - private def nextOffset(messages: ByteBufferMessageSet): Long = { - var nextOffset = PartitionTopicInfo.InvalidOffset - val iter = messages.shallowIterator - while(iter.hasNext) - nextOffset = iter.next.nextOffset - nextOffset - } - override def toString(): String = topic + ":" + partitionId.toString + ": fetched offset = " + fetchedOffset.get + ": consumed offset = " + consumedOffset.get } diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 0921ce6..3f3a239 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -165,7 +165,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, if (config.autoCommitEnable) scheduler.shutdown() fetcher match { - case Some(f) => f.shutdown + case Some(f) => f.stopConnections case None => } sendShutdownToAllQueues() @@ -486,7 +486,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val allPartitionInfos = topicRegistry.values.map(p => p.values).flatten fetcher match { case Some(f) => - f.stopAllConnections + f.stopConnections clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, messageStreams) info("Committing all offsets after clearing the fetcher queues") /** @@ -653,7 +653,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, newGauge( config.clientId + "-" + config.groupId + "-" + topicThreadId._1 + "-" + topicThreadId._2 + "-FetchQueueSize", new Gauge[Int] { - def getValue = q.size + def value = q.size } ) }) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 6e563d2..3164f78 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -144,7 +144,8 @@ class RequestSendThread(val controllerId: Int, } } -class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (RequestOrResponse) => Unit) => Unit, controllerId: Int) +class ControllerBrokerRequestBatch(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)]] @@ -189,8 +190,9 @@ class ControllerBrokerRequestBatch(sendRequest: (Int, RequestOrResponse, (Reques val broker = m._1 val partitionStateInfos = m._2.toMap val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet - val leaders = liveBrokers.filter(b => leaderIds.contains(b.id)) - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId) + val aliveLeaders = liveBrokers.filter(b => leaderIds.contains(b.id)) + val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, aliveLeaders, controllerId, controllerEpoch, correlationId, + clientId) for (p <- partitionStateInfos) { val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" stateChangeLogger.trace(("Controller %d epoch %d sending %s LeaderAndIsr request with correlationId %d to broker %d " + diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 25a8cfe..02510bd 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -46,7 +46,7 @@ class ControllerContext(val zkClient: ZkClient, val correlationId: AtomicInteger = new AtomicInteger(0), var allTopics: Set[String] = Set.empty, var partitionReplicaAssignment: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map.empty, - var allLeaders: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = mutable.Map.empty, + var partitionLeadershipInfo: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = mutable.Map.empty, var partitionsBeingReassigned: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap, var partitionsUndergoingPreferredReplicaElection: mutable.Set[TopicAndPartition] = @@ -87,21 +87,35 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private val replicaStateMachine = new ReplicaStateMachine(this) private val controllerElector = new ZookeeperLeaderElector(controllerContext, ZkUtils.ControllerPath, onControllerFailover, config.brokerId) + val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext) private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext) - private val brokerRequestBatch = new ControllerBrokerRequestBatch(sendRequest, config.brokerId) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(sendRequest, this.config.brokerId, this.clientId) registerControllerChangedListener() newGauge( "ActiveControllerCount", new Gauge[Int] { - def getValue() = if (isActive) 1 else 0 + def value() = if (isActive) 1 else 0 + } + ) + + newGauge( + "OfflinePartitionsCount", + new Gauge[Int] { + def value(): Int = { + controllerContext.controllerLock synchronized { + controllerContext.partitionLeadershipInfo.count(p => !controllerContext.liveBrokerIds.contains(p._2.leaderAndIsr.leader)) + } + } } ) def epoch = controllerContext.epoch + 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 @@ -137,8 +151,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } def replicatedPartitionsBrokerLeads() = controllerContext.controllerLock.synchronized { - trace("All leaders = " + controllerContext.allLeaders.mkString(",")) - controllerContext.allLeaders.filter { + trace("All leaders = " + controllerContext.partitionLeadershipInfo.mkString(",")) + controllerContext.partitionLeadershipInfo.filter { case (topicAndPartition, leaderIsrAndControllerEpoch) => leaderIsrAndControllerEpoch.leaderAndIsr.leader == id && controllerContext.partitionReplicaAssignment(topicAndPartition).size > 1 }.map(_._1) @@ -151,11 +165,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg val (topic, partition) = topicAndPartition.asTuple // move leadership serially to relinquish lock. controllerContext.controllerLock synchronized { - controllerContext.allLeaders.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch => + controllerContext.partitionLeadershipInfo.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch => if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id) { partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, - controlledShutdownPartitionLeaderSelector) - val newLeaderIsrAndControllerEpoch = controllerContext.allLeaders(topicAndPartition) + controlledShutdownPartitionLeaderSelector) + val newLeaderIsrAndControllerEpoch = controllerContext.partitionLeadershipInfo(topicAndPartition) // mark replica offline only if leadership was moved successfully if (newLeaderIsrAndControllerEpoch.leaderAndIsr.leader != currLeaderIsrAndControllerEpoch.leaderAndIsr.leader) @@ -180,7 +194,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg allPartitionsAndReplicationFactorOnBroker foreach { case(topicAndPartition, replicationFactor) => val (topic, partition) = topicAndPartition.asTuple - if (controllerContext.allLeaders(topicAndPartition).leaderAndIsr.leader != id) { + if (controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader != id) { brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topic, partition, deletePartition = false) removeReplicaFromIsr(topic, partition, id) match { case Some(updatedLeaderIsrAndControllerEpoch) => @@ -224,10 +238,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg partitionStateMachine.registerListeners() replicaStateMachine.registerListeners() initializeControllerContext() - partitionStateMachine.startup() replicaStateMachine.startup() + partitionStateMachine.startup() Utils.registerMBean(this, KafkaController.MBeanName) info("Broker %d is ready to serve as the new controller with epoch %d".format(config.brokerId, epoch)) + initializeAndMaybeTriggerPartitionReassignment() + initializeAndMaybeTriggerPreferredReplicaElection() } else info("Controller has been shut down, aborting startup/failover") @@ -237,7 +253,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * Returns true if this broker is the current controller. */ def isActive(): Boolean = { - controllerContext.controllerChannelManager != null + controllerContext.controllerLock synchronized { + controllerContext.controllerChannelManager != null + } } /** @@ -257,9 +275,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("New broker startup callback for %s".format(newBrokers.mkString(","))) val newBrokersSet = newBrokers.toSet - // update partition state machine - partitionStateMachine.triggerOnlinePartitionStateChange() + // 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{ @@ -289,7 +310,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg val deadBrokersSet = deadBrokers.toSet // trigger OfflinePartition state for all partitions whose current leader is one amongst the dead brokers - val partitionsWithoutLeader = controllerContext.allLeaders.filter(partitionAndLeader => + val partitionsWithoutLeader = controllerContext.partitionLeadershipInfo.filter(partitionAndLeader => deadBrokersSet.contains(partitionAndLeader._2.leaderAndIsr.leader)).keySet partitionStateMachine.handleStateChanges(partitionsWithoutLeader, OfflinePartition) // trigger OnlinePartition state changes for offline or new partitions @@ -321,7 +342,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("New partition creation callback for %s".format(newPartitions.mkString(","))) partitionStateMachine.handleStateChanges(newPartitions, NewPartition) replicaStateMachine.handleStateChanges(getAllReplicasForPartition(newPartitions), NewReplica) - partitionStateMachine.handleStateChanges(newPartitions, OnlinePartition) + partitionStateMachine.handleStateChanges(newPartitions, OnlinePartition, offlinePartitionSelector) replicaStateMachine.handleStateChanges(getAllReplicasForPartition(newPartitions), OnlineReplica) } @@ -370,8 +391,14 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg def onPreferredReplicaElection(partitions: Set[TopicAndPartition]) { info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(","))) - controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions - partitionStateMachine.handleStateChanges(partitions, OnlinePartition, preferredReplicaPartitionLeaderSelector) + try { + controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions + partitionStateMachine.handleStateChanges(partitions, OnlinePartition, preferredReplicaPartitionLeaderSelector) + } catch { + case e => error("Error completing preferred replica leader election for partitions %s".format(partitions.mkString(",")), e) + } finally { + removePartitionsFromPreferredReplicaElection(partitions) + } } /** @@ -450,7 +477,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg controllerContext.liveBrokers = ZkUtils.getAllBrokersInCluster(zkClient).toSet controllerContext.allTopics = ZkUtils.getAllTopics(zkClient).toSet controllerContext.partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, controllerContext.allTopics.toSeq) - controllerContext.allLeaders = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch] + controllerContext.partitionLeadershipInfo = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch] // update the leader and isr cache for all existing partitions from Zookeeper updateLeaderAndIsrCache() // start the channel manager @@ -458,8 +485,6 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("Currently active brokers in the cluster: %s".format(controllerContext.liveBrokerIds)) info("Currently shutting brokers in the cluster: %s".format(controllerContext.shuttingDownBrokerIds)) info("Current list of topics in the cluster: %s".format(controllerContext.allTopics)) - initializeAndMaybeTriggerPartitionReassignment() - initializeAndMaybeTriggerPreferredReplicaElection() } private def initializeAndMaybeTriggerPartitionReassignment() { @@ -482,7 +507,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg val partitionsUndergoingPreferredReplicaElection = ZkUtils.getPartitionsUndergoingPreferredReplicaElection(zkClient) // check if they are already completed val partitionsThatCompletedPreferredReplicaElection = partitionsUndergoingPreferredReplicaElection.filter(partition => - controllerContext.allLeaders(partition).leaderAndIsr.leader == controllerContext.partitionReplicaAssignment(partition).head) + controllerContext.partitionLeadershipInfo(partition).leaderAndIsr.leader == controllerContext.partitionReplicaAssignment(partition).head) controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitionsUndergoingPreferredReplicaElection controllerContext.partitionsUndergoingPreferredReplicaElection --= partitionsThatCompletedPreferredReplicaElection info("Partitions undergoing preferred replica election: %s".format(partitionsUndergoingPreferredReplicaElection.mkString(","))) @@ -498,16 +523,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private def updateLeaderAndIsrCache() { val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, controllerContext.allTopics.toSeq) - for((topicPartition, leaderIsrAndControllerEpoch) <- leaderAndIsrInfo) { - // If the leader specified in the leaderAndIsr is no longer alive, there is no need to recover it - controllerContext.liveBrokerIds.contains(leaderIsrAndControllerEpoch.leaderAndIsr.leader) match { - case true => - controllerContext.allLeaders.put(topicPartition, leaderIsrAndControllerEpoch) - case false => - debug("While refreshing controller's leader and isr cache, leader %d for ".format(leaderIsrAndControllerEpoch.leaderAndIsr.leader) + - "partition %s is dead, just ignore it".format(topicPartition)) - } - } + for((topicPartition, leaderIsrAndControllerEpoch) <- leaderAndIsrInfo) + controllerContext.partitionLeadershipInfo.put(topicPartition, leaderIsrAndControllerEpoch) } private def areReplicasInIsr(topic: String, partition: Int, replicas: Seq[Int]): Boolean = { @@ -522,7 +539,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private def moveReassignedPartitionLeaderIfRequired(topicAndPartition: TopicAndPartition, reassignedPartitionContext: ReassignedPartitionsContext) { val reassignedReplicas = reassignedPartitionContext.newReplicas - val currentLeader = controllerContext.allLeaders(topicAndPartition).leaderAndIsr.leader + val currentLeader = controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader if(!reassignedPartitionContext.newReplicas.contains(currentLeader)) { info("Leader %s for partition %s being reassigned, ".format(currentLeader, topicAndPartition) + "is not in the new list of replicas %s. Re-electing leader".format(reassignedReplicas.mkString(","))) @@ -626,7 +643,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicAndPartition]) { for(partition <- partitionsToBeRemoved) { // check the status - val currentLeader = controllerContext.allLeaders(partition).leaderAndIsr.leader + val currentLeader = controllerContext.partitionLeadershipInfo(partition).leaderAndIsr.leader val preferredReplica = controllerContext.partitionReplicaAssignment(partition).head if(currentLeader == preferredReplica) { info("Partition %s completed preferred replica leader election. New leader is %d".format(partition, preferredReplica)) @@ -894,20 +911,15 @@ class PreferredReplicaElectionListener(controller: KafkaController) extends IZkD */ @throws(classOf[Exception]) def handleDataChange(dataPath: String, data: Object) { - debug("Preferred replica election listener fired for path %s. Record partitions to undergo preferred replica election" + - " %s".format(dataPath, data.toString)) - val partitionsForPreferredReplicaElection = - PreferredReplicaLeaderElectionCommand.parsePreferredReplicaJsonData(data.toString) - val newPartitions = partitionsForPreferredReplicaElection -- controllerContext.partitionsUndergoingPreferredReplicaElection + debug("Preferred replica election listener fired for path %s. Record partitions to undergo preferred replica election %s" + .format(dataPath, data.toString)) + val partitionsForPreferredReplicaElection = PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(data.toString) + controllerContext.controllerLock synchronized { - try { - controller.onPreferredReplicaElection(newPartitions) - } catch { - case e => error("Error completing preferred replica leader election for partitions %s" - .format(partitionsForPreferredReplicaElection.mkString(",")), e) - } finally { - controller.removePartitionsFromPreferredReplicaElection(newPartitions) - } + info("These partitions are already undergoing preferred replica election: %s" + .format(controllerContext.partitionsUndergoingPreferredReplicaElection.mkString(","))) + val newPartitions = partitionsForPreferredReplicaElection -- controllerContext.partitionsUndergoingPreferredReplicaElection + controller.onPreferredReplicaElection(newPartitions) } } @@ -962,10 +974,18 @@ case class ReassignedPartitionsContext(var newReplicas: Seq[Int] = Seq.empty, case class PartitionAndReplica(topic: String, partition: Int, replica: Int) -case class LeaderIsrAndControllerEpoch(val leaderAndIsr: LeaderAndIsr, controllerEpoch: Int) +case class LeaderIsrAndControllerEpoch(val leaderAndIsr: LeaderAndIsr, controllerEpoch: Int) { + override def toString(): String = { + val leaderAndIsrInfo = new StringBuilder + leaderAndIsrInfo.append("(Leader:" + leaderAndIsr.leader) + leaderAndIsrInfo.append(",ISR:" + leaderAndIsr.isr.mkString(",")) + leaderAndIsrInfo.append(",LeaderEpoch:" + leaderAndIsr.leaderEpoch) + leaderAndIsrInfo.append(",ControllerEpoch:" + controllerEpoch + ")") + leaderAndIsrInfo.toString() + } +} object ControllerStats extends KafkaMetricsGroup { - val offlinePartitionRate = newMeter("OfflinePartitionsPerSec", "partitions", TimeUnit.SECONDS) - val uncleanLeaderElectionRate = newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS) + val uncleanLeaderElectionRate = newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS) val leaderElectionTimer = new KafkaTimer(newTimer("LeaderElectionRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) } diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index 3ed9b7e..21b0e24 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -18,14 +18,14 @@ package kafka.controller import kafka.api.LeaderAndIsr import kafka.utils.Logging -import kafka.common.{TopicAndPartition, StateChangeFailedException, PartitionOfflineException} +import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException} trait PartitionLeaderSelector { /** * @param topicAndPartition The topic and partition whose leader needs to be elected * @param currentLeaderAndIsr The current leader and isr of input partition read from zookeeper - * @throws PartitionOfflineException If no replica in the assigned replicas list is alive + * @throws NoReplicaOnlineException If no replica in the assigned replicas list is alive * @return The leader and isr request, with the newly selected leader info, to send to the brokers * Also, returns the list of replicas the returned leader and isr request should be sent to * This API selects a new leader for the input partition @@ -38,7 +38,7 @@ trait PartitionLeaderSelector { * This API selects a new leader for the input partition - * 1. If at least one broker from the isr is alive, it picks a broker from the isr as the new leader * 2. Else, it picks some alive broker from the assigned replica list as the new leader - * 3. If no broker in the assigned replica list is alive, it throws PartitionOfflineException + * 3. If no broker in the assigned replica list is alive, it throws NoReplicaOnlineException * Once the leader is successfully registered in zookeeper, it updates the allLeaders cache */ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector with Logging { @@ -53,53 +53,53 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten val currentLeaderIsrZkPathVersion = currentLeaderAndIsr.zkVersion val newLeaderAndIsr = liveBrokersInIsr.isEmpty match { case true => - debug("No broker is ISR is alive, picking the leader from the alive assigned replicas: %s" - .format(liveAssignedReplicasToThisPartition.mkString(","))) + debug("No broker in ISR is alive for %s. Pick the leader from the alive assigned replicas: %s" + .format(topicAndPartition, liveAssignedReplicasToThisPartition.mkString(","))) liveAssignedReplicasToThisPartition.isEmpty match { case true => - ControllerStats.offlinePartitionRate.mark() - throw new PartitionOfflineException(("No replica for partition " + + throw new NoReplicaOnlineException(("No replica for partition " + "%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) + " Assigned replicas are: [%s]".format(assignedReplicas)) case false => ControllerStats.uncleanLeaderElectionRate.mark() val newLeader = liveAssignedReplicasToThisPartition.head - warn("No broker in ISR is alive, elected leader from the alive replicas is [%s], ".format(newLeader) + - "There's potential data loss") + warn("No broker in ISR is alive for %s. Elect leader from broker %s. There's potential data loss." + .format(topicAndPartition, newLeader)) new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, List(newLeader), currentLeaderIsrZkPathVersion + 1) } case false => val newLeader = liveBrokersInIsr.head - debug("Some broker in ISR is alive, selecting the leader from the ISR: " + newLeader) + debug("Some broker in ISR is alive for %s. Select %d from ISR to be the leader.".format(topicAndPartition, newLeader)) new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, liveBrokersInIsr.toList, currentLeaderIsrZkPathVersion + 1) } info("Selected new leader and ISR %s for offline partition %s".format(newLeaderAndIsr.toString(), topicAndPartition)) (newLeaderAndIsr, liveAssignedReplicasToThisPartition) case None => - ControllerStats.offlinePartitionRate.mark() - throw new PartitionOfflineException("Partition %s doesn't have".format(topicAndPartition) + "replicas assigned to it") + throw new NoReplicaOnlineException("Partition %s doesn't have".format(topicAndPartition) + "replicas assigned to it") } } } /** - * Picks one of the alive in-sync reassigned replicas as the new leader + * Picks one of the alive in-sync reassigned replicas as the new leader. */ class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector with Logging { this.logIdent = "[ReassignedPartitionLeaderSelector]: " + /** + * The reassigned replicas are already in the ISR when selectLeader is called. + */ def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { - val reassignedReplicas = controllerContext.partitionsBeingReassigned(topicAndPartition).newReplicas + val reassignedInSyncReplicas = controllerContext.partitionsBeingReassigned(topicAndPartition).newReplicas val currentLeaderEpoch = currentLeaderAndIsr.leaderEpoch val currentLeaderIsrZkPathVersion = currentLeaderAndIsr.zkVersion - // pick any replica from the newly assigned replicas list that is in the ISR - val aliveReassignedReplicas = reassignedReplicas.filter(r => controllerContext.liveBrokerIds.contains(r)) - val newLeaderOpt = aliveReassignedReplicas.headOption + val aliveReassignedInSyncReplicas = reassignedInSyncReplicas.filter(r => controllerContext.liveBrokerIds.contains(r)) + val newLeaderOpt = aliveReassignedInSyncReplicas.headOption newLeaderOpt match { case Some(newLeader) => (new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, currentLeaderAndIsr.isr, - currentLeaderIsrZkPathVersion + 1), reassignedReplicas) + currentLeaderIsrZkPathVersion + 1), reassignedInSyncReplicas) case None => - reassignedReplicas.size match { + reassignedInSyncReplicas.size match { case 0 => throw new StateChangeFailedException("List of reassigned replicas for partition " + " %s is empty. Current leader and ISR: [%s]".format(topicAndPartition, currentLeaderAndIsr)) @@ -124,10 +124,10 @@ with Logging { val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) val preferredReplica = assignedReplicas.head // check if preferred replica is the current leader - val currentLeader = controllerContext.allLeaders(topicAndPartition).leaderAndIsr.leader - if(currentLeader == preferredReplica) { - throw new StateChangeFailedException("Preferred replica %d is already the current leader for partition %s" - .format(preferredReplica, topicAndPartition)) + val currentLeader = controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader + if (currentLeader == preferredReplica) { + throw new LeaderElectionNotNeededException("Preferred replica %d is already the current leader for partition %s" + .format(preferredReplica, topicAndPartition)) } else { info("Current leader %d for partition %s is not the preferred replica.".format(currentLeader, topicAndPartition) + " Trigerring preferred replica leader election") @@ -177,6 +177,18 @@ class ControlledShutdownLeaderSelector(controllerContext: ControllerContext) " shutting down brokers %s").format(currentLeaderAndIsr.isr.mkString(","), topicAndPartition, currentLeader, controllerContext.shuttingDownBrokerIds.mkString(","))) } } - } +/** + * Essentially does nothing. Returns the current leader and ISR, and the current + * set of replicas assigned to a given topic/partition. + */ +class NoOpLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector with Logging { + + this.logIdent = "[NoOpLeaderSelector]: " + + def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { + warn("I should never have been asked to perform leader election, returning the current LeaderAndIsr and replica assignment.") + (currentLeaderAndIsr, controllerContext.partitionReplicaAssignment(topicAndPartition)) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index b25e9f4..c017727 100644 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -20,7 +20,7 @@ import collection._ import collection.JavaConversions._ import java.util.concurrent.atomic.AtomicBoolean import kafka.api.LeaderAndIsr -import kafka.common.{TopicAndPartition, StateChangeFailedException, PartitionOfflineException} +import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException} import kafka.utils.{Logging, ZkUtils} import org.I0Itec.zkclient.IZkChildListener import org.I0Itec.zkclient.exception.ZkNodeExistsException @@ -43,9 +43,10 @@ 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) - val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext) - private val isShuttingDown = new AtomicBoolean(false) + val brokerRequestBatch = new ControllerBrokerRequestBatch(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) @@ -55,9 +56,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { * the OnlinePartition state change for all new or offline partitions. */ def startup() { - isShuttingDown.set(false) // initialize partition state initializePartitionState() + hasStarted.set(true) // try to move partitions to online state triggerOnlinePartitionStateChange() info("Started partition state machine with initial state -> " + partitionState.toString()) @@ -72,7 +73,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { * Invoked on controller shutdown. */ def shutdown() { - isShuttingDown.compareAndSet(false, true) + hasShutdown.compareAndSet(false, true) partitionState.clear() } @@ -86,7 +87,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { // try to move all partitions in NewPartition or OfflinePartition state to OnlinePartition state for((topicAndPartition, partitionState) <- partitionState) { if(partitionState.equals(OfflinePartition) || partitionState.equals(NewPartition)) - handleStateChange(topicAndPartition.topic, topicAndPartition.partition, OnlinePartition, offlinePartitionSelector) + handleStateChange(topicAndPartition.topic, topicAndPartition.partition, OnlinePartition, controller.offlinePartitionSelector) } brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) } catch { @@ -101,7 +102,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { * @param targetState The state that the partitions should be moved to */ def handleStateChanges(partitions: Set[TopicAndPartition], targetState: PartitionState, - leaderSelector: PartitionLeaderSelector = offlinePartitionSelector) { + leaderSelector: PartitionLeaderSelector = noOpPartitionLeaderSelector) { info("Invoking state change to %s for partitions %s".format(targetState, partitions.mkString(","))) try { brokerRequestBatch.newBatch() @@ -111,6 +112,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) }catch { case e => error("Error while moving some partitions to %s state".format(targetState), e) + // TODO: It is not enough to bail out and log an error, it is important to trigger state changes for those partitions } } @@ -124,6 +126,10 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { private def handleStateChange(topic: String, partition: Int, targetState: PartitionState, leaderSelector: PartitionLeaderSelector) { val topicAndPartition = TopicAndPartition(topic, partition) + if (!hasStarted.get) + throw new StateChangeFailedException(("Controller %d epoch %d initiated state change for partition %s to %s failed because " + + "the partition state machine has not started") + .format(controllerId, controller.epoch, topicAndPartition, targetState)) val currState = partitionState.getOrElseUpdate(topicAndPartition, NonExistentPartition) try { targetState match { @@ -149,7 +155,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { case _ => // should never come here since illegal previous states are checked above } partitionState.put(topicAndPartition, OnlinePartition) - val leader = controllerContext.allLeaders(topicAndPartition).leaderAndIsr.leader + val leader = controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader stateChangeLogger.trace("Controller %d epoch %d changed partition %s from %s to OnlinePartition with leader %d" .format(controllerId, controller.epoch, topicAndPartition, partitionState(topicAndPartition), leader)) // post: partition has a leader @@ -172,7 +178,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } catch { case t: Throwable => stateChangeLogger.error("Controller %d epoch %d initiated state change for partition %s from %s to %s failed" - .format(controllerId, controller.epoch, topicAndPartition, currState, targetState), t) + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState), t) } } @@ -232,7 +238,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { val liveAssignedReplicas = replicaAssignment.filter(r => controllerContext.liveBrokerIds.contains(r)) liveAssignedReplicas.size match { case 0 => - ControllerStats.offlinePartitionRate.mark() val failMsg = ("encountered error during state change of partition %s from New to Online, assigned replicas are [%s], " + "live brokers are [%s]. No assigned replica is alive.") .format(topicAndPartition, replicaAssignment.mkString(","), controllerContext.liveBrokerIds) @@ -253,14 +258,12 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { // GC pause brokerRequestBatch.addLeaderAndIsrRequestForBrokers(liveAssignedReplicas, topicAndPartition.topic, topicAndPartition.partition, leaderIsrAndControllerEpoch, replicaAssignment.size) - controllerContext.allLeaders.put(topicAndPartition, leaderIsrAndControllerEpoch) - partitionState.put(topicAndPartition, OnlinePartition) + controllerContext.partitionLeadershipInfo.put(topicAndPartition, leaderIsrAndControllerEpoch) } catch { case e: ZkNodeExistsException => // read the controller epoch val leaderIsrAndEpoch = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition).get - ControllerStats.offlinePartitionRate.mark() val failMsg = ("encountered error while changing partition %s's state from New to Online since LeaderAndIsr path already " + "exists with value %s and controller epoch %d") .format(topicAndPartition, leaderIsrAndEpoch.leaderAndIsr.toString(), leaderIsrAndEpoch.controllerEpoch) @@ -310,22 +313,21 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } val newLeaderIsrAndControllerEpoch = new LeaderIsrAndControllerEpoch(newLeaderAndIsr, controller.epoch) // update the leader cache - controllerContext.allLeaders.put(TopicAndPartition(topic, partition), newLeaderIsrAndControllerEpoch) + 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)) // store new leader and isr info in cache brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasForThisPartition, topic, partition, newLeaderIsrAndControllerEpoch, controllerContext.partitionReplicaAssignment(TopicAndPartition(topic, partition)).size) } catch { - case poe: PartitionOfflineException => throw new PartitionOfflineException("All replicas %s for partition %s are dead." - .format(controllerContext.partitionReplicaAssignment(topicAndPartition).mkString(","), topicAndPartition) + - " Marking this partition offline", poe) + case lenne: LeaderElectionNotNeededException => // swallow + case nroe: NoReplicaOnlineException => throw nroe case sce => val failMsg = "encountered error while electing leader for partition %s due to: %s.".format(topicAndPartition, sce.getMessage) stateChangeLogger.error("Controller %d epoch %d ".format(controllerId, controller.epoch) + failMsg) throw new StateChangeFailedException(failMsg, sce) } - debug("After leader election, leader cache is updated to %s".format(controllerContext.allLeaders.map(l => (l._1, l._2)))) + debug("After leader election, leader cache is updated to %s".format(controllerContext.partitionLeadershipInfo.map(l => (l._1, l._2)))) } private def registerTopicChangeListener() = { @@ -355,8 +357,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { @throws(classOf[Exception]) def handleChildChange(parentPath : String, children : java.util.List[String]) { - if(!isShuttingDown.get()) { - controllerContext.controllerLock synchronized { + controllerContext.controllerLock synchronized { + if (!hasShutdown.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 88058ec..bea1644 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -42,8 +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, controller.config.brokerId) - private val isShuttingDown = new AtomicBoolean(false) + val brokerRequestBatch = new ControllerBrokerRequestBatch(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) @@ -53,9 +54,9 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * Then triggers the OnlineReplica state change for all replicas. */ def startup() { - isShuttingDown.set(false) // initialize replica state initializeReplicaState() + hasStarted.set(true) // move all Online replicas to Online handleStateChanges(ZkUtils.getAllReplicasOnBroker(zkClient, controllerContext.allTopics.toSeq, controllerContext.liveBrokerIds.toSeq), OnlineReplica) @@ -71,7 +72,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * Invoked on controller shutdown. */ def shutdown() { - isShuttingDown.compareAndSet(false, true) + hasShutdown.compareAndSet(false, true) replicaState.clear() } @@ -102,6 +103,10 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { */ def handleStateChange(topic: String, partition: Int, replicaId: Int, targetState: ReplicaState) { val topicAndPartition = TopicAndPartition(topic, partition) + if (!hasStarted.get) + throw new StateChangeFailedException(("Controller %d epoch %d initiated state change of replica %d for partition %s " + + "to %s failed because replica state machine has not started") + .format(controllerId, controller.epoch, replicaId, topicAndPartition, targetState)) try { replicaState.getOrElseUpdate((topic, partition, replicaId), NonExistentReplica) val replicaAssignment = controllerContext.partitionReplicaAssignment(topicAndPartition) @@ -138,32 +143,30 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case NewReplica => // add this replica to the assigned replicas list for its partition val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) - controllerContext.partitionReplicaAssignment.put(topicAndPartition, currentAssignedReplicas :+ replicaId) + if(!currentAssignedReplicas.contains(replicaId)) + controllerContext.partitionReplicaAssignment.put(topicAndPartition, currentAssignedReplicas :+ replicaId) stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica" .format(controllerId, controller.epoch, replicaId, topicAndPartition)) case _ => - // check if the leader for this partition is alive or even exists - controllerContext.allLeaders.get(topicAndPartition) match { + // check if the leader for this partition ever existed + controllerContext.partitionLeadershipInfo.get(topicAndPartition) match { case Some(leaderIsrAndControllerEpoch) => - controllerContext.liveBrokerIds.contains(leaderIsrAndControllerEpoch.leaderAndIsr.leader) match { - case true => // leader is alive - brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId), - topic, partition, leaderIsrAndControllerEpoch, - replicaAssignment.size) - 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)) - case false => // ignore partitions whose leader is not alive - } - case None => // ignore partitions who don't have a leader yet + brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId), topic, partition, leaderIsrAndControllerEpoch, + replicaAssignment.size) + 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)) + case None => // that means the partition was never in OnlinePartition state, this means the broker never + // started a log for that partition and does not have a high watermark value for this partition } + } replicaState.put((topic, partition, replicaId), OnlineReplica) case OfflineReplica => assertValidPreviousStates(topic, partition, replicaId, List(NewReplica, OnlineReplica), targetState) // As an optimization, the controller removes dead replicas from the ISR val leaderAndIsrIsEmpty: Boolean = - controllerContext.allLeaders.get(topicAndPartition) match { + controllerContext.partitionLeadershipInfo.get(topicAndPartition) match { case Some(currLeaderIsrAndControllerEpoch) => if (currLeaderIsrAndControllerEpoch.leaderAndIsr.isr.contains(replicaId)) controller.removeReplicaFromIsr(topic, partition, replicaId) match { @@ -237,8 +240,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { 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(","))) - if(!isShuttingDown.get()) { - controllerContext.controllerLock synchronized { + controllerContext.controllerLock synchronized { + if (!hasShutdown.get) { try { val curBrokerIds = currentBrokerList.map(_.toInt).toSet val newBrokerIds = curBrokerIds -- controllerContext.liveOrShuttingDownBrokerIds diff --git a/core/src/main/scala/kafka/javaapi/producer/Producer.scala b/core/src/main/scala/kafka/javaapi/producer/Producer.scala index 424ef39..7265328 100644 --- a/core/src/main/scala/kafka/javaapi/producer/Producer.scala +++ b/core/src/main/scala/kafka/javaapi/producer/Producer.scala @@ -26,7 +26,7 @@ class Producer[K,V](private val underlying: kafka.producer.Producer[K,V]) // for /** * Sends the data to a single topic, partitioned by key, using either the * synchronous or the asynchronous producer - * @param producerData the producer data object that encapsulates the topic, key and message data + * @param message the producer data object that encapsulates the topic, key and message data */ def send(message: KeyedMessage[K,V]) { underlying.send(message) @@ -34,7 +34,7 @@ class Producer[K,V](private val underlying: kafka.producer.Producer[K,V]) // for /** * Use this API to send data to multiple topics - * @param producerData list of producer data objects that encapsulate the topic, key and message data + * @param messages list of producer data objects that encapsulate the topic, key and message data */ def send(messages: java.util.List[KeyedMessage[K,V]]) { import collection.JavaConversions._ diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index abb160c..1afb533 100644 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -53,7 +53,7 @@ class FileMessageSet private[kafka](@volatile var file: File, /* if this is not a slice, update the file pointer to the end of the file */ if (!isSlice) { - info("Creating or reloading log segment %s".format(file.getAbsolutePath)) + debug("Creating or reloading log segment %s".format(file.getAbsolutePath)) /* set the file position to the last byte in the file */ channel.position(channel.size) } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index d1c3d72..8c8d877 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -74,13 +74,13 @@ class Log(val dir: File, /* Calculate the offset of the next message */ private val nextOffset: AtomicLong = new AtomicLong(activeSegment.nextOffset()) - debug("Completed load of log %s with log end offset %d".format(name, logEndOffset)) + info("Completed load of log %s with log end offset %d".format(name, logEndOffset)) newGauge(name + "-" + "NumLogSegments", - new Gauge[Int] { def getValue = numberOfSegments }) + new Gauge[Int] { def value = numberOfSegments }) newGauge(name + "-" + "LogEndOffset", - new Gauge[Long] { def getValue = logEndOffset }) + new Gauge[Long] { def value = logEndOffset }) /** The name of this log */ def name = dir.getName() diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 0d567e4..9002483 100644 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -204,7 +204,7 @@ class LogManager(val logDirs: Array[File], scheduler, time) logs.put(topicAndPartition, log) - info("Created log for topic %s partition %d in %s with properties {%s}." + info("Created log for partition [%s,%d] in %s with properties {%s}." .format(topicAndPartition.topic, topicAndPartition.partition, dataDir.getAbsolutePath, diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 30d2e91..fbdc553 100644 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -147,7 +147,15 @@ class LogSegment(val log: FileMessageSet, val entry = iter.next entry.message.ensureValid() if(validBytes - lastIndexEntry > indexIntervalBytes) { - index.append(entry.offset, validBytes) + // we need to decompress the message, if required, to get the offset of the first uncompressed message + val startOffset = + entry.message.compressionCodec match { + case NoCompressionCodec => + entry.offset + case _ => + ByteBufferMessageSet.decompress(entry.message).head.offset + } + index.append(startOffset, validBytes) lastIndexEntry = validBytes } validBytes += MessageSet.entrySize(entry.message) diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index eff213e..361a9db 100644 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -90,7 +90,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi /* the last offset in the index */ var lastOffset = readLastOffset() - info("Loaded index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d, file position = %d" + debug("Loaded index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d, file position = %d" .format(file.getAbsolutePath, maxEntries, maxIndexSize, entries(), lastOffset, mmap.position)) /** diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index caf253f..73401c5 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -154,7 +154,7 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message return allDone() val offset = topIter.getLong() val size = topIter.getInt() - if(size < 0 || size < Message.MinHeaderSize) + if(size < Message.MinHeaderSize) throw new InvalidMessageException("Message found with corrupt size (" + size + ")") // we have an incomplete message diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 209fdfa..7b8d1f0 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -26,6 +26,7 @@ import kafka.common.TopicAndPartition import kafka.utils.{Logging, SystemTime} import kafka.message.ByteBufferMessageSet import java.net._ +import org.apache.log4j.Logger object RequestChannel extends Logging { @@ -47,6 +48,7 @@ object RequestChannel extends Logging { val requestId = buffer.getShort() val requestObj: RequestOrResponse = RequestKeys.deserializerForKey(requestId)(buffer) buffer = null + private val requestLogger = Logger.getLogger("kafka.request.logger") trace("Received request : %s".format(requestObj)) def updateRequestMetrics() { @@ -76,8 +78,9 @@ object RequestChannel extends Logging { m.responseSendTimeHist.update(responseSendTime) m.totalTimeHist.update(totalTime) } - trace("Completed request : %s, totalTime:%d, queueTime:%d, localTime:%d, remoteTime:%d, sendTime:%d" - .format(requestObj, totalTime, queueTime, apiLocalTime, apiRemoteTime, responseSendTime)) + if(requestLogger.isTraceEnabled) + requestLogger.trace("Completed request:%s from client %s;totalTime:%d,queueTime:%d,localTime:%d,remoteTime:%d,sendTime:%d" + .format(requestObj, remoteAddress, totalTime, queueTime, apiLocalTime, apiRemoteTime, responseSendTime)) } } @@ -99,10 +102,19 @@ class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMe newGauge( "RequestQueueSize", new Gauge[Int] { - def getValue = requestQueue.size + def value = requestQueue.size } ) + for(i <- 0 until numProcessors) { + newGauge( + "Processor-" + i + "-ResponseQueueSize", + new Gauge[Int] { + def value = responseQueues(i).size() + } + ) + } + /** Send a request to be handled, potentially blocking until there is room in the queue for the request */ def sendRequest(request: RequestChannel.Request) { requestQueue.put(request) diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 865f7b4..134e60c 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -37,6 +37,8 @@ class SocketServer(val brokerId: Int, val port: Int, val numProcessorThreads: Int, val maxQueuedRequests: Int, + val sendBufferSize: Int, + val recvBufferSize: Int, val maxRequestSize: Int = Int.MaxValue) extends Logging { this.logIdent = "[Socket Server on Broker " + brokerId + "], " private val time = SystemTime @@ -56,7 +58,7 @@ class SocketServer(val brokerId: Int, requestChannel.addResponseListener((id:Int) => processors(id).wakeup()) // start accepting connections - this.acceptor = new Acceptor(host, port, processors) + this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize) Utils.newThread("kafka-acceptor", acceptor, false).start() acceptor.awaitStartup info("started") @@ -128,7 +130,8 @@ private[kafka] abstract class AbstractServerThread extends Runnable with Logging /** * Thread that accepts and configures new connections. There is only need for one of these */ -private[kafka] class Acceptor(val host: String, val port: Int, private val processors: Array[Processor]) extends AbstractServerThread { +private[kafka] class Acceptor(val host: String, val port: Int, private val processors: Array[Processor], + val sendBufferSize: Int, val recvBufferSize: Int) extends AbstractServerThread { val serverChannel = openServerSocket(host, port) /** @@ -192,10 +195,19 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce * Accept a new connection */ def accept(key: SelectionKey, processor: Processor) { - val socketChannel = key.channel().asInstanceOf[ServerSocketChannel].accept() - debug("Accepted connection from " + socketChannel.socket.getInetAddress() + " on " + socketChannel.socket.getLocalSocketAddress) + val serverSocketChannel = key.channel().asInstanceOf[ServerSocketChannel] + serverSocketChannel.socket().setReceiveBufferSize(recvBufferSize) + + val socketChannel = serverSocketChannel.accept() socketChannel.configureBlocking(false) socketChannel.socket().setTcpNoDelay(true) + socketChannel.socket().setSendBufferSize(sendBufferSize) + + debug("Accepted connection from %s on %s. sendBufferSize [actual|requested]: [%d|%d] recvBufferSize [actual|requested]: [%d|%d]" + .format(socketChannel.socket.getInetAddress, socketChannel.socket.getLocalSocketAddress, + socketChannel.socket.getSendBufferSize, sendBufferSize, + socketChannel.socket.getReceiveBufferSize, recvBufferSize)) + processor.accept(socketChannel) } diff --git a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala index 72597ef..82e6e4d 100644 --- a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala +++ b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala @@ -57,10 +57,10 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig, partitionMetadata.map { m => m.leader match { case Some(leader) => - debug("Topic %s partition %d has leader %d".format(topic, m.partitionId, leader.id)) + debug("Partition [%s,%d] has leader %d".format(topic, m.partitionId, leader.id)) new PartitionAndLeader(topic, m.partitionId, Some(leader.id)) case None => - debug("Topic %s partition %d does not have a leader yet".format(topic, m.partitionId)) + debug("Partition [%s,%d] does not have a leader yet".format(topic, m.partitionId)) new PartitionAndLeader(topic, m.partitionId, None) } }.sortWith((s, t) => s.partitionId < t.partitionId) diff --git a/core/src/main/scala/kafka/producer/ConsoleProducer.scala b/core/src/main/scala/kafka/producer/ConsoleProducer.scala index eebfda6..5539bce 100644 --- a/core/src/main/scala/kafka/producer/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/producer/ConsoleProducer.scala @@ -126,7 +126,7 @@ object ConsoleProducer { cmdLineProps.put("topic", topic) val props = new Properties() - props.put("broker.list", brokerList) + props.put("metadata.broker.list", brokerList) val codec = if(compress) DefaultCompressionCodec.codec else NoCompressionCodec.codec props.put("compression.codec", codec.toString) props.put("producer.type", if(sync) "sync" else "async") @@ -196,7 +196,7 @@ object ConsoleProducer { topic = props.getProperty("topic") if(props.containsKey("parse.key")) parseKey = props.getProperty("parse.key").trim.toLowerCase.equals("true") - if(props.containsKey("key.seperator")) + if(props.containsKey("key.separator")) keySeparator = props.getProperty("key.separator") if(props.containsKey("ignore.error")) ignoreError = props.getProperty("ignore.error").trim.toLowerCase.equals("true") diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index 3d22e6d..88ae784 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -64,9 +64,9 @@ class KafkaLog4jAppender extends AppenderSkeleton with Logging { // check for config parameter validity val props = new Properties() if(brokerList != null) - props.put("broker.list", brokerList) + props.put("metadata.broker.list", brokerList) if(props.isEmpty) - throw new MissingConfigException("The broker.list property should be specified") + throw new MissingConfigException("The metadata.broker.list property should be specified") if(topic == null) throw new MissingConfigException("topic must be specified by the Kafka log4j appender") if(serializerClass == null) { diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index e27ec44..7947b18 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -63,7 +63,7 @@ class ProducerConfig private (val props: VerifiableProperties) * format is host1:port1,host2:port2, and the list can be a subset of brokers or * a VIP pointing to a subset of brokers. */ - val brokerList = props.getString("broker.list") + val brokerList = props.getString("metadata.broker.list") /** the partitioner class for partitioning events amongst sub-topics */ val partitionerClass = props.getString("partitioner.class", "kafka.producer.DefaultPartitioner") diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index 2e3e383..698ad1c 100644 --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -135,7 +135,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, } else { // currently, if in async mode, we just log the serialization error. We need to revisit // this when doing kafka-496 - error("Error serializing message ", t) + error("Error serializing message for topic %s".format(e.topic), t) } } } diff --git a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala index 6691147..090400d 100644 --- a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala +++ b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala @@ -36,7 +36,7 @@ class ProducerSendThread[K,V](val threadName: String, newGauge(clientId + "-ProducerQueueSize", new Gauge[Int] { - def getValue = queue.size + def value = queue.size }) override def run { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index e8702e2..4269219 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -46,13 +46,13 @@ abstract class AbstractFetcherManager(protected val name: String, numFetchers: I fetcherThread.start } fetcherThread.addPartition(topic, partitionId, initialOffset) - info("adding fetcher on topic %s, partion %d, initOffset %d to broker %d with fetcherId %d" + info("Adding fetcher for partition [%s,%d], initOffset %d to broker %d with fetcherId %d" .format(topic, partitionId, initialOffset, sourceBroker.id, key.fetcherId)) } } def removeFetcher(topic: String, partitionId: Int) { - info("removing fetcher on topic %s, partition %d".format(topic, partitionId)) + info("Removing fetcher for partition [%s,%d]".format(topic, partitionId)) mapLock synchronized { for ((key, fetcher) <- fetcherThreadMap) { fetcher.removePartition(topic, partitionId) diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 4ee23cd..162c749 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -18,10 +18,8 @@ package kafka.server import kafka.cluster.Broker -import kafka.common.{ClientIdAndBroker, TopicAndPartition, ErrorMapping} import collection.mutable -import kafka.message.ByteBufferMessageSet -import kafka.message.MessageAndOffset +import kafka.message.{InvalidMessageException, ByteBufferMessageSet, MessageAndOffset} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge import java.util.concurrent.atomic.AtomicLong @@ -30,6 +28,7 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import kafka.consumer.{PartitionTopicInfo, SimpleConsumer} import kafka.api.{FetchRequest, FetchResponse, FetchResponsePartitionData, FetchRequestBuilder} +import kafka.common.{KafkaException, ClientIdAndBroker, TopicAndPartition, ErrorMapping} /** @@ -97,7 +96,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke response = simpleConsumer.fetch(fetchRequest) } catch { case t => - debug("error in fetch %s".format(fetchRequest), t) + warn("Error in fetch %s".format(fetchRequest), t) if (isRunning.get) { partitionMapLock synchronized { partitionsWithError ++= partitionMap.keys @@ -118,30 +117,42 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke if (currentOffset.isDefined && fetchRequest.requestInfo(topicAndPartition).offset == currentOffset.get) { partitionData.error match { case ErrorMapping.NoError => - val messages = partitionData.messages.asInstanceOf[ByteBufferMessageSet] - val validBytes = messages.validBytes - val newOffset = messages.lastOption match { - case Some(m: MessageAndOffset) => m.nextOffset - case None => currentOffset.get + try { + val messages = partitionData.messages.asInstanceOf[ByteBufferMessageSet] + val validBytes = messages.validBytes + val newOffset = messages.shallowIterator.toSeq.lastOption match { + case Some(m: MessageAndOffset) => m.nextOffset + case None => currentOffset.get + } + partitionMap.put(topicAndPartition, newOffset) + fetcherLagStats.getFetcherLagStats(topic, partitionId).lag = partitionData.hw - newOffset + fetcherStats.byteRate.mark(validBytes) + // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread + processPartitionData(topicAndPartition, currentOffset.get, partitionData) + } catch { + case ime: InvalidMessageException => + // we log the error and continue. This ensures two things + // 1. If there is a corrupt message in a topic partition, it does not bring the fetcher thread down and cause other topic partition to also lag + // 2. If the message is corrupt due to a transient state in the log (truncation, partial writes can cause this), we simply continue and + // should get fixed in the subsequent fetches + logger.warn("Found invalid messages during fetch for partition [" + topic + "," + partitionId + "] offset " + currentOffset.get + " error " + ime.getMessage) + case e => + throw new KafkaException("error processing data for partition [%s,%d] offset %d" + .format(topic, partitionId, currentOffset.get), e) } - partitionMap.put(topicAndPartition, newOffset) - fetcherLagStats.getFetcherLagStats(topic, partitionId).lag = partitionData.hw - newOffset - fetcherStats.byteRate.mark(validBytes) - // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread - processPartitionData(topicAndPartition, currentOffset.get, partitionData) case ErrorMapping.OffsetOutOfRangeCode => try { val newOffset = handleOffsetOutOfRange(topicAndPartition) partitionMap.put(topicAndPartition, newOffset) - warn("current offset %d for topic %s partition %d out of range; reset offset to %d" + warn("current offset %d for partition [%s,%d] out of range; reset offset to %d" .format(currentOffset.get, topic, partitionId, newOffset)) } catch { case e => - warn("error getting offset for %s %d to broker %d".format(topic, partitionId, sourceBroker.id), e) + warn("error getting offset for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), e) partitionsWithError += topicAndPartition } case _ => - warn("error for %s %d to broker %d".format(topic, partitionId, sourceBroker.id), + warn("error for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), ErrorMapping.exceptionFor(partitionData.error)) partitionsWithError += topicAndPartition } @@ -159,7 +170,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke } def addPartition(topic: String, partitionId: Int, initialOffset: Long) { - partitionMapLock.lock() + partitionMapLock.lockInterruptibly() try { val topicPartition = TopicAndPartition(topic, partitionId) partitionMap.put( @@ -172,7 +183,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke } def removePartition(topic: String, partitionId: Int) { - partitionMapLock.lock() + partitionMapLock.lockInterruptibly() try { partitionMap.remove(TopicAndPartition(topic, partitionId)) } finally { @@ -180,17 +191,8 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke } } - def hasPartition(topic: String, partitionId: Int): Boolean = { - partitionMapLock.lock() - try { - partitionMap.get(TopicAndPartition(topic, partitionId)).isDefined - } finally { - partitionMapLock.unlock() - } - } - def partitionCount() = { - partitionMapLock.lock() + partitionMapLock.lockInterruptibly() try { partitionMap.size } finally { @@ -204,7 +206,7 @@ class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMet newGauge( metricId + "-ConsumerLag", new Gauge[Long] { - def getValue = lagVal.get + def value = lagVal.get } ) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index d7d8bbd..358d617 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -50,7 +50,6 @@ class KafkaApis(val requestChannel: RequestChannel, new FetchRequestPurgatory(requestChannel, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) private val delayedRequestMetrics = new DelayedRequestMetrics - private val requestLogger = Logger.getLogger("kafka.request.logger") this.logIdent = "[KafkaApi-%d] ".format(brokerId) /** @@ -58,8 +57,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handle(request: RequestChannel.Request) { try{ - if(requestLogger.isTraceEnabled) - requestLogger.trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress) + trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress) request.requestId match { case RequestKeys.ProduceKey => handleProducerRequest(request) case RequestKeys.FetchKey => handleFetchRequest(request) @@ -194,17 +192,20 @@ class KafkaApis(val requestChannel: RequestChannel, BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) try { - val localReplica = replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) - val log = localReplica.log.get - val info = log.append(messages.asInstanceOf[ByteBufferMessageSet], assignOffsets = true) + val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) + val info = + partitionOpt match { + case Some(partition) => partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet]) + case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" + .format(topicAndPartition, brokerId)) + + } val numAppendedMessages = if (info.firstOffset == -1L || info.lastOffset == -1L) 0 else (info.lastOffset - info.firstOffset + 1) // update stats BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) - - // we may need to increment high watermark since ISR could be down to 1 - localReplica.partition.maybeIncrementLeaderHW(localReplica) + trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) ProduceResult(topicAndPartition, info.firstOffset, info.lastOffset) @@ -217,10 +218,10 @@ class KafkaApis(val requestChannel: RequestChannel, Runtime.getRuntime.halt(1) null case utpe: UnknownTopicOrPartitionException => - warn(utpe.getMessage) + warn("Produce request: " + utpe.getMessage) new ProduceResult(topicAndPartition, utpe) case nle: NotLeaderForPartitionException => - warn(nle.getMessage) + warn("Produce request: " + nle.getMessage) new ProduceResult(topicAndPartition, nle) case e => BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() @@ -261,7 +262,8 @@ class KafkaApis(val requestChannel: RequestChannel, val response = new FetchResponse(fetchRequest.correlationId, dataRead) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) } else { - debug("Putting fetch request into purgatory") + debug("Putting fetch request with correlation id %d from client %s into purgatory".format(fetchRequest.correlationId, + fetchRequest.clientId)) // create a list of (topic, partition) pairs to use as keys for this delayed request val delayedFetchKeys = fetchRequest.requestInfo.keys.toSeq.map(new RequestKey(_)) val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest, fetchRequest.maxWait, bytesReadable) @@ -294,7 +296,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (!isFetchFromFollower) { new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, messages) } else { - debug("Leader %d for topic %s partition %d received fetch request from follower %d" + debug("Leader %d for partition [%s,%d] received fetch request from follower %d" .format(brokerId, topic, partition, fetchRequest.replicaId)) new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, messages) } @@ -303,15 +305,17 @@ 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(utpe.getMessage) + warn("Fetch request: " + utpe.getMessage) new FetchResponsePartitionData(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) case nle: NotLeaderForPartitionException => - warn(nle.getMessage) + warn("Fetch request: " + nle.getMessage) new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) case t => BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark() - error("error when processing request " + (topic, partition, offset, fetchSize), t) + error("Error when processing fetch request for partition [%s,%d] offset %d from %s with correlation id %d" + .format(topic, partition, offset, if (isFetchFromFollower) "follower" else "consumer", fetchRequest.correlationId), + t) new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) } (TopicAndPartition(topic, partition), partitionData) @@ -341,7 +345,7 @@ class KafkaApis(val requestChannel: RequestChannel, case Some(log) => log.read(offset, maxSize, maxOffsetOpt) case None => - error("Leader for topic %s partition %d on broker %d does not have a local log".format(topic, partition, brokerId)) + error("Leader for partition [%s,%d] on broker %d does not have a local log".format(topic, partition, brokerId)) MessageSet.Empty } (messages, localReplica.highWatermark) @@ -503,9 +507,6 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetCommitRequest(request: RequestChannel.Request) { val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] - if(requestLogger.isTraceEnabled) - requestLogger.trace("Handling offset commit request " + offsetCommitRequest.toString) - trace("Handling offset commit request " + offsetCommitRequest.toString) val responseInfo = offsetCommitRequest.requestInfo.map{ case (topicAndPartition, metaAndError) => { val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicAndPartition.topic) @@ -533,8 +534,6 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetFetchRequest(request: RequestChannel.Request) { val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] - if(requestLogger.isTraceEnabled) - requestLogger.trace("Handling offset fetch request " + offsetFetchRequest.toString) val responseInfo = offsetFetchRequest.requestInfo.map( t => { val topicDirs = new ZKGroupTopicDirs(offsetFetchRequest.groupId, t.topic) try { diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index e2f4e91..5f3b92c 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -72,6 +72,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.port, config.numNetworkThreads, config.queuedMaxRequests, + config.socketSendBufferBytes, + config.socketReceiveBufferBytes, config.socketRequestMaxBytes) socketServer.startup() @@ -113,8 +115,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg */ private def registerStats() { BrokerTopicStats.getBrokerAllTopicsStats() - ControllerStats.offlinePartitionRate ControllerStats.uncleanLeaderElectionRate + ControllerStats.leaderElectionTimer } /** diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 99e6f4e..018c76f 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -47,14 +47,14 @@ class ReplicaFetcherThread(name:String, if (fetchOffset != replica.logEndOffset) throw new RuntimeException("Offset mismatch: fetched offset = %d, log end offset = %d.".format(fetchOffset, replica.logEndOffset)) - trace("Follower %d has replica log end offset %d. Received %d messages and leader hw %d" - .format(replica.brokerId, replica.logEndOffset, messageSet.sizeInBytes, partitionData.hw)) + trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d" + .format(replica.brokerId, replica.logEndOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.hw)) replica.log.get.append(messageSet, assignOffsets = false) - trace("Follower %d has replica log end offset %d after appending %d bytes of messages" - .format(replica.brokerId, replica.logEndOffset, messageSet.sizeInBytes)) + trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s" + .format(replica.brokerId, replica.logEndOffset, messageSet.sizeInBytes, topicAndPartition)) val followerHighWatermark = replica.logEndOffset.min(partitionData.hw) replica.highWatermark = followerHighWatermark - trace("Follower %d set replica highwatermark for topic %s partition %d to %d" + trace("Follower %d set replica highwatermark for partition [%s,%d] to %d" .format(replica.brokerId, topic, partitionId, followerHighWatermark)) } catch { case e: KafkaStorageException => @@ -83,6 +83,8 @@ class ReplicaFetcherThread(name:String, val leaderEndOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, OffsetRequest.LatestTime, brokerConfig.brokerId) if (leaderEndOffset < log.logEndOffset) { log.truncateTo(leaderEndOffset) + warn("Replica %d for partition %s reset its fetch offset to current leader %d's latest offset %d" + .format(brokerConfig.brokerId, topicAndPartition, sourceBroker.id, leaderEndOffset)) leaderEndOffset } else { /** @@ -93,6 +95,8 @@ class ReplicaFetcherThread(name:String, */ val leaderStartOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, OffsetRequest.EarliestTime, brokerConfig.brokerId) log.truncateFullyAndStartAt(leaderStartOffset) + warn("Replica %d for partition %s reset its fetch offset to current leader %d's start offset %d" + .format(brokerConfig.brokerId, topicAndPartition, sourceBroker.id, leaderStartOffset)) leaderStartOffset } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 477f60e..a7b2146 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -59,7 +59,7 @@ class ReplicaManager(val config: KafkaConfig, newGauge( "LeaderCount", new Gauge[Int] { - def getValue = { + def value = { leaderPartitionsLock synchronized { leaderPartitions.size } @@ -69,13 +69,13 @@ class ReplicaManager(val config: KafkaConfig, newGauge( "PartitionCount", new Gauge[Int] { - def getValue = allPartitions.size + def value = allPartitions.size } ) newGauge( "UnderReplicatedPartitions", new Gauge[Int] { - def getValue = { + def value = { leaderPartitionsLock synchronized { leaderPartitions.count(_.isUnderReplicated) } @@ -122,18 +122,18 @@ class ReplicaManager(val config: KafkaConfig, leaderPartitionsLock synchronized { leaderPartitions -= replica.partition } - allPartitions.remove((topic, partitionId)) - info("After removing partition [%s,%d], the rest of allReplicas is: [%s]".format(topic, partitionId, allPartitions)) + if(deletePartition) + allPartitions.remove((topic, partitionId)) case None => //do nothing if replica no longer exists } - stateChangeLogger.trace("Broker %d finished handling stop replica [%s,%d]".format(localBrokerId, topic, partitionId)) + stateChangeLogger.trace("Broker %d finished handling stop replica for partition [%s,%d]".format(localBrokerId, topic, partitionId)) errorCode } def stopReplicas(stopReplicaRequest: StopReplicaRequest): (mutable.Map[(String, Int), Short], Short) = { val responseMap = new collection.mutable.HashMap[(String, Int), Short] if(stopReplicaRequest.controllerEpoch < controllerEpoch) { - stateChangeLogger.error("Broker %d received stop replica request from an old controller epoch %d." + stateChangeLogger.warn("Broker %d received stop replica request from an old controller epoch %d." .format(localBrokerId, stopReplicaRequest.controllerEpoch) + " Latest known controller epoch is %d " + controllerEpoch) (responseMap, ErrorMapping.StaleControllerEpochCode) @@ -170,19 +170,19 @@ class ReplicaManager(val config: KafkaConfig, if(replicaOpt.isDefined) return replicaOpt.get else - throw new ReplicaNotAvailableException("Replica %d is not available for partiton [%s,%d] yet".format(config.brokerId, topic, partition)) + throw new ReplicaNotAvailableException("Replica %d is not available for partition [%s,%d]".format(config.brokerId, topic, partition)) } def getLeaderReplicaIfLocal(topic: String, partitionId: Int): Replica = { val partitionOpt = getPartition(topic, partitionId) partitionOpt match { case None => - throw new UnknownTopicOrPartitionException("Topic %s partition %d doesn't exist on %d".format(topic, partitionId, config.brokerId)) + throw new UnknownTopicOrPartitionException("Partition [%s,%d] doesn't exist on %d".format(topic, partitionId, config.brokerId)) case Some(partition) => partition.leaderReplicaIfLocal match { case Some(leaderReplica) => leaderReplica case None => - throw new NotLeaderForPartitionException("Leader not local for topic %s partition %d on broker %d" + throw new NotLeaderForPartitionException("Leader not local for partition [%s,%d] on broker %d" .format(topic, partitionId, config.brokerId)) } } @@ -198,14 +198,14 @@ class ReplicaManager(val config: KafkaConfig, def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): (collection.Map[(String, Int), Short], Short) = { leaderAndISRRequest.partitionStateInfos.foreach(p => - stateChangeLogger.trace("Broker %d handling LeaderAndIsr request correlationId %d received from controller %d epoch %d for partition [%s,%d]" + stateChangeLogger.trace("Broker %d handling LeaderAndIsr request correlation id %d received from controller %d epoch %d for partition [%s,%d]" .format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, leaderAndISRRequest.controllerEpoch, p._1._1, p._1._2))) info("Handling LeaderAndIsr request %s".format(leaderAndISRRequest)) val responseMap = new collection.mutable.HashMap[(String, Int), Short] if(leaderAndISRRequest.controllerEpoch < controllerEpoch) { - stateChangeLogger.error("Broker %d received LeaderAndIsr request correlationId %d with an old controllerEpoch %d, latest known controllerEpoch is %d" + stateChangeLogger.warn("Broker %d received LeaderAndIsr request correlation id %d with an old controller epoch %d. Latest known controller epoch is %d" .format(localBrokerId, leaderAndISRRequest.controllerEpoch, leaderAndISRRequest.correlationId, controllerEpoch)) (responseMap, ErrorMapping.StaleControllerEpochCode) }else { @@ -221,7 +221,7 @@ class ReplicaManager(val config: KafkaConfig, if(requestedLeaderId == config.brokerId) makeLeader(controllerId, controllerEpoch, topic, partitionId, partitionStateInfo, leaderAndISRRequest.correlationId) else - makeFollower(controllerId, controllerEpoch, topic, partitionId, partitionStateInfo, leaderAndISRRequest.leaders, + makeFollower(controllerId, controllerEpoch, topic, partitionId, partitionStateInfo, leaderAndISRRequest.aliveLeaders, leaderAndISRRequest.correlationId) } catch { case e => @@ -232,10 +232,9 @@ class ReplicaManager(val config: KafkaConfig, errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) } responseMap.put(topicAndPartition, errorCode) - leaderAndISRRequest.partitionStateInfos.foreach(p => - stateChangeLogger.trace("Broker %d handled LeaderAndIsr request correlationId %d received from controller %d epoch %d for partition [%s,%d]" - .format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, - leaderAndISRRequest.controllerEpoch, p._1._1, p._1._2))) + stateChangeLogger.trace("Broker %d handled LeaderAndIsr request correlationId %d received from controller %d epoch %d for partition [%s,%d]" + .format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, leaderAndISRRequest.controllerEpoch, + topicAndPartition._1, topicAndPartition._2)) } info("Handled leader and isr request %s".format(leaderAndISRRequest)) // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions @@ -266,15 +265,14 @@ class ReplicaManager(val config: KafkaConfig, } private def makeFollower(controllerId: Int, epoch: Int, topic: String, partitionId: Int, - partitionStateInfo: PartitionStateInfo, liveBrokers: Set[Broker], correlationId: Int) { + partitionStateInfo: PartitionStateInfo, aliveLeaders: Set[Broker], correlationId: Int) { val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch - val leaderBrokerId: Int = leaderIsrAndControllerEpoch.leaderAndIsr.leader stateChangeLogger.trace(("Broker %d received LeaderAndIsr request correlationId %d from controller %d epoch %d " + "starting the become-follower transition for partition [%s,%d]") .format(localBrokerId, correlationId, controllerId, epoch, topic, partitionId)) val partition = getOrCreatePartition(topic, partitionId, partitionStateInfo.replicationFactor) - if (partition.makeFollower(controllerId, topic, partitionId, leaderIsrAndControllerEpoch, liveBrokers, correlationId)) { + if (partition.makeFollower(controllerId, topic, partitionId, leaderIsrAndControllerEpoch, aliveLeaders, correlationId)) { // remove this replica's partition from the ISR expiration queue leaderPartitionsLock synchronized { leaderPartitions -= partition diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index afe9e22..c064c5c 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -72,14 +72,14 @@ abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0, purge newGauge( "PurgatorySize", new Gauge[Int] { - def getValue = watchersForKey.values.map(_.numRequests).sum + expiredRequestReaper.numRequests + def value = watchersForKey.values.map(_.numRequests).sum + expiredRequestReaper.numRequests } ) newGauge( "NumDelayedRequests", new Gauge[Int] { - def getValue = expiredRequestReaper.unsatisfied.get() + def value = expiredRequestReaper.unsatisfied.get() } ) diff --git a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java index ad00db6..7909d25 100644 --- a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java +++ b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java @@ -221,7 +221,7 @@ public class KafkaMigrationTool kafkaProducerProperties_08.setProperty("serializer.class", "kafka.serializer.DefaultEncoder"); // create a producer channel instead int queueSize = options.valueOf(queueSizeOpt); - ProducerDataChannel> producerDataChannel = new ProducerDataChannel>(queueSize); + ProducerDataChannel> producerDataChannel = new ProducerDataChannel>(queueSize); int threadId = 0; Runtime.getRuntime().addShutdownHook(new Thread() { @@ -280,8 +280,7 @@ public class KafkaMigrationTool } } - - private static class ProducerDataChannel { + static class ProducerDataChannel { private final int producerQueueSize; private final BlockingQueue producerRequestQueue; @@ -301,14 +300,14 @@ public class KafkaMigrationTool private static class MigrationThread extends Thread { private final Object stream; - private final ProducerDataChannel> producerDataChannel; + private final ProducerDataChannel> producerDataChannel; private final int threadId; private final String threadName; private final org.apache.log4j.Logger logger; private CountDownLatch shutdownComplete = new CountDownLatch(1); private final AtomicBoolean isRunning = new AtomicBoolean(true); - MigrationThread(Object _stream, ProducerDataChannel> _producerDataChannel, int _threadId) { + MigrationThread(Object _stream, ProducerDataChannel> _producerDataChannel, int _threadId) { stream = _stream; producerDataChannel = _producerDataChannel; threadId = _threadId; @@ -337,7 +336,7 @@ public class KafkaMigrationTool ((ByteBuffer)payload_07).get(bytes); if(logger.isDebugEnabled()) logger.debug("Migration thread " + threadId + " sending message of size " + bytes.length + " to topic "+ topic); - KeyedMessage producerData = new KeyedMessage((String)topic, null, bytes); + KeyedMessage producerData = new KeyedMessage((String)topic, null, bytes); producerDataChannel.sendRequest(producerData); } logger.info("Migration thread " + threadName + " finished running"); @@ -363,17 +362,17 @@ public class KafkaMigrationTool } } - private static class ProducerThread extends Thread { - private final ProducerDataChannel> producerDataChannel; - private final Producer producer; + static class ProducerThread extends Thread { + private final ProducerDataChannel> producerDataChannel; + private final Producer producer; private final int threadId; private String threadName; private org.apache.log4j.Logger logger; private CountDownLatch shutdownComplete = new CountDownLatch(1); - private KeyedMessage shutdownMessage = new KeyedMessage("shutdown", null, null); + private KeyedMessage shutdownMessage = new KeyedMessage("shutdown", null, null); - public ProducerThread(ProducerDataChannel> _producerDataChannel, - Producer _producer, + public ProducerThread(ProducerDataChannel> _producerDataChannel, + Producer _producer, int _threadId) { producerDataChannel = _producerDataChannel; producer = _producer; @@ -386,9 +385,11 @@ public class KafkaMigrationTool public void run() { try{ while(true) { - KeyedMessage data = producerDataChannel.receiveRequest(); - if(!data.equals(shutdownMessage)) + KeyedMessage data = producerDataChannel.receiveRequest(); + if(!data.equals(shutdownMessage)) { producer.send(data); + if(logger.isDebugEnabled()) logger.debug("Sending message %s".format(new String(data.message()))); + } else break; } @@ -412,6 +413,7 @@ public class KafkaMigrationTool public void awaitShutdown() { try { shutdownComplete.await(); + producer.close(); logger.info("Producer thread " + threadName + " shutdown complete"); } catch(InterruptedException ie) { logger.warn("Interrupt during shutdown of ProducerThread", ie); diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 5c4b3d2..3d22dc7 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -24,6 +24,9 @@ import scala.collection.JavaConversions._ import java.util.concurrent.CountDownLatch import kafka.consumer._ import kafka.serializer._ +import collection.mutable.ListBuffer +import kafka.tools.KafkaMigrationTool.{ProducerThread, ProducerDataChannel} +import kafka.javaapi object MirrorMaker extends Logging { @@ -59,7 +62,13 @@ object MirrorMaker extends Logging { .describedAs("Number of threads") .ofType(classOf[java.lang.Integer]) .defaultsTo(1) - + + val bufferSizeOpt = parser.accepts("queue.size", "Number of messages that are buffered between the consumer and producer") + .withRequiredArg() + .describedAs("Queue size in terms of number of messages") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(10000); + val whitelistOpt = parser.accepts("whitelist", "Whitelist of topics to mirror.") .withRequiredArg() @@ -88,6 +97,7 @@ object MirrorMaker extends Logging { } val numStreams = options.valueOf(numStreamsOpt) + val bufferSize = options.valueOf(bufferSizeOpt).intValue() val producers = (1 to options.valueOf(numProducersOpt).intValue()).map(_ => { val config = new ProducerConfig( @@ -95,52 +105,63 @@ object MirrorMaker extends Logging { new Producer[Array[Byte], Array[Byte]](config) }) - val threads = { - val connectors = options.valuesOf(consumerConfigOpt).toList - .map(cfg => new ConsumerConfig(Utils.loadProps(cfg.toString))) - .map(new ZookeeperConsumerConnector(_)) + val connectors = options.valuesOf(consumerConfigOpt).toList + .map(cfg => new ConsumerConfig(Utils.loadProps(cfg.toString))) + .map(new ZookeeperConsumerConnector(_)) - Runtime.getRuntime.addShutdownHook(new Thread() { - override def run() { - connectors.foreach(_.shutdown()) - producers.foreach(_.close()) - } - }) + val filterSpec = if (options.has(whitelistOpt)) + new Whitelist(options.valueOf(whitelistOpt)) + else + new Blacklist(options.valueOf(blacklistOpt)) - val filterSpec = if (options.has(whitelistOpt)) - new Whitelist(options.valueOf(whitelistOpt)) - else - new Blacklist(options.valueOf(blacklistOpt)) + val streams = + connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue(), new DefaultDecoder(), new DefaultDecoder())) - val streams = - connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue(), new DefaultDecoder(), new DefaultDecoder())) + val producerDataChannel = new ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]](bufferSize); - streams.flatten.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producers, streamAndIndex._2)) - } + val consumerThreads = + streams.flatten.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producerDataChannel, streamAndIndex._2)) + + val producerThreads = new ListBuffer[ProducerThread]() + + 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"); + } + }) - threads.foreach(_.start()) + // create producer threads + var i: Int = 1 + for(producer <- producers) { + val producerThread: KafkaMigrationTool.ProducerThread = new KafkaMigrationTool.ProducerThread(producerDataChannel, + new javaapi.producer.Producer[Array[Byte], Array[Byte]](producer), i) + producerThreads += producerThread + i += 1 + } - threads.foreach(_.awaitShutdown()) + consumerThreads.foreach(_.start) + producerThreads.foreach(_.start) } class MirrorMakerThread(stream: KafkaStream[Array[Byte], Array[Byte]], - producers: Seq[Producer[Array[Byte], Array[Byte]]], + producerDataChannel: ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]], threadId: Int) extends Thread with Logging { private val shutdownLatch = new CountDownLatch(1) private val threadName = "mirrormaker-" + threadId - private val producerSelector = Utils.circularIterator(producers) this.setName(threadName) override def run() { try { for (msgAndMetadata <- stream) { - val producer = producerSelector.next() - val pd = new KeyedMessage[Array[Byte], Array[Byte]]( - msgAndMetadata.topic, msgAndMetadata.message) - producer.send(pd) + val pd = new KeyedMessage[Array[Byte], Array[Byte]](msgAndMetadata.topic, msgAndMetadata.message) + producerDataChannel.sendRequest(pd) } } catch { case e => @@ -155,9 +176,7 @@ object MirrorMaker extends Logging { try { shutdownLatch.await() } catch { - case e: InterruptedException => fatal( - "Shutdown of thread %s interrupted. This might leak data!" - .format(threadName)) + case e: InterruptedException => fatal("Shutdown of thread %s interrupted. This might leak data!".format(threadName)) } } } diff --git a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala index d744a78..814d61a 100644 --- a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala +++ b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala @@ -43,7 +43,7 @@ object ReplayLogProducer extends Logging { // consumer properties val consumerProps = new Properties consumerProps.put("group.id", GroupId) - consumerProps.put("zk.connect", config.zkConnect) + consumerProps.put("zookeeper.connect", config.zkConnect) consumerProps.put("consumer.timeout.ms", "10000") consumerProps.put("auto.offset.reset", OffsetRequest.SmallestTimeString) consumerProps.put("fetch.message.max.bytes", (1024*1024).toString) @@ -139,7 +139,7 @@ object ReplayLogProducer extends Logging { class ZKConsumerThread(config: Config, stream: KafkaStream[Array[Byte], Array[Byte]]) extends Thread with Logging { val shutdownLatch = new CountDownLatch(1) val props = new Properties() - props.put("broker.list", config.brokerList) + props.put("metadata.broker.list", config.brokerList) props.put("reconnect.interval", Integer.MAX_VALUE.toString) props.put("send.buffer.bytes", (64*1024).toString) props.put("compression.codec", config.compressionCodec.codec.toString) diff --git a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala index d9c8bae..eac9af2 100644 --- a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala +++ b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala @@ -25,7 +25,7 @@ object VerifyConsumerRebalance extends Logging { def main(args: Array[String]) { val parser = new OptionParser() - val zkConnectOpt = parser.accepts("zk.connect", "ZooKeeper connect string."). + val zkConnectOpt = parser.accepts("zookeeper.connect", "ZooKeeper connect string."). withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]); val groupOpt = parser.accepts("group", "Consumer group."). withRequiredArg().ofType(classOf[String]) @@ -99,7 +99,7 @@ object VerifyConsumerRebalance extends Logging { partitions.foreach { partition => // check if there is a node for [partition] if(!partitionsWithOwners.exists(p => p.equals(partition))) { - error("No owner for topic %s partition %s".format(topic, partition)) + error("No owner for partition [%s,%d]".format(topic, partition)) rebalanceSucceeded = false } // try reading the partition owner path for see if a valid consumer id exists there @@ -109,7 +109,7 @@ object VerifyConsumerRebalance extends Logging { case None => null } if(partitionOwner == null) { - error("No owner for topic %s partition %s".format(topic, partition)) + error("No owner for partition [%s,%d]".format(topic, partition)) rebalanceSucceeded = false } else { @@ -117,12 +117,12 @@ object VerifyConsumerRebalance extends Logging { consumerIdsForTopic match { case Some(consumerIds) => if(!consumerIds.contains(partitionOwner)) { - error("Owner %s for topic %s partition %s is not a valid member of consumer " + - "group %s".format(partitionOwner, topic, partition, group)) + error(("Owner %s for partition [%s,%d] is not a valid member of consumer " + + "group %s").format(partitionOwner, topic, partition, group)) rebalanceSucceeded = false } else - info("Owner of topic %s partition %s is %s".format(topic, partition, partitionOwner)) + info("Owner of partition [%s,%d] is %s".format(topic, partition, partitionOwner)) case None => { error("No consumer ids registered for topic " + topic) rebalanceSucceeded = false diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 8692abc..4eaeae8 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -487,7 +487,7 @@ object Utils extends Logging { val builder = new StringBuilder builder.append("[ ") if (valueInQuotes) - builder.append(jsonData.map("\"" + _ + "\"")).mkString(", ") + builder.append(jsonData.map("\"" + _ + "\"").mkString(", ")) else builder.append(jsonData.mkString(", ")) builder.append(" ]") diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 8c68821..c230b65 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -74,7 +74,7 @@ object ZkUtils extends Logging { ZkUtils.getChildren(zkClient, BrokerIdsPath).map(_.toInt).sorted def getAllBrokersInCluster(zkClient: ZkClient): Seq[Broker] = { - val brokerIds = ZkUtils.getChildren(zkClient, ZkUtils.BrokerIdsPath).sorted + val brokerIds = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerIdsPath).sorted brokerIds.map(_.toInt).map(getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) } @@ -108,7 +108,7 @@ object ZkUtils extends Logging { val isr = leaderIsrAndEpochInfo.get("isr").get.asInstanceOf[List[Int]] val controllerEpoch = leaderIsrAndEpochInfo.get("controller_epoch").get.asInstanceOf[Int] val zkPathVersion = stat.getVersion - debug("Leader %d, Epoch %d, Isr %s, Zk path version %d for topic %s and partition %d".format(leader, epoch, + debug("Leader %d, Epoch %d, Isr %s, Zk path version %d for partition [%s,%d]".format(leader, epoch, isr.toString(), zkPathVersion, topic, partition)) Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch)) case None => None @@ -138,10 +138,10 @@ object ZkUtils extends Logging { leaderAndIsrOpt match { case Some(leaderAndIsr) => Json.parseFull(leaderAndIsr) match { - case None => throw new NoEpochForPartitionException("No epoch, leaderAndISR data for topic %s partition %d is invalid".format(topic, partition)) + case None => throw new NoEpochForPartitionException("No epoch, leaderAndISR data for partition [%s,%d] is invalid".format(topic, partition)) case Some(m) => m.asInstanceOf[Map[String, Any]].get("leader_epoch").get.asInstanceOf[Int] } - case None => throw new NoEpochForPartitionException("No epoch, ISR path for topic %s partition %d is empty" + case None => throw new NoEpochForPartitionException("No epoch, ISR path for partition [%s,%d] is empty" .format(topic, partition)) } } @@ -184,7 +184,7 @@ object ZkUtils extends Logging { def isPartitionOnBroker(zkClient: ZkClient, topic: String, partition: Int, brokerId: Int): Boolean = { val replicas = getReplicasForPartition(zkClient, topic, partition) - debug("The list of replicas for topic %s, partition %d is %s".format(topic, partition, replicas)) + debug("The list of replicas for partition [%s,%d] is %s".format(topic, partition, replicas)) replicas.contains(brokerId.toString) } @@ -333,12 +333,12 @@ object ZkUtils extends Logging { def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = { try { val stat = client.writeData(path, data, expectVersion) - info("Conditional update of zkPath %s with value %s and expected version %d succeeded, returning the new version: %d" + 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 zkPath %s with data %s and expected version %d failed".format(path, data, + error("Conditional update of path %s with data %s and expected version %d failed".format(path, data, expectVersion), e) (false, -1) } @@ -351,13 +351,13 @@ object ZkUtils extends Logging { def conditionalUpdatePersistentPathIfExists(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = { try { val stat = client.writeData(path, data, expectVersion) - info("Conditional update of zkPath %s with value %s and expected version %d succeeded, returning the new version: %d" + 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 zkPath %s with data %s and expected version %d failed".format(path, data, + error("Conditional update of path %s with data %s and expected version %d failed".format(path, data, expectVersion), e) (false, -1) } @@ -638,22 +638,7 @@ object ZkUtils extends Logging { // read the partitions and their new replica list val jsonPartitionListOpt = readDataMaybeNull(zkClient, PreferredReplicaLeaderElectionPath)._1 jsonPartitionListOpt match { - case Some(jsonPartitionList) => parsePreferredReplicaElectionData(jsonPartitionList) - case None => Set.empty[TopicAndPartition] - } - } - - def parsePreferredReplicaElectionData(jsonData: String):Set[TopicAndPartition] = { - Json.parseFull(jsonData) match { - case Some(m) => - val topicAndPartitions = m.asInstanceOf[Array[Map[String, String]]] - val partitions = topicAndPartitions.map { p => - val topicPartitionMap = p - val topic = topicPartitionMap.get("topic").get - val partition = topicPartitionMap.get("partition").get.toInt - TopicAndPartition(topic, partition) - } - Set.empty[TopicAndPartition] ++ partitions + case Some(jsonPartitionList) => PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(jsonPartitionList) case None => Set.empty[TopicAndPartition] } } @@ -792,14 +777,14 @@ class ZKGroupTopicDirs(group: String, topic: String) extends ZKGroupDirs(group) class ZKConfig(props: VerifiableProperties) { /** ZK host string */ - val zkConnect = props.getString("zk.connect", null) + val zkConnect = props.getString("zookeeper.connect", null) /** zookeeper session timeout */ - val zkSessionTimeoutMs = props.getInt("zk.session.timeout.ms", 6000) + val zkSessionTimeoutMs = props.getInt("zookeeper.session.timeout.ms", 6000) /** the max time that the client waits to establish a connection to zookeeper */ - val zkConnectionTimeoutMs = props.getInt("zk.connection.timeout.ms",zkSessionTimeoutMs) + val zkConnectionTimeoutMs = props.getInt("zookeeper.connection.timeout.ms",zkSessionTimeoutMs) /** how far a ZK follower can be behind a ZK leader */ - val zkSyncTimeMs = props.getInt("zk.sync.time.ms", 2000) + val zkSyncTimeMs = props.getInt("zookeeper.sync.time.ms", 2000) } diff --git a/core/src/test/scala/other/kafka/TestEndToEndLatency.scala b/core/src/test/scala/other/kafka/TestEndToEndLatency.scala index 98c12b7..c4aed10 100644 --- a/core/src/test/scala/other/kafka/TestEndToEndLatency.scala +++ b/core/src/test/scala/other/kafka/TestEndToEndLatency.scala @@ -38,7 +38,7 @@ object TestEndToEndLatency { consumerProps.put("group.id", topic) consumerProps.put("auto.commit", "true") consumerProps.put("auto.offset.reset", "largest") - consumerProps.put("zk.connect", zkConnect) + consumerProps.put("zookeeper.connect", zkConnect) consumerProps.put("socket.timeout.ms", 1201000.toString) val config = new ConsumerConfig(consumerProps) @@ -47,7 +47,7 @@ object TestEndToEndLatency { val iter = stream.iterator val producerProps = new Properties() - producerProps.put("broker.list", brokerList) + producerProps.put("metadata.broker.list", brokerList) producerProps.put("producer.type", "sync") val producer = new Producer[Any, Any](new ProducerConfig(producerProps)) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index b73e5d4..1754c25 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -288,7 +288,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val preferredReplicaElectionZkData = ZkUtils.readData(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath)._1 val partitionsUndergoingPreferredReplicaElection = - PreferredReplicaLeaderElectionCommand.parsePreferredReplicaJsonData(preferredReplicaElectionZkData) + PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(preferredReplicaElectionZkData) assertEquals("Preferred replica election ser-de failed", partitionsForPreferredReplicaElection, partitionsUndergoingPreferredReplicaElection) } @@ -336,6 +336,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { 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) leaderBeforeShutdown = leaderAfterShutdown controllerId = ZkUtils.getController(zkClient) @@ -345,7 +346,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient) leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id assertTrue(leaderAfterShutdown != leaderBeforeShutdown) - assertEquals(1, controller.controllerContext.allLeaders(TopicAndPartition("test", 1)).leaderAndIsr.isr.size) + assertEquals(1, controller.controllerContext.partitionLeadershipInfo(TopicAndPartition("test", 1)).leaderAndIsr.isr.size) leaderBeforeShutdown = leaderAfterShutdown controllerId = ZkUtils.getController(zkClient) @@ -355,7 +356,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient) leaderAfterShutdown = topicMetadata.partitionsMetadata.head.leader.get.id assertTrue(leaderAfterShutdown == leaderBeforeShutdown) - assertEquals(1, controller.controllerContext.allLeaders(TopicAndPartition("test", 1)).leaderAndIsr.isr.size) + assertEquals(1, controller.controllerContext.partitionLeadershipInfo(TopicAndPartition("test", 1)).leaderAndIsr.isr.size) } 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 517e9ac..045cd06 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -89,7 +89,7 @@ object SerializationTestUtils{ val leaderAndIsr2 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader2, 1, isr2, 2), 1) val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, 3)), ((topic2, 0), PartitionStateInfo(leaderAndIsr2, 3))) - new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 0, 1, 0) + new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 0, 1, 0, "") } def createTestLeaderAndIsrResponse() : LeaderAndIsrResponse = { diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index c70a435..778556e 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -330,7 +330,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar compression: CompressionCodec = NoCompressionCodec): List[String] = { val header = "test-%d-%d".format(config.brokerId, partition) val props = new Properties() - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) props.put("partitioner.class", "kafka.utils.FixedValuePartitioner") props.put("compression.codec", compression.codec.toString) props.put("key.serializer.class", classOf[IntEncoder].getName.toString) @@ -338,7 +338,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val producer: Producer[Int, String] = new Producer[Int, String](new ProducerConfig(props)) val ms = 0.until(numMessages).map(x => header + config.brokerId + "-" + partition + "-" + x) producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) - debug("Sent %d messages to broker %d for topic %s and partition %d".format(ms.size, config.brokerId, topic, partition)) + debug("Sent %d messages to broker %d for partition [%s,%d]".format(ms.size, config.brokerId, topic, partition)) producer.close() ms.toList } @@ -350,7 +350,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar numParts: Int): List[String]= { var messages: List[String] = Nil val props = new Properties() - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) props.put("partitioner.class", "kafka.utils.FixedValuePartitioner") props.put("key.serializer.class", classOf[IntEncoder].getName.toString) props.put("serializer.class", classOf[StringEncoder].getName) @@ -359,7 +359,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val ms = 0.until(messagesPerNode).map(x => header + config.brokerId + "-" + partition + "-" + x) producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) messages ++= ms - debug("Sent %d messages to broker %d for topic %s and partition %d".format(ms.size, config.brokerId, topic, partition)) + debug("Sent %d messages to broker %d for partition [%s,%d]".format(ms.size, config.brokerId, topic, partition)) } producer.close() messages diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 845b966..67ed201 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -58,12 +58,12 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0 -> Seq(configs.head.brokerId))) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient) - fetcher.stopAllConnections() + fetcher.stopConnections() fetcher.startConnections(topicInfos, cluster) } override def tearDown() { - fetcher.shutdown() + fetcher.stopConnections() super.tearDown } diff --git a/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala b/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala index a3f85cf..fe5bc09 100644 --- a/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala @@ -35,20 +35,20 @@ class KafkaTimerTest extends JUnit3Suite { timer.time { clock.addMillis(1000) } - assertEquals(1, metric.getCount()) - assertTrue((metric.getMax() - 1000).abs <= Double.Epsilon) - assertTrue((metric.getMin() - 1000).abs <= Double.Epsilon) + assertEquals(1, metric.count()) + assertTrue((metric.max() - 1000).abs <= Double.Epsilon) + assertTrue((metric.min() - 1000).abs <= Double.Epsilon) } private class ManualClock extends Clock { private var ticksInNanos = 0L - override def getTick() = { + override def tick() = { ticksInNanos } - override def getTime() = { + override def time() = { TimeUnit.NANOSECONDS.toMillis(ticksInNanos) } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index b347e66..94b5a2a 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -38,6 +38,8 @@ class SocketServerTest extends JUnitSuite { port = kafka.utils.TestUtils.choosePort, numProcessorThreads = 1, maxQueuedRequests = 50, + sendBufferSize = 300000, + recvBufferSize = 300000, maxRequestSize = 50) server.startup() diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index 65a67e8..ba035b3 100644 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -61,7 +61,7 @@ class AsyncProducerTest extends JUnit3Suite { val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) props.put("producer.type", "async") props.put("queue.buffering.max.messages", "10") props.put("batch.num.messages", "1") @@ -86,7 +86,7 @@ class AsyncProducerTest extends JUnit3Suite { def testProduceAfterClosed() { val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) props.put("producer.type", "async") props.put("batch.num.messages", "1") @@ -165,7 +165,7 @@ class AsyncProducerTest extends JUnit3Suite { producerDataList.append(new KeyedMessage[Int,Message]("topic2", 4, new Message("msg5".getBytes))) val props = new Properties() - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) val broker1 = new Broker(0, "localhost", 9092) val broker2 = new Broker(1, "localhost", 9093) broker1 @@ -215,7 +215,7 @@ class AsyncProducerTest extends JUnit3Suite { def testSerializeEvents() { val produceData = TestUtils.getMsgStrings(5).map(m => new KeyedMessage[String,String]("topic1",m)) val props = new Properties() - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) val config = new ProducerConfig(props) // form expected partitions metadata val topic1Metadata = getTopicMetadata("topic1", 0, 0, "localhost", 9092) @@ -241,7 +241,7 @@ class AsyncProducerTest extends JUnit3Suite { val producerDataList = new ArrayBuffer[KeyedMessage[String,Message]] producerDataList.append(new KeyedMessage[String,Message]("topic1", "key1", new Message("msg1".getBytes))) val props = new Properties() - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) val config = new ProducerConfig(props) // form expected partitions metadata @@ -270,7 +270,7 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testNoBroker() { val props = new Properties() - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) val config = new ProducerConfig(props) // create topic metadata with 0 partitions @@ -301,7 +301,7 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testIncompatibleEncoder() { val props = new Properties() - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) val config = new ProducerConfig(props) val producer=new Producer[String, String](config) @@ -318,7 +318,7 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testRandomPartitioner() { val props = new Properties() - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) val config = new ProducerConfig(props) // create topic metadata with 0 partitions @@ -392,7 +392,7 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testFailedSendRetryLogic() { val props = new Properties() - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) props.put("request.required.acks", "1") props.put("serializer.class", classOf[StringEncoder].getName.toString) props.put("key.serializer.class", classOf[NullEncoder[Int]].getName.toString) diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 507e6a8..1d82598 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -91,7 +91,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500) val props1 = new util.Properties() - props1.put("broker.list", "localhost:80,localhost:81") + props1.put("metadata.broker.list", "localhost:80,localhost:81") props1.put("serializer.class", "kafka.serializer.StringEncoder") val producerConfig1 = new ProducerConfig(props1) val producer1 = new Producer[String, String](producerConfig1) @@ -106,7 +106,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ } val props2 = new util.Properties() - props2.put("broker.list", "localhost:80," + TestUtils.getBrokerListStrFromConfigs(Seq( config1))) + props2.put("metadata.broker.list", "localhost:80," + TestUtils.getBrokerListStrFromConfigs(Seq( config1))) props2.put("serializer.class", "kafka.serializer.StringEncoder") val producerConfig2= new ProducerConfig(props2) val producer2 = new Producer[String, String](producerConfig2) @@ -119,7 +119,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ } val props3 = new util.Properties() - props3.put("broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + props3.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) props3.put("serializer.class", "kafka.serializer.StringEncoder") val producerConfig3 = new ProducerConfig(props3) val producer3 = new Producer[String, String](producerConfig3) @@ -137,7 +137,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val props1 = new util.Properties() props1.put("serializer.class", "kafka.serializer.StringEncoder") props1.put("partitioner.class", "kafka.utils.StaticPartitioner") - props1.put("broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + props1.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) props1.put("request.required.acks", "2") props1.put("request.timeout.ms", "1000") @@ -198,7 +198,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ props.put("partitioner.class", "kafka.utils.StaticPartitioner") props.put("request.timeout.ms", "2000") props.put("request.required.acks", "1") - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) // create topic AdminUtils.createTopicWithAssignment(zkClient, "new-topic", Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0))) @@ -250,7 +250,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ props.put("serializer.class", "kafka.serializer.StringEncoder") props.put("partitioner.class", "kafka.utils.StaticPartitioner") props.put("request.timeout.ms", String.valueOf(timeoutMs)) - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) props.put("request.required.acks", "1") val config = new ProducerConfig(props) @@ -302,7 +302,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") props.put("partitioner.class", "kafka.utils.StaticPartitioner") - props.put("broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) val config = new ProducerConfig(props) val producer = new Producer[String, String](config) diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 8a42074..48c63ad 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -133,7 +133,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { leaderAndIsr.put((topic, partitionId), new LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId2, List(brokerId1, brokerId2)), 2)) val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, 1)).toMap - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, controllerId, staleControllerEpoch, 0) + val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, controllerId, + staleControllerEpoch, 0, "") controllerChannelManager.sendRequest(brokerId2, leaderAndIsrRequest, staleControllerEpochCallback) TestUtils.waitUntilTrue(() => staleControllerEpochDetected == true, 1000) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 6801f4e..9aea67b 100644 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -207,7 +207,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { props.put("log.retention.hours", "10") props.put("log.retention.check.interval.ms", (5*1000*60).toString) props.put("log.segment.bytes", logSize.toString) - props.put("zk.connect", zkConnect.toString) + props.put("zookeeper.connect", zkConnect.toString) props } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 40bfacb..00ea98f 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -129,7 +129,7 @@ object TestUtils extends Logging { props.put("host.name", "localhost") props.put("port", port.toString) props.put("log.dir", TestUtils.tempDir().getAbsolutePath) - props.put("zk.connect", TestZKUtils.zookeeperConnect) + props.put("zookeeper.connect", TestZKUtils.zookeeperConnect) props.put("replica.socket.timeout.ms", "1500") props } @@ -140,12 +140,12 @@ object TestUtils extends Logging { def createConsumerProperties(zkConnect: String, groupId: String, consumerId: String, consumerTimeout: Long = -1): Properties = { val props = new Properties - props.put("zk.connect", zkConnect) + props.put("zookeeper.connect", zkConnect) props.put("group.id", groupId) props.put("consumer.id", consumerId) props.put("consumer.timeout.ms", consumerTimeout.toString) - props.put("zk.session.timeout.ms", "400") - props.put("zk.sync.time.ms", "200") + props.put("zookeeper.session.timeout.ms", "400") + props.put("zookeeper.sync.time.ms", "200") props.put("auto.commit.interval.ms", "1000") props.put("rebalance.max.retries", "4") @@ -293,7 +293,7 @@ object TestUtils extends Logging { encoder: Encoder[V] = new DefaultEncoder(), keyEncoder: Encoder[K] = new DefaultEncoder()): Producer[K, V] = { val props = new Properties() - props.put("broker.list", brokerList) + props.put("metadata.broker.list", brokerList) props.put("send.buffer.bytes", "65536") props.put("connect.timeout.ms", "100000") props.put("reconnect.interval", "10000") @@ -304,7 +304,7 @@ object TestUtils extends Logging { def getProducerConfig(brokerList: String, partitioner: String = "kafka.producer.DefaultPartitioner"): Properties = { val props = new Properties() - props.put("broker.list", brokerList) + props.put("metadata.broker.list", brokerList) props.put("partitioner.class", partitioner) props.put("message.send.max.retries", "3") props.put("retry.backoff.ms", "1000") @@ -409,7 +409,7 @@ object TestUtils extends Logging { ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition), ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch)) } catch { - case oe => error("Error while electing leader for topic %s partition %d".format(topic, partition), oe) + case oe => error("Error while electing leader for partition [%s,%d]".format(topic, partition), oe) } } } @@ -420,9 +420,9 @@ object TestUtils extends Logging { val leaderExistsOrChanged = leaderLock.newCondition() if(oldLeaderOpt == None) - info("Waiting for leader to be elected for topic %s partition %d".format(topic, partition)) + info("Waiting for leader to be elected for partition [%s,%d]".format(topic, partition)) else - info("Waiting for leader for topic %s partition %d to be changed from old leader %d".format(topic, partition, oldLeaderOpt.get)) + info("Waiting for leader for partition [%s,%d] to be changed from old leader %d".format(topic, partition, oldLeaderOpt.get)) leaderLock.lock() try { @@ -433,10 +433,10 @@ object TestUtils extends Logging { leader match { case Some(l) => if(oldLeaderOpt == None) - info("Leader %d is elected for topic %s partition %d".format(l, topic, partition)) + info("Leader %d is elected for partition [%s,%d]".format(l, topic, partition)) else - info("Leader for topic %s partition %d is changed from %d to %d".format(topic, partition, oldLeaderOpt.get, l)) - case None => error("Timing out after %d ms since leader is not elected for topic %s partition %d" + info("Leader for partition [%s,%d] is changed from %d to %d".format(topic, partition, oldLeaderOpt.get, l)) + case None => error("Timing out after %d ms since leader is not elected for partition [%s,%d]" .format(timeoutMs, topic, partition)) } leader diff --git a/examples/src/main/java/kafka/examples/Consumer.java b/examples/src/main/java/kafka/examples/Consumer.java index 3e9bfb8..13135b9 100644 --- a/examples/src/main/java/kafka/examples/Consumer.java +++ b/examples/src/main/java/kafka/examples/Consumer.java @@ -42,10 +42,10 @@ public class Consumer extends Thread private static ConsumerConfig createConsumerConfig() { Properties props = new Properties(); - props.put("zk.connect", KafkaProperties.zkConnect); + props.put("zookeeper.connect", KafkaProperties.zkConnect); props.put("group.id", KafkaProperties.groupId); - props.put("zk.session.timeout.ms", "400"); - props.put("zk.sync.time.ms", "200"); + props.put("zookeeper.session.timeout.ms", "400"); + props.put("zookeeper.sync.time.ms", "200"); props.put("auto.commit.interval.ms", "1000"); return new ConsumerConfig(props); diff --git a/examples/src/main/java/kafka/examples/Producer.java b/examples/src/main/java/kafka/examples/Producer.java index a770a18..96e9893 100644 --- a/examples/src/main/java/kafka/examples/Producer.java +++ b/examples/src/main/java/kafka/examples/Producer.java @@ -30,7 +30,7 @@ public class Producer extends Thread public Producer(String topic) { props.put("serializer.class", "kafka.serializer.StringEncoder"); - props.put("broker.list", "localhost:9092"); + props.put("metadata.broker.list", "localhost:9092"); // Use random partitioner. Don't need the key type. Just set it to Integer. // The message is of type String. producer = new kafka.javaapi.producer.Producer(new ProducerConfig(props)); diff --git a/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala b/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala index ee2ce95..3158a22 100644 --- a/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala +++ b/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala @@ -128,7 +128,7 @@ object ConsumerPerformance { props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString) props.put("auto.offset.reset", if(options.has(resetBeginningOffsetOpt)) "largest" else "smallest") - props.put("zk.connect", options.valueOf(zkConnectOpt)) + props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) props.put("consumer.timeout.ms", "5000") val consumerConfig = new ConsumerConfig(props) val numThreads = options.valueOf(numThreadsOpt).intValue diff --git a/perf/src/main/scala/kafka/perf/ProducerPerformance.scala b/perf/src/main/scala/kafka/perf/ProducerPerformance.scala index 851a99e..ad2ac26 100644 --- a/perf/src/main/scala/kafka/perf/ProducerPerformance.scala +++ b/perf/src/main/scala/kafka/perf/ProducerPerformance.scala @@ -174,7 +174,7 @@ object ProducerPerformance extends Logging { val allDone: CountDownLatch, val rand: Random) extends Runnable { val props = new Properties() - props.put("broker.list", config.brokerList) + props.put("metadata.broker.list", config.brokerList) props.put("compression.codec", config.compressionCodec.codec.toString) props.put("reconnect.interval", Integer.MAX_VALUE.toString) props.put("send.buffer.bytes", (64*1024).toString) diff --git a/project/Build.scala b/project/Build.scala index 4bbdfee..f177215 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -17,18 +17,23 @@ import sbt._ import Keys._ -import java.io.File +import Process._ import scala.xml.{Node, Elem} import scala.xml.transform.{RewriteRule, RuleTransformer} 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( - version := "0.8-SNAPSHOT", organization := "org.apache", 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", + 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}, javacOptions ++= Seq("-Xlint:unchecked", "-source", "1.5"), parallelExecution in Test := false, // Prevent tests from overrunning each other libraryDependencies ++= Seq( @@ -78,17 +83,42 @@ object KafkaBuild extends Build { ) - val coreSettings = Seq( - pomPostProcess := { (pom: Node) => MetricsDepAdder(ZkClientDepAdder(pom)) } - ) val runRat = TaskKey[Unit]("run-rat-task", "Runs Apache rat on Kafka") val runRatTask = runRat := { "bin/run-rat.sh" ! } - lazy val kafka = Project(id = "Kafka", base = file(".")).aggregate(core, examples, contrib, perf).settings((commonSettings ++ runRatTask): _*) - lazy val core = Project(id = "core", base = file("core")).settings(commonSettings: _*).settings(coreSettings: _*) + val release = TaskKey[Unit]("release", "Creates a deployable release directory file with dependencies, config, and scripts.") + val releaseTask = release <<= ( packageBin in (core, Compile), dependencyClasspath in (core, Runtime), exportedProducts in Compile, + target, releaseName in core ) map { (packageBin, deps, products, target, releaseName) => + val jarFiles = deps.files.filter(f => !products.files.contains(f) && f.getName.endsWith(".jar")) + val destination = target / "RELEASE" / releaseName + IO.copyFile(packageBin, destination / packageBin.getName) + IO.copy(jarFiles.map { f => (f, destination / "libs" / f.getName) }) + IO.copyDirectory(file("config"), destination / "config") + IO.copyDirectory(file("bin"), destination / "bin") + for {file <- (destination / "bin").listFiles} { file.setExecutable(true, true) } + } + + val releaseZip = TaskKey[Unit]("release-zip", "Creates a deployable zip file with dependencies, config, and scripts.") + val releaseZipTask = releaseZip <<= (release, target, releaseName in core) map { (release, target, releaseName) => + val zipPath = target / "RELEASE" / "%s.zip".format(releaseName) + IO.delete(zipPath) + IO.zip((target/"RELEASE" ** releaseName ***) x relativeTo(target/"RELEASE"), zipPath) + } + + val releaseTar = TaskKey[Unit]("release-tar", "Creates a deployable tar.gz file with dependencies, config, and scripts.") + val releaseTarTask = releaseTar <<= ( release, target, releaseName in core) map { (release, target, releaseName) => + Process(Seq("tar", "czf", "%s.tar.gz".format(releaseName), releaseName), target / "RELEASE").! match { + case 0 => () + case n => sys.error("Failed to run native tar application!") + } + } + + lazy val kafka = Project(id = "Kafka", base = file(".")).aggregate(core, examples, contrib, perf).settings((commonSettings ++ + runRatTask ++ releaseTask ++ releaseZipTask ++ releaseTarTask): _*) + lazy val core = Project(id = "core", base = file("core")).settings(commonSettings: _*) lazy val examples = Project(id = "java-examples", base = file("examples")).settings(commonSettings :_*) dependsOn (core) lazy val perf = Project(id = "perf", base = file("perf")).settings((Seq(name := "kafka-perf") ++ commonSettings):_*) dependsOn (core) @@ -96,48 +126,4 @@ object KafkaBuild extends Build { lazy val hadoopProducer = Project(id = "hadoop-producer", base = file("contrib/hadoop-producer")).settings(hadoopSettings ++ commonSettings: _*) dependsOn (core) lazy val hadoopConsumer = Project(id = "hadoop-consumer", base = file("contrib/hadoop-consumer")).settings(hadoopSettings ++ commonSettings: _*) dependsOn (core) - - // POM Tweaking for core: - def zkClientDep = - - zkclient - zkclient - 20120522 - compile - - - def metricsDeps = - - - com.yammer.metrics - metrics-core - 3.0.0-c0c8be71 - compile - - - com.yammer.metrics - metrics-annotations - 3.0.0-c0c8be71 - compile - - - - object ZkClientDepAdder extends RuleTransformer(new RewriteRule() { - override def transform(node: Node): Seq[Node] = node match { - case Elem(prefix, "dependencies", attribs, scope, deps @ _*) => { - Elem(prefix, "dependencies", attribs, scope, deps ++ zkClientDep:_*) - } - case other => other - } - }) - - object MetricsDepAdder extends RuleTransformer(new RewriteRule() { - override def transform(node: Node): Seq[Node] = node match { - case Elem(prefix, "dependencies", attribs, scope, deps @ _*) => { - Elem(prefix, "dependencies", attribs, scope, deps ++ metricsDeps:_*) - } - case other => other - } - }) - } diff --git a/project/build/KafkaProject.scala b/project/build/KafkaProject.scala index fac723a..cd406c1 100644 --- a/project/build/KafkaProject.scala +++ b/project/build/KafkaProject.scala @@ -62,52 +62,6 @@ class KafkaProject(info: ProjectInfo) extends ParentProject(info) with IdeaProje - def zkClientDep = - - com.101tec - zkclient - 0.2 - compile - - - def metricsDepsCore = - - com.yammer.metrics - metrics-core - 3.0.0-SNAPSHOT - compile - - - def metricsDepsAnnotations = - - com.yammer.metrics - metrics-annotation - 3.0.0-SNAPSHOT - compile - - - object ZkClientDepAdder extends RuleTransformer(new RewriteRule() { - override def transform(node: Node): Seq[Node] = node match { - case Elem(prefix, "dependencies", attribs, scope, deps @ _*) => { - Elem(prefix, "dependencies", attribs, scope, deps ++ zkClientDep:_*) - } - case other => other - } - }) - - object MetricsDepAdder extends RuleTransformer(new RewriteRule() { - override def transform(node: Node): Seq[Node] = node match { - case Elem(prefix, "dependencies", attribs, scope, deps @ _*) => { - Elem(prefix, "dependencies", attribs, scope, deps ++ metricsDepsCore ++ metricsDepsAnnotations:_*) - } - case other => other - } - }) - - override def pomPostProcess(pom: Node): Node = { - MetricsDepAdder(ZkClientDepAdder(pom)) - } - override def organization = "org.apache" override def filterScalaJars = false diff --git a/project/plugins.sbt b/project/plugins.sbt index 48d44c8..e8c3e53 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,5 +1,5 @@ resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns) -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.5") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.8") addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") diff --git a/sbt b/sbt index 9ffb03b..944ebf8 100755 --- a/sbt +++ b/sbt @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -java -Xmx1024M -XX:MaxPermSize=512m -jar `dirname $0`/lib/sbt-launch.jar "$@" +java -Xmx1024M -XX:MaxPermSize=512m -Dbuild.number="$BUILD_NUMBER" -jar `dirname $0`/lib/sbt-launch.jar "$@" diff --git a/system_test/migration_tool_testsuite/config/migration_producer.properties b/system_test/migration_tool_testsuite/config/migration_producer.properties index af080ae..1750807 100644 --- a/system_test/migration_tool_testsuite/config/migration_producer.properties +++ b/system_test/migration_tool_testsuite/config/migration_producer.properties @@ -20,7 +20,7 @@ # configure brokers statically # format: host1:port1,host2:port2 ... -broker.list=localhost:9094,localhost:9095,localhost:9096 +metadata.broker.list=localhost:9094,localhost:9095,localhost:9096 # discover brokers from ZK #zk.connect= diff --git a/system_test/migration_tool_testsuite/config/server.properties b/system_test/migration_tool_testsuite/config/server.properties index d231d4c..6ecbb71 100644 --- a/system_test/migration_tool_testsuite/config/server.properties +++ b/system_test/migration_tool_testsuite/config/server.properties @@ -115,8 +115,10 @@ enable.zookeeper=true # You can also append an optional chroot string to the urls to specify the # root directory for all kafka znodes. zk.connect=localhost:2181 +zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper zk.connection.timeout.ms=1000000 +zookeeper.connection.timeout.ms=1000000 monitoring.period.secs=1 diff --git a/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json b/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json index cf84caa..1904ab5 100644 --- a/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "brokerid": "1", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_1_logs", "log_filename": "kafka_server_1.log", @@ -39,7 +39,7 @@ "entity_id": "2", "port": "9092", "brokerid": "2", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_2.log", @@ -49,7 +49,7 @@ "entity_id": "3", "port": "9093", "brokerid": "3", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_3.log", diff --git a/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json b/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json index 0b413c4..8cacc69 100644 --- a/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json @@ -30,7 +30,7 @@ "entity_id": "1", "port": "9091", "brokerid": "1", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_1_logs", "log_filename": "kafka_server_1.log", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "brokerid": "2", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_2.log", @@ -50,7 +50,7 @@ "entity_id": "3", "port": "9093", "brokerid": "3", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_3.log", diff --git a/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json b/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json index 5c6baaf..4dbd80b 100644 --- a/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json @@ -30,7 +30,7 @@ "entity_id": "1", "port": "9091", "brokerid": "1", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_1_logs", "log_filename": "kafka_server_1.log", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "brokerid": "2", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_2.log", @@ -50,7 +50,7 @@ "entity_id": "3", "port": "9093", "brokerid": "3", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_3.log", diff --git a/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json b/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json index 8597e1a..e46b453 100644 --- a/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json @@ -39,7 +39,7 @@ "entity_id": "2", "port": "9091", "brokerid": "1", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_2.log", @@ -49,7 +49,7 @@ "entity_id": "3", "port": "9092", "brokerid": "2", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_3.log", @@ -59,7 +59,7 @@ "entity_id": "4", "port": "9093", "brokerid": "3", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", diff --git a/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json b/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json index ec3290f..10f5955 100644 --- a/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json +++ b/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json @@ -39,7 +39,7 @@ "entity_id": "2", "port": "9091", "brokerid": "1", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_2.log", @@ -49,7 +49,7 @@ "entity_id": "3", "port": "9092", "brokerid": "2", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_3.log", @@ -59,7 +59,7 @@ "entity_id": "4", "port": "9093", "brokerid": "3", - "07_client": "true", + "version": "0.7", "log.file.size": "51200", "log.dir": "/tmp/kafka_server_4_logs", "log_filename": "kafka_server_4.log", diff --git a/system_test/mirror_maker_testsuite/cluster_config.json b/system_test/mirror_maker_testsuite/cluster_config.json index b679eea..5b908ff 100644 --- a/system_test/mirror_maker_testsuite/cluster_config.json +++ b/system_test/mirror_maker_testsuite/cluster_config.json @@ -121,6 +121,16 @@ "kafka_home": "default", "java_home": "default", "jmx_port": "9112" + }, + + { + "entity_id": "13", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9113" } ] } diff --git a/system_test/mirror_maker_testsuite/config/mirror_consumer.properties b/system_test/mirror_maker_testsuite/config/mirror_consumer.properties index bb1a1cc..e90634a 100644 --- a/system_test/mirror_maker_testsuite/config/mirror_consumer.properties +++ b/system_test/mirror_maker_testsuite/config/mirror_consumer.properties @@ -1,5 +1,5 @@ -zk.connect=localhost:2108 -zk.connection.timeout.ms=1000000 +zookeeper.connect=localhost:2108 +zookeeper.connection.timeout.ms=1000000 group.id=mm_regtest_grp auto.commit.interval.ms=120000 auto.offset.reset=smallest @@ -8,5 +8,5 @@ auto.offset.reset=smallest #rebalance.backoff.ms=2000 socket.receive.buffer.bytes=1048576 fetch.message.max.bytes=1048576 -zk.sync.time.ms=15000 +zookeeper.sync.time.ms=15000 shallow.iterator.enable=false diff --git a/system_test/mirror_maker_testsuite/config/mirror_producer.properties b/system_test/mirror_maker_testsuite/config/mirror_producer.properties index 7db5bfc..b2bf2c2 100644 --- a/system_test/mirror_maker_testsuite/config/mirror_producer.properties +++ b/system_test/mirror_maker_testsuite/config/mirror_producer.properties @@ -1,5 +1,6 @@ producer.type=async queue.enqueue.timeout.ms=-1 -broker.list=localhost:9094 +metadata.broker.list=localhost:9094 compression.codec=0 - +message.send.max.retries=3 +request.required.acks=1 diff --git a/system_test/mirror_maker_testsuite/config/server.properties b/system_test/mirror_maker_testsuite/config/server.properties index dacf158..36dd68d 100644 --- a/system_test/mirror_maker_testsuite/config/server.properties +++ b/system_test/mirror_maker_testsuite/config/server.properties @@ -114,10 +114,10 @@ enable.zookeeper=true # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". # You can also append an optional chroot string to the urls to specify the # root directory for all kafka znodes. -zk.connect=localhost:2181 +zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 +zookeeper.connection.timeout.ms=1000000 monitoring.period.secs=1 message.max.bytes=1000000 diff --git a/system_test/mirror_maker_testsuite/mirror_maker_test.py b/system_test/mirror_maker_testsuite/mirror_maker_test.py index 48b0d25..4dc3cdf 100644 --- a/system_test/mirror_maker_testsuite/mirror_maker_test.py +++ b/system_test/mirror_maker_testsuite/mirror_maker_test.py @@ -76,6 +76,8 @@ class MirrorMakerTest(ReplicationUtils, SetupUtils): self.testSuiteAbsPathName, SystemTestEnv.SYSTEM_TEST_CASE_PREFIX) testCasePathNameList.sort() + replicationUtils = ReplicationUtils(self) + # ============================================================= # launch each testcase one by one: testcase_1, testcase_2, ... # ============================================================= @@ -166,13 +168,13 @@ class MirrorMakerTest(ReplicationUtils, SetupUtils): self.log_message("starting mirror makers") kafka_system_test_utils.start_mirror_makers(self.systemTestEnv, self.testcaseEnv) - self.anonLogger.info("sleeping for 5s") - time.sleep(5) + self.anonLogger.info("sleeping for 10s") + time.sleep(10) - self.log_message("creating topics") - kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv) - self.anonLogger.info("sleeping for 5s") - time.sleep(5) + #self.log_message("creating topics") + #kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv) + #self.anonLogger.info("sleeping for 5s") + #time.sleep(5) # ============================================= @@ -190,7 +192,7 @@ class MirrorMakerTest(ReplicationUtils, SetupUtils): # ============================================= i = 1 numIterations = int(self.testcaseEnv.testcaseArgumentsDict["num_iteration"]) - bouncedEntityDownTimeSec = 1 + bouncedEntityDownTimeSec = 15 try: bouncedEntityDownTimeSec = int(self.testcaseEnv.testcaseArgumentsDict["bounced_entity_downtime_sec"]) except: @@ -252,6 +254,17 @@ class MirrorMakerTest(ReplicationUtils, SetupUtils): self.testcaseEnv.lock.release() time.sleep(2) + self.anonLogger.info("sleeping for 15s") + time.sleep(15) + self.anonLogger.info("terminate Mirror Maker") + cmdStr = "ps auxw | grep Mirror | grep -v grep | tr -s ' ' | cut -f2 -d ' ' | xargs kill -15" + subproc = system_test_utils.sys_call_return_subproc(cmdStr) + for line in subproc.stdout.readlines(): + line = line.rstrip('\n') + self.anonLogger.info("#### ["+line+"]") + self.anonLogger.info("sleeping for 15s") + time.sleep(15) + # ============================================= # starting consumer # ============================================= @@ -282,9 +295,8 @@ class MirrorMakerTest(ReplicationUtils, SetupUtils): # validate the data matched and checksum # ============================================= self.log_message("validating data matched") - #kafka_system_test_utils.validate_data_matched(self.systemTestEnv, self.testcaseEnv) - kafka_system_test_utils.validate_simple_consumer_data_matched(self.systemTestEnv, self.testcaseEnv) - kafka_system_test_utils.validate_broker_log_segment_checksum(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.validate_data_matched(self.systemTestEnv, self.testcaseEnv, replicationUtils) + kafka_system_test_utils.validate_broker_log_segment_checksum(self.systemTestEnv, self.testcaseEnv, "source") kafka_system_test_utils.validate_broker_log_segment_checksum(self.systemTestEnv, self.testcaseEnv, "target") # ============================================= diff --git a/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json b/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json index abb500b..287cab9 100644 --- a/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json @@ -56,6 +56,8 @@ "broker.id": "1", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" }, @@ -65,6 +67,8 @@ "broker.id": "2", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" }, @@ -74,6 +78,8 @@ "broker.id": "3", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" }, @@ -83,6 +89,8 @@ "broker.id": "4", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" }, @@ -92,6 +100,8 @@ "broker.id": "5", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" }, @@ -101,6 +111,8 @@ "broker.id": "6", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" }, @@ -114,6 +126,7 @@ "message": "500", "request-num-acks": "-1", "sync":"true", + "producer-num-retries":"5", "log_filename": "producer_performance_10.log", "config_filename": "producer_performance_10.properties" }, @@ -131,6 +144,15 @@ "log_filename": "mirror_maker_12.log", "mirror_consumer_config_filename": "mirror_consumer_12.properties", "mirror_producer_config_filename": "mirror_producer_12.properties" + }, + + { + "entity_id": "13", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_13.log", + "config_filename": "console_consumer_13.properties" } ] } diff --git a/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json b/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json index 8d25bad..5457eb1 100644 --- a/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json @@ -56,6 +56,8 @@ "broker.id": "1", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" }, @@ -65,6 +67,8 @@ "broker.id": "2", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" }, @@ -74,6 +78,8 @@ "broker.id": "3", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" }, @@ -83,6 +89,8 @@ "broker.id": "4", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" }, @@ -92,6 +100,8 @@ "broker.id": "5", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" }, @@ -101,6 +111,8 @@ "broker.id": "6", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" }, @@ -114,6 +126,7 @@ "message": "100", "request-num-acks": "-1", "sync":"true", + "producer-num-retries":"5", "log_filename": "producer_performance_10.log", "config_filename": "producer_performance_10.properties" }, @@ -131,6 +144,15 @@ "log_filename": "mirror_maker_12.log", "mirror_consumer_config_filename": "mirror_consumer_12.properties", "mirror_producer_config_filename": "mirror_producer_12.properties" + }, + + { + "entity_id": "13", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_13.log", + "config_filename": "console_consumer_13.properties" } ] } diff --git a/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json b/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json index b9d72a1..98fefee 100644 --- a/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json @@ -57,6 +57,8 @@ "broker.id": "1", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" }, @@ -66,6 +68,8 @@ "broker.id": "2", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" }, @@ -75,6 +79,8 @@ "broker.id": "3", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" }, @@ -84,6 +90,8 @@ "broker.id": "4", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" }, @@ -93,6 +101,8 @@ "broker.id": "5", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" }, @@ -102,6 +112,8 @@ "broker.id": "6", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" }, @@ -115,6 +127,7 @@ "message": "100", "request-num-acks": "-1", "sync":"false", + "producer-num-retries":"5", "log_filename": "producer_performance_10.log", "config_filename": "producer_performance_10.properties" }, diff --git a/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json b/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json index b72f265..6067b12 100644 --- a/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json @@ -57,6 +57,8 @@ "broker.id": "1", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" }, @@ -66,6 +68,8 @@ "broker.id": "2", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" }, @@ -75,6 +79,8 @@ "broker.id": "3", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" }, @@ -84,6 +90,8 @@ "broker.id": "4", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" }, @@ -93,6 +101,8 @@ "broker.id": "5", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" }, @@ -102,6 +112,8 @@ "broker.id": "6", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" }, @@ -115,6 +127,7 @@ "message": "100", "request-num-acks": "1", "sync":"false", + "producer-num-retries":"5", "log_filename": "producer_performance_10.log", "config_filename": "producer_performance_10.properties" }, diff --git a/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json b/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json index 83ca969..58483ad 100644 --- a/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json @@ -57,6 +57,8 @@ "broker.id": "1", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" }, @@ -66,6 +68,8 @@ "broker.id": "2", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" }, @@ -75,6 +79,8 @@ "broker.id": "3", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" }, @@ -84,6 +90,8 @@ "broker.id": "4", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" }, @@ -93,6 +101,8 @@ "broker.id": "5", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" }, @@ -102,6 +112,8 @@ "broker.id": "6", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" }, @@ -115,6 +127,7 @@ "message": "100", "request-num-acks": "-1", "sync":"false", + "producer-num-retries":"5", "log_filename": "producer_performance_10.log", "config_filename": "producer_performance_10.properties" }, @@ -127,6 +140,7 @@ "message": "100", "request-num-acks": "-1", "sync":"false", + "producer-num-retries":"5", "log_filename": "producer_performance_11.log", "config_filename": "producer_performance_11.properties" }, diff --git a/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json b/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json index fe32712..1d9190c 100644 --- a/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json @@ -57,6 +57,8 @@ "broker.id": "1", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_4.log", "config_filename": "kafka_server_4.properties" }, @@ -66,6 +68,8 @@ "broker.id": "2", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_5.log", "config_filename": "kafka_server_5.properties" }, @@ -75,6 +79,8 @@ "broker.id": "3", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_6.log", "config_filename": "kafka_server_6.properties" }, @@ -84,6 +90,8 @@ "broker.id": "4", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_7.log", "config_filename": "kafka_server_7.properties" }, @@ -93,6 +101,8 @@ "broker.id": "5", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_8.log", "config_filename": "kafka_server_8.properties" }, @@ -102,6 +112,8 @@ "broker.id": "6", "log.segment.bytes": "10240", "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", "log_filename": "kafka_server_9.log", "config_filename": "kafka_server_9.properties" }, @@ -115,6 +127,7 @@ "message": "100", "request-num-acks": "1", "sync":"false", + "producer-num-retries":"5", "log_filename": "producer_performance_10.log", "config_filename": "producer_performance_10.properties" }, @@ -127,6 +140,7 @@ "message": "100", "request-num-acks": "1", "sync":"false", + "producer-num-retries":"5", "log_filename": "producer_performance_11.log", "config_filename": "producer_performance_11.properties" }, diff --git a/system_test/replication_testsuite/config/server.properties b/system_test/replication_testsuite/config/server.properties index dacf158..36dd68d 100644 --- a/system_test/replication_testsuite/config/server.properties +++ b/system_test/replication_testsuite/config/server.properties @@ -114,10 +114,10 @@ enable.zookeeper=true # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". # You can also append an optional chroot string to the urls to specify the # root directory for all kafka znodes. -zk.connect=localhost:2181 +zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 +zookeeper.connection.timeout.ms=1000000 monitoring.period.secs=1 message.max.bytes=1000000 diff --git a/system_test/replication_testsuite/replica_basic_test.py b/system_test/replication_testsuite/replica_basic_test.py index 3fc47d9..40c1157 100644 --- a/system_test/replication_testsuite/replica_basic_test.py +++ b/system_test/replication_testsuite/replica_basic_test.py @@ -77,6 +77,8 @@ class ReplicaBasicTest(ReplicationUtils, SetupUtils): self.testSuiteAbsPathName, SystemTestEnv.SYSTEM_TEST_CASE_PREFIX) testCasePathNameList.sort() + replicationUtils = ReplicationUtils(self) + # ============================================================= # launch each testcase one by one: testcase_1, testcase_2, ... # ============================================================= @@ -423,16 +425,17 @@ class ReplicaBasicTest(ReplicationUtils, SetupUtils): self.log_message("validating data matched") if logRetentionTest.lower() == "true": - kafka_system_test_utils.validate_simple_consumer_data_matched_across_replicas(self.systemTestEnv, self.testcaseEnv) - kafka_system_test_utils.validate_data_matched(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.validate_data_matched(self.systemTestEnv, self.testcaseEnv, replicationUtils) elif consumerMultiTopicsMode.lower() == "true": - #kafka_system_test_utils.validate_broker_log_segment_checksum(self.systemTestEnv, self.testcaseEnv) - kafka_system_test_utils.validate_data_matched_in_multi_topics_from_single_consumer_producer(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.validate_data_matched_in_multi_topics_from_single_consumer_producer( + self.systemTestEnv, self.testcaseEnv, replicationUtils) else: kafka_system_test_utils.validate_simple_consumer_data_matched_across_replicas(self.systemTestEnv, self.testcaseEnv) - #kafka_system_test_utils.validate_broker_log_segment_checksum(self.systemTestEnv, self.testcaseEnv) - kafka_system_test_utils.validate_data_matched(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.validate_broker_log_segment_checksum(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.validate_data_matched(self.systemTestEnv, self.testcaseEnv, replicationUtils) + kafka_system_test_utils.validate_index_log(self.systemTestEnv, self.testcaseEnv) + # ============================================= # draw graphs # ============================================= diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index 9411405..ae393bc 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -313,8 +313,8 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv logger.info("testcase config (dest) pathname : " + cfgDestPathname, extra=d) # loop through all zookeepers (if more than 1) to retrieve host and clientPort - # to construct a zk.connect str for broker in the form of: - # zk.connect=:,:,... + # to construct a zookeeper.connect str for broker in the form of: + # zookeeper.connect=:,:,... testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] = "" testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"] = "" testcaseEnv.userDefinedEnvVarDict["sourceZkEntityIdList"] = [] @@ -409,28 +409,35 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv # copy the associated .properties template, update values, write to testcase_/config - if ( clusterCfg["role"] == "broker" ): - if clusterCfg["cluster_name"] == "source": - tcCfg["zk.connect"] = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] - elif clusterCfg["cluster_name"] == "target": - tcCfg["zk.connect"] = testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"] - else: - logger.error("Unknown cluster name: " + clusterName, extra=d) - sys.exit(1) - - zeroSevenClient = "false" + if (clusterCfg["role"] == "broker"): + brokerVersion = "0.8" try: - zeroSevenClient = tcCfg["07_client"] + brokerVersion = tcCfg["version"] except: pass + if (brokerVersion == "0.7"): + if clusterCfg["cluster_name"] == "source": + tcCfg["zk.connect"] = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] + else: + logger.error("Unknown cluster name for 0.7: " + clusterName, extra=d) + sys.exit(1) + else: + if clusterCfg["cluster_name"] == "source": + tcCfg["zookeeper.connect"] = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] + elif clusterCfg["cluster_name"] == "target": + tcCfg["zookeeper.connect"] = testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"] + else: + logger.error("Unknown cluster name: " + clusterName, extra=d) + sys.exit(1) + addedCSVConfig = {} addedCSVConfig["kafka.csv.metrics.dir"] = get_testcase_config_log_dir_pathname(testcaseEnv, "broker", clusterCfg["entity_id"], "metrics") addedCSVConfig["kafka.metrics.polling.interval.secs"] = "5" addedCSVConfig["kafka.metrics.reporters"] = "kafka.metrics.KafkaCSVMetricsReporter" addedCSVConfig["kafka.csv.metrics.reporter.enabled"] = "true" - if zeroSevenClient == "true": + if brokerVersion == "0.7": addedCSVConfig["brokerid"] = tcCfg["brokerid"] copy_file_with_dict_values(cfgTemplatePathname + "/server.properties", @@ -450,12 +457,12 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv sys.exit(1) elif ( clusterCfg["role"] == "mirror_maker"): - tcCfg["broker.list"] = testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] + tcCfg["metadata.broker.list"] = testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] copy_file_with_dict_values(cfgTemplatePathname + "/mirror_producer.properties", cfgDestPathname + "/" + tcCfg["mirror_producer_config_filename"], tcCfg, None) - # update zk.connect with the zk entities specified in cluster_config.json - tcCfg["zk.connect"] = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] + # update zookeeper.connect with the zk entities specified in cluster_config.json + tcCfg["zookeeper.connect"] = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] copy_file_with_dict_values(cfgTemplatePathname + "/mirror_consumer.properties", cfgDestPathname + "/" + tcCfg["mirror_consumer_config_filename"], tcCfg, None) @@ -818,7 +825,7 @@ def start_console_consumer(systemTestEnv, testcaseEnv): if len(formatterOption) > 0: formatterOption = " --formatter " + formatterOption + " " - # get zk.connect + # get zookeeper connect string zkConnectStr = "" if clusterName == "source": zkConnectStr = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] @@ -1189,17 +1196,21 @@ def get_message_checksum(logPathName): return messageChecksumList -def validate_data_matched(systemTestEnv, testcaseEnv): +def validate_data_matched(systemTestEnv, testcaseEnv, replicationUtils): + logger.debug("#### Inside validate_data_matched", extra=d) + validationStatusDict = testcaseEnv.validationStatusDict clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList prodPerfCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "producer_performance") consumerCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "console_consumer") + consumerDuplicateCount = 0 + for prodPerfCfg in prodPerfCfgList: producerEntityId = prodPerfCfg["entity_id"] - #topic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", producerEntityId, "topic") - topic = testcaseEnv.producerTopicsString + topic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", producerEntityId, "topic") + logger.debug("working on topic : " + topic, extra=d) acks = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", producerEntityId, "request-num-acks") consumerEntityIdList = system_test_utils.get_data_from_list_of_dicts( \ @@ -1207,13 +1218,14 @@ def validate_data_matched(systemTestEnv, testcaseEnv): matchingConsumerEntityId = None for consumerEntityId in consumerEntityIdList: - #consumerTopic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", consumerEntityId, "topic") - consumerTopic = testcaseEnv.consumerTopicsString + consumerTopic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", consumerEntityId, "topic") if consumerTopic in topic: matchingConsumerEntityId = consumerEntityId + logger.debug("matching consumer entity id found", extra=d) break if matchingConsumerEntityId is None: + logger.debug("matching consumer entity id NOT found", extra=d) break msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( \ @@ -1229,10 +1241,11 @@ def validate_data_matched(systemTestEnv, testcaseEnv): producerMsgIdSet = set(producerMsgIdList) consumerMsgIdSet = set(consumerMsgIdList) - missingMsgIdInConsumer = producerMsgIdSet - consumerMsgIdSet + consumerDuplicateCount = len(consumerMsgIdList) - len(consumerMsgIdSet) + missingUniqConsumerMsgId = system_test_utils.subtract_list(producerMsgIdSet, consumerMsgIdSet) outfile = open(msgIdMissingInConsumerLogPathName, "w") - for id in missingMsgIdInConsumer: + for id in missingUniqConsumerMsgId: outfile.write(id + "\n") outfile.close() @@ -1241,20 +1254,28 @@ def validate_data_matched(systemTestEnv, testcaseEnv): validationStatusDict["Unique messages from producer on [" + topic + "]"] = str(len(producerMsgIdSet)) validationStatusDict["Unique messages from consumer on [" + topic + "]"] = str(len(consumerMsgIdSet)) - if ( len(missingMsgIdInConsumer) == 0 and len(producerMsgIdSet) > 0 ): + missingPercentage = len(missingUniqConsumerMsgId) * 100.00 / len(producerMsgIdSet) + logger.info("Data loss threshold % : " + str(replicationUtils.ackOneDataLossThresholdPercent), extra=d) + logger.warn("Data loss % on topic : " + topic + " : " + str(missingPercentage), extra=d) + + if ( len(missingUniqConsumerMsgId) == 0 and len(producerMsgIdSet) > 0 ): validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "PASSED" elif (acks == "1"): - missingPercentage = len(missingMsgIdInConsumer) * 100 / len(producerMsgIdSet) - print "#### missing Percent : ", missingPercentage - if missingPercentage <= 1: + if missingPercentage <= replicationUtils.ackOneDataLossThresholdPercent: validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "PASSED" - logger.warn("Test case passes with less than 1% data loss : [" + str(len(missingMsgIdInConsumer)) + "] missing messages", extra=d) + logger.warn("Test case (Acks = 1) passes with less than " + str(replicationUtils.ackOneDataLossThresholdPercent) \ + + "% data loss : [" + str(len(missingUniqConsumerMsgId)) + "] missing messages", extra=d) + else: + validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "FAILED" + logger.error("Test case (Acks = 1) failed with more than " + str(replicationUtils.ackOneDataLossThresholdPercent) \ + + "% data loss : [" + str(len(missingUniqConsumerMsgId)) + "] missing messages", extra=d) else: validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "FAILED" logger.info("See " + msgIdMissingInConsumerLogPathName + " for missing MessageID", extra=d) def validate_leader_election_successful(testcaseEnv, leaderDict, validationStatusDict): + logger.debug("#### Inside validate_leader_election_successful", extra=d) if ( len(leaderDict) > 0 ): try: @@ -1545,6 +1566,8 @@ def start_migration_tool(systemTestEnv, testcaseEnv, onlyThisEntityId=None): def validate_07_08_migrated_data_matched(systemTestEnv, testcaseEnv): + logger.debug("#### Inside validate_07_08_migrated_data_matched", extra=d) + validationStatusDict = testcaseEnv.validationStatusDict clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList @@ -1614,6 +1637,7 @@ def validate_07_08_migrated_data_matched(systemTestEnv, testcaseEnv): logger.info("See " + msgChecksumMissingInConsumerLogPathName + " for missing MessageID", extra=d) def validate_broker_log_segment_checksum(systemTestEnv, testcaseEnv, clusterName="source"): + logger.debug("#### Inside validate_broker_log_segment_checksum", extra=d) anonLogger.info("================================================") anonLogger.info("validating merged broker log segment checksums") @@ -1823,77 +1847,6 @@ def start_simple_consumer(systemTestEnv, testcaseEnv, minStartingOffsetDict=None partitionId += 1 replicaIndex += 1 -def validate_simple_consumer_data_matched(systemTestEnv, testcaseEnv): - validationStatusDict = testcaseEnv.validationStatusDict - clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList - - prodPerfCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "producer_performance") - consumerCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "console_consumer") - - mismatchCount = 0 - - for prodPerfCfg in prodPerfCfgList: - producerEntityId = prodPerfCfg["entity_id"] - topic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", producerEntityId, "topic") - acks = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", producerEntityId, "request-num-acks") - logger.debug("request-num-acks [" + acks + "]", extra=d) - - consumerEntityIdList = system_test_utils.get_data_from_list_of_dicts( \ - clusterEntityConfigDictList, "role", "console_consumer", "entity_id") - - matchingConsumerEntityId = None - for consumerEntityId in consumerEntityIdList: - consumerTopic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", consumerEntityId, "topic") - if consumerTopic in topic: - matchingConsumerEntityId = consumerEntityId - break - - if matchingConsumerEntityId is None: - break - - producerLogPath = get_testcase_config_log_dir_pathname(testcaseEnv, "producer_performance", producerEntityId, "default") - producerLogPathName = producerLogPath + "/producer_performance.log" - producerMsgIdList = get_message_id(producerLogPathName) - producerMsgIdSet = set(producerMsgIdList) - logger.info("no. of unique messages on topic [" + topic + "] sent from publisher : " + str(len(producerMsgIdSet)), extra=d) - validationStatusDict["Unique messages from producer on [" + topic + "]"] = str(len(producerMsgIdSet)) - - consumerLogPath = get_testcase_config_log_dir_pathname(testcaseEnv, "console_consumer", matchingConsumerEntityId, "default") - for logFile in sorted(os.listdir(consumerLogPath)): - # only process log file: *.log - if logFile.endswith(".log"): - consumerLogPathName = consumerLogPath + "/" + logFile - consumerMsgIdList = get_message_id(consumerLogPathName) - consumerMsgIdSet = set(consumerMsgIdList) - missingMsgIdInConsumer = producerMsgIdSet - consumerMsgIdSet - msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( - testcaseEnv, "console_consumer", matchingConsumerEntityId, "default") + \ - "/" + logFile + "_msg_id_missing_in_consumer.log" - - outfile = open(msgIdMissingInConsumerLogPathName, "w") - for id in missingMsgIdInConsumer: - outfile.write(id + "\n") - outfile.close() - - logger.info("no. of unique messages on topic [" + topic + "] at " + logFile + " : " + str(len(consumerMsgIdSet)), extra=d) - validationStatusDict["Unique messages from consumer on [" + topic + "] at " + logFile] = str(len(consumerMsgIdSet)) - - if acks == "-1" and len(missingMsgIdInConsumer) > 0: - mismatchCount += 1 - elif acks == "1" and len(missingMsgIdInConsumer) > 0: - missingPercentage = len(missingMsgIdInConsumer) * 100 / len(producerMsgIdSet) - logger.debug("missing percentage [" + str(missingPercentage) + "]", extra=d) - if missingPercentage <= 1: - logger.warn("Test case (acks == 1) passes with < 1% data loss : [" + \ - str(len(missingMsgIdInConsumer)) + "] missing messages", extra=d) - else: - mismatchCount += 1 - - if mismatchCount == 0: - validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "PASSED" - else: - validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "FAILED" - def get_controller_attributes(systemTestEnv, testcaseEnv): logger.info("Querying Zookeeper for Controller info ...", extra=d) @@ -1917,7 +1870,7 @@ def get_controller_attributes(systemTestEnv, testcaseEnv): "\"JAVA_HOME=" + javaHome, kafkaRunClassBin + " org.apache.zookeeper.ZooKeeperMain", "-server " + testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"], - "'get /controller' 2> /dev/null | tail -1\""] + "get /controller 2> /dev/null | tail -1\""] cmdStr = " ".join(cmdStrList) logger.debug("executing command [" + cmdStr + "]", extra=d) @@ -2007,6 +1960,8 @@ def getMinCommonStartingOffset(systemTestEnv, testcaseEnv, clusterName="source") return minCommonStartOffsetDict def validate_simple_consumer_data_matched_across_replicas(systemTestEnv, testcaseEnv): + logger.debug("#### Inside validate_simple_consumer_data_matched_across_replicas", extra=d) + validationStatusDict = testcaseEnv.validationStatusDict clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList consumerEntityIdList = system_test_utils.get_data_from_list_of_dicts( @@ -2014,101 +1969,100 @@ def validate_simple_consumer_data_matched_across_replicas(systemTestEnv, testcas replicaFactor = testcaseEnv.testcaseArgumentsDict["replica_factor"] numPartition = testcaseEnv.testcaseArgumentsDict["num_partition"] - # Unique messages from producer on [test_1] : 1500 - # Unique messages from consumer on [test_1] : 1500 + for consumerEntityId in consumerEntityIdList: - # Unique messages from consumer on [test_1] at simple_consumer_test_1-0_r1.log : 750 - # Unique messages from consumer on [test_1] at simple_consumer_test_1-0_r2.log : 750 - # Unique messages from consumer on [test_1] at simple_consumer_test_1-0_r3.log : 0 + # get topic string from multi consumer "entity" + topicStr = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", consumerEntityId, "topic") - # Unique messages from consumer on [test_1] at simple_consumer_test_1-1_r1.log : 0 - # Unique messages from consumer on [test_1] at simple_consumer_test_1-1_r2.log : 750 - # Unique messages from consumer on [test_1] at simple_consumer_test_1-1_r3.log : 750 + # the topic string could be multi topics separated by ',' + topicList = topicStr.split(',') - # ================================================== + for topic in topicList: + logger.debug("working on topic : " + topic, extra=d) + consumerLogPath = get_testcase_config_log_dir_pathname(testcaseEnv, "console_consumer", consumerEntityId, "default") + + # keep track of total msg count across replicas for each topic-partition + # (should be greater than 0 for passing) + totalMsgCounter = 0 + + # keep track of the mismatch msg count for each topic-partition + # (should be equal to 0 for passing) + mismatchCounter = 0 + + replicaIdxMsgIdList = [] + # replicaIdxMsgIdList : + # - This is a list of dictionaries of topic-partition (key) + # mapping to list of MessageID in that topic-partition (val) + # - The list index is mapped to (replicaId - 1) + # [ + # // list index = 0 => replicaId = idx(0) + 1 = 1 + # { + # "topic1-0" : [ "0000000001", "0000000002", "0000000003"], + # "topic1-1" : [ "0000000004", "0000000005", "0000000006"] + # }, + # // list index = 1 => replicaId = idx(1) + 1 = 2 + # { + # "topic1-0" : [ "0000000001", "0000000002", "0000000003"], + # "topic1-1" : [ "0000000004", "0000000005", "0000000006"] + # } + # ] + + # initialize replicaIdxMsgIdList + j = 0 + while j < int(replicaFactor): + newDict = {} + replicaIdxMsgIdList.append(newDict) + j += 1 + + # retrieve MessageID from all simple consumer log4j files + for logFile in sorted(os.listdir(consumerLogPath)): + + if logFile.startswith("simple_consumer_"+topic) and logFile.endswith(".log"): + logger.debug("working on file : " + logFile, extra=d) + matchObj = re.match("simple_consumer_"+topic+"-(\d*)_r(\d*)\.log" , logFile) + partitionId = int(matchObj.group(1)) + replicaIdx = int(matchObj.group(2)) + + consumerLogPathName = consumerLogPath + "/" + logFile + consumerMsgIdList = get_message_id(consumerLogPathName) - # Unique messages from producer on [test_2] : 1000 - # Unique messages from consumer on [test_2] : 1000 + topicPartition = topic + "-" + str(partitionId) + replicaIdxMsgIdList[replicaIdx - 1][topicPartition] = consumerMsgIdList - # Unique messages from consumer on [test_2] at simple_consumer_test_2-0_r1.log : 500 - # Unique messages from consumer on [test_2] at simple_consumer_test_2-0_r2.log : 0 - # Unique messages from consumer on [test_2] at simple_consumer_test_2-0_r3.log : 500 + logger.info("no. of messages on topic [" + topic + "] at " + logFile + " : " + str(len(consumerMsgIdList)), extra=d) + validationStatusDict["No. of messages from consumer on [" + topic + "] at " + logFile] = str(len(consumerMsgIdList)) - # Unique messages from consumer on [test_2] at simple_consumer_test_2-1_r1.log : 500 - # Unique messages from consumer on [test_2] at simple_consumer_test_2-1_r2.log : 500 - # Unique messages from consumer on [test_2] at simple_consumer_test_2-1_r3.log : 0 + # print replicaIdxMsgIdList - mismatchCounter = 0 - for consumerEntityId in consumerEntityIdList: + # take the first dictionary of replicaIdxMsgIdList and compare with the rest + firstMsgIdDict = replicaIdxMsgIdList[0] - topic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", consumerEntityId, "topic") - consumerLogPath = get_testcase_config_log_dir_pathname(testcaseEnv, "console_consumer", consumerEntityId, "default") - - replicaIdxMsgCountDictList = [] - # replicaIdxMsgCountDictList is being used as follows: - # - # the above replica message count will be organized as follows: - # index of the list would map to the partitionId - # each element in the list maps to the replicaIdx-MessageCount - # to validate that : - # 1. there should be "no. of broker" of non-zero message count and they are equal - # 2. there should be "no. of broker - replication factor" of zero count - # [{"1": "750", "2": "750", "3": "0" }, - # {"1": "0" , "2": "750", "3": "750"}] - - j = 0 - while j < int(numPartition): - newDict = {} - replicaIdxMsgCountDictList.append(newDict) - j += 1 - - for logFile in sorted(os.listdir(consumerLogPath)): - - if logFile.startswith("simple_consumer_") and logFile.endswith(".log"): - matchObj = re.match("simple_consumer_"+topic+"-(\d*)_r(\d*)\.log" , logFile) - partitionId = int(matchObj.group(1)) - replicaIdx = int(matchObj.group(2)) - - consumerLogPathName = consumerLogPath + "/" + logFile - consumerMsgIdList = get_message_id(consumerLogPathName) - consumerMsgIdSet = set(consumerMsgIdList) - - replicaIdxMsgCountDictList[partitionId][replicaIdx] = len(consumerMsgIdSet) - - logger.info("no. of unique messages on topic [" + topic + "] at " + logFile + " : " + str(len(consumerMsgIdSet)), extra=d) - validationStatusDict["Unique messages from consumer on [" + topic + "] at " + logFile] = str(len(consumerMsgIdSet)) - - pprint.pprint(replicaIdxMsgCountDictList) - - partitionId = 0 - while partitionId < int(numPartition): - zeroMsgCounter = 0 - nonZeroMsgCounter = 0 - nonZeroMsgValue = -1 - - for replicaIdx in sorted(replicaIdxMsgCountDictList[partitionId].iterkeys()): - if replicaIdxMsgCountDictList[partitionId][int(replicaIdx)] == 0: - zeroMsgCounter += 1 - else: - if nonZeroMsgValue == -1: - nonZeroMsgValue = replicaIdxMsgCountDictList[partitionId][int(replicaIdx)] - else: - if nonZeroMsgValue != replicaIdxMsgCountDictList[partitionId][int(replicaIdx)]: - mismatchCounter += 1 - nonZeroMsgCounter += 1 - partitionId += 1 + # loop through all 'topic-partition' such as topic1-0, topic1-1, ... + for topicPartition in sorted(firstMsgIdDict.iterkeys()): - logger.info("topic " + topic + " : no. of brokers with zero msg count : " + str(zeroMsgCounter), extra=d) - logger.info("topic " + topic + " : no. of brokers with non-zero msg count : " + str(nonZeroMsgCounter), extra=d) - logger.info("topic " + topic + " : non-zero brokers msg count : " + str(nonZeroMsgValue), extra=d) + # compare all replicas' MessageID in corresponding topic-partition + for i in range(len(replicaIdxMsgIdList)): + # skip the first dictionary + if i == 0: + totalMsgCounter += len(firstMsgIdDict[topicPartition]) + continue - if mismatchCounter == 0 and nonZeroMsgCounter > 0: - validationStatusDict["Validate for data matched on topic [" + topic + "] across replicas"] = "PASSED" - else: - validationStatusDict["Validate for data matched on topic [" + topic + "] across replicas"] = "FAILED" + totalMsgCounter += len(replicaIdxMsgIdList[i][topicPartition]) + + # get the count of mismatch MessageID between first MessageID list and the other lists + diffCount = system_test_utils.diff_lists(firstMsgIdDict[topicPartition], replicaIdxMsgIdList[i][topicPartition]) + mismatchCounter += diffCount + logger.info("Mismatch count of topic-partition [" + topicPartition + "] in replica id [" + str(i+1) + "] : " + str(diffCount), extra=d) + + if mismatchCounter == 0 and totalMsgCounter > 0: + validationStatusDict["Validate for data matched on topic [" + topic + "] across replicas"] = "PASSED" + else: + validationStatusDict["Validate for data matched on topic [" + topic + "] across replicas"] = "FAILED" -def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTestEnv, testcaseEnv): +def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTestEnv, testcaseEnv, replicationUtils): + logger.debug("#### Inside validate_data_matched_in_multi_topics_from_single_consumer_producer", extra=d) + validationStatusDict = testcaseEnv.validationStatusDict clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList @@ -2140,6 +2094,7 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe topicList = topicStr.split(',') for topic in topicList: + consumerDuplicateCount = 0 msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( testcaseEnv, "console_consumer", matchingConsumerEntityId, "default") \ + "/msg_id_missing_in_consumer_" + topic + ".log" @@ -2148,10 +2103,11 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe producerMsgIdSet = set(producerMsgIdList) consumerMsgIdSet = set(consumerMsgIdList) - missingMsgIdInConsumer = producerMsgIdSet - consumerMsgIdSet + consumerDuplicateCount = len(consumerMsgIdList) -len(consumerMsgIdSet) + missingUniqConsumerMsgId = system_test_utils.subtract_list(producerMsgIdSet, consumerMsgIdSet) outfile = open(msgIdMissingInConsumerLogPathName, "w") - for id in missingMsgIdInConsumer: + for id in missingUniqConsumerMsgId: outfile.write(id + "\n") outfile.close() @@ -2160,17 +2116,98 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe validationStatusDict["Unique messages from producer on [" + topic + "]"] = str(len(producerMsgIdSet)) validationStatusDict["Unique messages from consumer on [" + topic + "]"] = str(len(consumerMsgIdSet)) - if ( len(missingMsgIdInConsumer) == 0 and len(producerMsgIdSet) > 0 ): + missingPercentage = len(missingUniqConsumerMsgId) * 100.00 / len(producerMsgIdSet) + logger.info("Data loss threshold % : " + str(replicationUtils.ackOneDataLossThresholdPercent), extra=d) + logger.warn("Data loss % on topic : " + topic + " : " + str(missingPercentage), extra=d) + + if ( len(missingUniqConsumerMsgId) == 0 and len(producerMsgIdSet) > 0 ): validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "PASSED" elif (acks == "1"): - missingPercentage = len(missingMsgIdInConsumer) * 100 / len(producerMsgIdSet) - print "#### missing Percent : ", missingPercentage - if missingPercentage <= 1: + if missingPercentage <= replicationUtils.ackOneDataLossThresholdPercent: validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "PASSED" - logger.warn("Test case passes with less than 1% data loss : [" + str(len(missingMsgIdInConsumer)) + "] missing messages", extra=d) + logger.warn("Test case (Acks = 1) passes with less than " + str(replicationUtils.ackOneDataLossThresholdPercent) \ + + "% data loss : [" + str(len(missingUniqConsumerMsgId)) + "] missing messages", extra=d) + else: + validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "FAILED" + logger.error("Test case (Acks = 1) failed with more than " + str(replicationUtils.ackOneDataLossThresholdPercent) \ + + "% data loss : [" + str(len(missingUniqConsumerMsgId)) + "] missing messages", extra=d) else: validationStatusDict["Validate for data matched on topic [" + topic + "]"] = "FAILED" logger.info("See " + msgIdMissingInConsumerLogPathName + " for missing MessageID", extra=d) +def validate_index_log(systemTestEnv, testcaseEnv, clusterName="source"): + logger.debug("#### Inside validate_index_log", extra=d) + + failureCount = 0 + brokerLogCksumDict = {} + testCaseBaseDir = testcaseEnv.testCaseBaseDir + tcConfigsList = testcaseEnv.testcaseConfigsList + validationStatusDict = testcaseEnv.validationStatusDict + clusterConfigList = systemTestEnv.clusterEntityConfigDictList + allBrokerConfigList = system_test_utils.get_dict_from_list_of_dicts(clusterConfigList, "role", "broker") + brokerEntityIdList = system_test_utils.get_data_from_list_of_dicts(allBrokerConfigList, "cluster_name", clusterName, "entity_id") + + # loop through all brokers + for brokerEntityId in brokerEntityIdList: + logCksumDict = {} + # remoteLogSegmentPathName : /tmp/kafka_server_4_logs + # => remoteLogSegmentDir : kafka_server_4_logs + remoteLogSegmentPathName = system_test_utils.get_data_by_lookup_keyval(tcConfigsList, "entity_id", brokerEntityId, "log.dir") + remoteLogSegmentDir = os.path.basename(remoteLogSegmentPathName) + logPathName = get_testcase_config_log_dir_pathname(testcaseEnv, "broker", brokerEntityId, "default") + localLogSegmentPath = logPathName + "/" + remoteLogSegmentDir + kafkaHome = system_test_utils.get_data_by_lookup_keyval(clusterConfigList, "entity_id", brokerEntityId, "kafka_home") + hostname = system_test_utils.get_data_by_lookup_keyval(clusterConfigList, "entity_id", brokerEntityId, "hostname") + kafkaRunClassBin = kafkaHome + "/bin/kafka-run-class.sh" + + # localLogSegmentPath : + # .../system_test/mirror_maker_testsuite/testcase_5002/logs/broker-4/kafka_server_4_logs + # |- test_1-0 + # |- 00000000000000000000.index + # |- 00000000000000000000.log + # |- 00000000000000000020.index + # |- 00000000000000000020.log + # |- . . . + # |- test_1-1 + # |- 00000000000000000000.index + # |- 00000000000000000000.log + # |- 00000000000000000020.index + # |- 00000000000000000020.log + # |- . . . + + # loop through all topicPartition directories such as : test_1-0, test_1-1, ... + for topicPartition in os.listdir(localLogSegmentPath): + # found a topic-partition directory + if os.path.isdir(localLogSegmentPath + "/" + topicPartition): + + # log segment files are located in : localLogSegmentPath + "/" + topicPartition + # sort the log segment files under each topic-partition and verify index + for logFile in sorted(os.listdir(localLogSegmentPath + "/" + topicPartition)): + # only process index file: *.index + if logFile.endswith(".index"): + offsetLogSegmentPathName = localLogSegmentPath + "/" + topicPartition + "/" + logFile + cmdStrList = ["ssh " + hostname, + kafkaRunClassBin + " kafka.tools.DumpLogSegments", + " --file " + offsetLogSegmentPathName, + "--verify-index-only 2>&1"] + cmdStr = " ".join(cmdStrList) + + showMismatchedIndexOffset = False + + logger.debug("executing command [" + cmdStr + "]", extra=d) + subproc = system_test_utils.sys_call_return_subproc(cmdStr) + for line in subproc.stdout.readlines(): + line = line.rstrip('\n') + if showMismatchedIndexOffset: + logger.debug("#### [" + line + "]", extra=d) + elif "Mismatches in :" in line: + logger.debug("#### error found [" + line + "]", extra=d) + failureCount += 1 + showMismatchedIndexOffset = True + + if failureCount == 0: + validationStatusDict["Validate index log in cluster [" + clusterName + "]"] = "PASSED" + else: + validationStatusDict["Validate index log in cluster [" + clusterName + "]"] = "FAILED" diff --git a/system_test/utils/replication_utils.py b/system_test/utils/replication_utils.py index 3e8efad..cfd80b2 100644 --- a/system_test/utils/replication_utils.py +++ b/system_test/utils/replication_utils.py @@ -65,3 +65,6 @@ class ReplicationUtils(object): self.controllerAttributesDict["REGX_CONTROLLER_STARTUP_PATTERN"] = "\[(.*?)\] .* \[Controller (.*?)\]: " + \ self.controllerAttributesDict["CONTROLLER_STARTUP_COMPLETE_MSG"] + # Data Loss Percentage Threshold in Ack = 1 cases + self.ackOneDataLossThresholdPercent = 5.0 + diff --git a/system_test/utils/system_test_utils.py b/system_test/utils/system_test_utils.py index 65db5c5..50340f0 100644 --- a/system_test/utils/system_test_utils.py +++ b/system_test/utils/system_test_utils.py @@ -21,6 +21,7 @@ # =================================== import copy +import difflib import inspect import json import logging @@ -554,5 +555,80 @@ def setup_remote_hosts_with_testsuite_level_cluster_config(systemTestEnv, testMo sys.exit(1) print - +# ================================================= +# lists_diff_count +# - find the no. of different items in both lists +# - both lists need not be sorted +# - input lists won't be changed +# ================================================= +def lists_diff_count(a, b): + c = list(b) + d = [] + for item in a: + try: + c.remove(item) + except: + d.append(item) + + if len(d) > 0: + print "#### Mismatch MessageID" + print d + + return len(c) + len(d) + +# ================================================= +# subtract_list +# - subtract items in listToSubtract from mainList +# and return the resulting list +# - both lists need not be sorted +# - input lists won't be changed +# ================================================= +def subtract_list(mainList, listToSubtract): + remainingList = list(mainList) + for item in listToSubtract: + try: + remainingList.remove(item) + except: + pass + return remainingList + +# ================================================= +# diff_lists +# - find the diff of 2 lists and return the +# total no. of mismatch from both lists +# - diff of both lists includes: +# - no. of items mismatch +# - ordering of the items +# +# sample lists: +# a = ['8','4','3','2','1'] +# b = ['8','3','4','2','1'] +# +# difflib will return the following: +# 8 +# + 3 +# 4 +# - 3 +# 2 +# 1 +# +# diff_lists(a,b) returns 2 and prints the following: +# #### only in seq 2 : + 3 +# #### only in seq 1 : - 3 +# ================================================= +def diff_lists(a, b): + mismatchCount = 0 + d = difflib.Differ() + diff = d.compare(a,b) + + for item in diff: + result = item[0:1].strip() + if len(result) > 0: + mismatchCount += 1 + if '-' in result: + logger.debug("#### only in seq 1 : " + item, extra=d) + elif '+' in result: + logger.debug("#### only in seq 2 : " + item, extra=d) + + return mismatchCount