diff --git a/README.md b/README.md index cf0d59e..9879427 100644 --- a/README.md +++ b/README.md @@ -1,59 +1,52 @@ -# Kafka is a distributed publish/subscribe messaging system # +# Apache Kafka # -It is designed to support the following +See our [web site](http://kafka.apache.org) details on the project. -* Persistent messaging with O(1) disk structures that provide constant time performance even with many TB of stored messages. -* High-throughput: even with very modest hardware Kafka can support hundreds of thousands of messages per second. -* Explicit support for partitioning messages over Kafka servers and distributing consumption over a cluster of consumer machines while maintaining per-partition ordering semantics. -* Support for parallel data load into Hadoop. - -Kafka is aimed at providing a publish-subscribe solution that can handle all activity stream data and processing on a consumer-scale web site. This kind of activity (page views, searches, and other user actions) are a key ingredient in many of the social feature on the modern web. This data is typically handled by "logging" and ad hoc log aggregation solutions due to the throughput requirements. This kind of ad hoc solution is a viable solution to providing logging data to an offline analysis system like Hadoop, but is very limiting for building real-time processing. Kafka aims to unify offline and online processing by providing a mechanism for parallel load into Hadoop as well as the ability to partition real-time consumption over a cluster of machines. - -See our [web site](http://kafka.apache.org/) for more details on the project. - -## Contribution ## - -Kafka is a new project, and we are interested in building the community; we would welcome any thoughts or [patches](https://issues.apache.org/jira/browse/KAFKA). You can reach us [on the Apache mailing lists](http://kafka.apache.org/contact.html). - -The Kafka code is available from: - * git clone http://git-wip-us.apache.org/repos/asf/kafka.git kafka - -To contribute you can follow: - * https://cwiki.apache.org/confluence/display/KAFKA/Git+Workflow +## Building It ## To build for all supported versions of Scala: - 1. ./sbt +package To build for a particular version of Scala (either 2.8.0, 2.8.2, 2.9.1 or 2.9.2): 1. ./sbt "++2.8.0 package" *or* ./sbt "++2.8.2 package" *or* ./sbt "++2.9.1 package" *or* ./sbt "++2.9.2 package" +## Running It ## + +To run follow the instructions here: +1. http://kafka.apache.org/08/quickstart.html + +## Other Build Tips ## + Here are some useful sbt commands, to be executed at the sbt command prompt (./sbt). Prefixing with "++ " runs the command for a specific Scala version, prefixing with "+" will perform the action for all versions of Scala, and no prefix runs the command for the default (2.8.0) version of Scala. - tasks : Lists all the sbt commands and their descriptions - clean : Deletes all generated files (the target directory). - compile : Compile all the sub projects, but not create the jars - test : Run all unit tests in all sub projects - release-zip : Create all the jars, run unit tests and create a deployable release zip - package: Creates jars for src, test, docs etc - projects : List all the sub projects - project sub_project_name : Switch to a particular sub-project. For example, to switch to the core kafka code, use "project core-kafka" -Following commands can be run only on a particular sub project - - +The following commands can be run only on a particular sub project - test-only package.test.TestName : Runs only the specified test in the current sub project - run : Provides options to run any of the classes that have a main method. For example, you can switch to project java-examples, and run the examples there by executing "project java-examples" followed by "run" For more details please see the [SBT documentation](https://github.com/harrah/xsbt/wiki) +## Contribution ## + +Kafka is a new project, and we are interested in building the community; we would welcome any thoughts or [patches](https://issues.apache.org/jira/browse/KAFKA). You can reach us [on the Apache mailing lists](http://kafka.apache.org/contact.html). + +To contribute follow the instructions here: + * http://kafka.apache.org/contributing.html + +We also welcome patches for the website and documentation which can be found here: + * https://svn.apache.org/repos/asf/kafka/site + + + + diff --git a/bin/kafka-add-partitions.sh b/bin/kafka-add-partitions.sh new file mode 100755 index 0000000..c08b837 --- /dev/null +++ b/bin/kafka-add-partitions.sh @@ -0,0 +1,18 @@ +#!/bin/bash +# 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. + +base_dir=$(dirname $0) +$base_dir/kafka-run-class.sh kafka.admin.AddPartitionsCommand $@ diff --git a/bin/kafka-check-reassignment-status.sh b/bin/kafka-check-reassignment-status.sh index 4eb9aca..1f21858 100755 --- a/bin/kafka-check-reassignment-status.sh +++ b/bin/kafka-check-reassignment-status.sh @@ -14,5 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -base_dir=$(dirname $0) -$base_dir/kafka-run-class.sh kafka.admin.CheckReassignmentStatus $@ +$(dirname $0)/kafka-run-class.sh kafka.admin.CheckReassignmentStatus $@ diff --git a/bin/kafka-console-consumer-log4j.properties b/bin/kafka-console-consumer-log4j.properties deleted file mode 100644 index 6b76444..0000000 --- a/bin/kafka-console-consumer-log4j.properties +++ /dev/null @@ -1,21 +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. -log4j.rootLogger=INFO, stderr - -log4j.appender.stderr=org.apache.log4j.ConsoleAppender -log4j.appender.stderr.target=System.err -log4j.appender.stderr.layout=org.apache.log4j.PatternLayout -log4j.appender.stderr.layout.ConversionPattern=[%d] %p %m (%c)%n - diff --git a/bin/kafka-console-consumer.sh b/bin/kafka-console-consumer.sh index 8a76c6c..95edc9e 100755 --- a/bin/kafka-console-consumer.sh +++ b/bin/kafka-console-consumer.sh @@ -14,6 +14,5 @@ # See the License for the specific language governing permissions and # limitations under the License. -base_dir=$(dirname $0) -export KAFKA_OPTS="-Xmx512M -server -Dcom.sun.management.jmxremote -Dlog4j.configuration=file:$base_dir/kafka-console-consumer-log4j.properties" -$base_dir/kafka-run-class.sh kafka.consumer.ConsoleConsumer $@ +export KAFKA_HEAP_OPTS="-Xmx512M" +$(dirname $0)/kafka-run-class.sh kafka.consumer.ConsoleConsumer $@ diff --git a/bin/kafka-console-producer.sh b/bin/kafka-console-producer.sh index 99d2dc6..f102c22 100755 --- a/bin/kafka-console-producer.sh +++ b/bin/kafka-console-producer.sh @@ -14,5 +14,5 @@ # See the License for the specific language governing permissions and # limitations under the License. -base_dir=$(dirname $0) -$base_dir/kafka-run-class.sh kafka.producer.ConsoleProducer $@ +export KAFKA_HEAP_OPTS="-Xmx512M" +$(dirname $0)/kafka-run-class.sh kafka.producer.ConsoleProducer $@ diff --git a/bin/kafka-consumer-perf-test.sh b/bin/kafka-consumer-perf-test.sh index 7745ea8..39434d1 100755 --- a/bin/kafka-consumer-perf-test.sh +++ b/bin/kafka-consumer-perf-test.sh @@ -14,4 +14,5 @@ # See the License for the specific language governing permissions and # limitations under the License. +export KAFKA_HEAP_OPTS="-Xmx512M" $(dirname $0)/kafka-run-class.sh kafka.perf.ConsumerPerformance $@ diff --git a/bin/kafka-preferred-replica-election.sh b/bin/kafka-preferred-replica-election.sh index 9c63e43..6416dc1 100755 --- a/bin/kafka-preferred-replica-election.sh +++ b/bin/kafka-preferred-replica-election.sh @@ -14,5 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -base_dir=$(dirname $0) -$base_dir/kafka-run-class.sh kafka.admin.PreferredReplicaLeaderElectionCommand $@ +$(dirname $0)/kafka-run-class.sh kafka.admin.PreferredReplicaLeaderElectionCommand $@ diff --git a/bin/kafka-producer-perf-test.sh b/bin/kafka-producer-perf-test.sh index 25c4dce..1935522 100755 --- a/bin/kafka-producer-perf-test.sh +++ b/bin/kafka-producer-perf-test.sh @@ -14,4 +14,5 @@ # See the License for the specific language governing permissions and # limitations under the License. +export KAFKA_HEAP_OPTS="-Xmx512M" $(dirname $0)/kafka-run-class.sh kafka.perf.ProducerPerformance $@ diff --git a/bin/kafka-reassign-partitions.sh b/bin/kafka-reassign-partitions.sh index 162872c..a6bd148 100755 --- a/bin/kafka-reassign-partitions.sh +++ b/bin/kafka-reassign-partitions.sh @@ -14,5 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -base_dir=$(dirname $0) -$base_dir/kafka-run-class.sh kafka.admin.ReassignPartitionsCommand $@ +$(dirname $0)/kafka-run-class.sh kafka.admin.ReassignPartitionsCommand $@ diff --git a/bin/kafka-replay-log-producer.sh b/bin/kafka-replay-log-producer.sh index f36e0e2..86943ee 100755 --- a/bin/kafka-replay-log-producer.sh +++ b/bin/kafka-replay-log-producer.sh @@ -14,6 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -base_dir=$(dirname $0) -export KAFKA_OPTS="-Xmx512M -server -Dcom.sun.management.jmxremote -Dlog4j.configuration=file:$base_dir/../config/log4j.properties" -$base_dir/kafka-run-class.sh kafka.tools.ReplayLogProducer $@ +$(dirname $0)/kafka-run-class.sh kafka.tools.ReplayLogProducer $@ diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 1b66655..eb6ff1b 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -22,10 +22,18 @@ fi base_dir=$(dirname $0)/.. -SCALA_VERSION=2.8.0 +# create logs directory +LOG_DIR=$base_dir/logs +if [ ! -d $LOG_DIR ]; then + mkdir $LOG_DIR +fi + +if [ -z "$SCALA_VERSION" ]; then + SCALA_VERSION=2.8.0 +fi # 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; +for file in $base_dir/core/target/scala-${SCALA_VERSION}/*.jar; do CLASSPATH=$CLASSPATH:$file done @@ -46,30 +54,59 @@ do CLASSPATH=$CLASSPATH:$file done +# JMX settings if [ -z "$KAFKA_JMX_OPTS" ]; then KAFKA_JMX_OPTS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " fi -if [ -z "$KAFKA_OPTS" ]; then - KAFKA_OPTS="-Xmx512M -server -Dlog4j.configuration=file:$base_dir/config/log4j.properties" -fi - +# JMX port to use if [ $JMX_PORT ]; then KAFKA_JMX_OPTS="$KAFKA_JMX_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT " fi +# Log4j settings +if [ -z "$KAFKA_LOG4J_OPTS" ]; then + KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/config/tools-log4j.properties" +fi + +# Generic jvm settings you want to add +if [ -z "$KAFKA_OPTS" ]; then + KAFKA_OPTS="" +fi + +# Which java to use if [ -z "$JAVA_HOME" ]; then JAVA="java" else JAVA="$JAVA_HOME/bin/java" fi -$JAVA $KAFKA_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH "$@" +# Memory options +if [ -z "$KAFKA_HEAP_OPTS" ]; then + KAFKA_HEAP_OPTS="-Xmx256M" +fi + +# JVM performance options +if [ -z "$KAFKA_JVM_PERFORMANCE_OPTS" ]; then + KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseCompressedOops -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC" +fi + +# GC options +GC_FILE_SUFFIX='-gc.log' +GC_LOG_FILE_NAME='' +if [ "$1" = "daemon" ] && [ -z "$KAFKA_GC_LOG_OPTS"] ; then + shift + GC_LOG_FILE_NAME=$1$GC_FILE_SUFFIX + shift + KAFKA_GC_LOG_OPTS="-Xloggc:$LOG_DIR/$GC_LOG_FILE_NAME -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps " +fi + +$JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_OPTS -cp $CLASSPATH $KAFKA_OPTS "$@" exitval=$? if [ $exitval -eq "1" ] ; then - $JAVA $KAFKA_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH "$@" >& exception.txt + $JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_OPTS -cp $CLASSPATH $KAFKA_OPTS "$@" >& exception.txt exception=`cat exception.txt` noBuildMessage='Please build the project using sbt. Documentation is available at http://kafka.apache.org/' pattern="(Could not find or load main class)|(java\.lang\.NoClassDefFoundError)" diff --git a/bin/kafka-server-start.sh b/bin/kafka-server-start.sh index 307f76c..4005289 100755 --- a/bin/kafka-server-start.sh +++ b/bin/kafka-server-start.sh @@ -19,7 +19,7 @@ then echo "USAGE: $0 server.properties" exit 1 fi - -export JMX_PORT=${JMX_PORT:-9999} - -$(dirname $0)/kafka-run-class.sh kafka.Kafka $@ +base_dir=$(dirname $0) +export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/log4j.properties" +export KAFKA_HEAP_OPTS="-Xmx1G -Xms1G" +$base_dir/kafka-run-class.sh daemon kafkaServer kafka.Kafka $@ diff --git a/bin/kafka-server-stop.sh b/bin/kafka-server-stop.sh index 516c6ad..35a26a6 100755 --- a/bin/kafka-server-stop.sh +++ b/bin/kafka-server-stop.sh @@ -13,4 +13,4 @@ # 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. -ps ax | grep -i 'kafka.Kafka' | grep -v grep | awk '{print $1}' | xargs kill -SIGTERM +ps ax | grep -i 'kafka\.Kafka' | grep java | grep -v grep | awk '{print $1}' | xargs kill -SIGINT diff --git a/bin/kafka-simple-consumer-perf-test.sh b/bin/kafka-simple-consumer-perf-test.sh index a9246cb..94c9ed0 100755 --- a/bin/kafka-simple-consumer-perf-test.sh +++ b/bin/kafka-simple-consumer-perf-test.sh @@ -14,4 +14,5 @@ # See the License for the specific language governing permissions and # limitations under the License. +export KAFKA_HEAP_OPTS="-Xmx512M" $(dirname $0)/kafka-run-class.sh kafka.perf.SimpleConsumerPerformance $@ diff --git a/bin/kafka-topics.sh b/bin/kafka-topics.sh index b3195ee..cb76590 100755 --- a/bin/kafka-topics.sh +++ b/bin/kafka-topics.sh @@ -14,6 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -base_dir=$(dirname $0) -export KAFKA_OPTS="-Xmx512M -server -Dcom.sun.management.jmxremote -Dlog4j.configuration=file:$base_dir/kafka-console-consumer-log4j.properties" -$base_dir/kafka-run-class.sh kafka.admin.TopicCommand $@ +$(dirname $0)/kafka-run-class.sh kafka.admin.TopicCommand $@ diff --git a/bin/zookeeper-server-start.sh b/bin/zookeeper-server-start.sh index 184a10b..e4746a8 100755 --- a/bin/zookeeper-server-start.sh +++ b/bin/zookeeper-server-start.sh @@ -19,5 +19,8 @@ then echo "USAGE: $0 zookeeper.properties" exit 1 fi +base_dir=$(dirname $0) +export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/log4j.properties" +export KAFKA_HEAP_OPTS="-Xmx512M -Xms512M" +$base_dir/kafka-run-class.sh daemon zookeeper org.apache.zookeeper.server.quorum.QuorumPeerMain $@ -$(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@ diff --git a/config/log4j.properties b/config/log4j.properties index c611786..782124d 100644 --- a/config/log4j.properties +++ b/config/log4j.properties @@ -20,19 +20,19 @@ log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n log4j.appender.kafkaAppender=org.apache.log4j.DailyRollingFileAppender log4j.appender.kafkaAppender.DatePattern='.'yyyy-MM-dd-HH -log4j.appender.kafkaAppender.File=server.log +log4j.appender.kafkaAppender.File=logs/server.log log4j.appender.kafkaAppender.layout=org.apache.log4j.PatternLayout log4j.appender.kafkaAppender.layout.ConversionPattern=[%d] %p %m (%c)%n log4j.appender.stateChangeAppender=org.apache.log4j.DailyRollingFileAppender log4j.appender.stateChangeAppender.DatePattern='.'yyyy-MM-dd-HH -log4j.appender.stateChangeAppender.File=state-change.log +log4j.appender.stateChangeAppender.File=logs/state-change.log log4j.appender.stateChangeAppender.layout=org.apache.log4j.PatternLayout log4j.appender.stateChangeAppender.layout.ConversionPattern=[%d] %p %m (%c)%n log4j.appender.requestAppender=org.apache.log4j.DailyRollingFileAppender log4j.appender.requestAppender.DatePattern='.'yyyy-MM-dd-HH -log4j.appender.requestAppender.File=kafka-request.log +log4j.appender.requestAppender.File=logs/kafka-request.log log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n @@ -44,7 +44,7 @@ log4j.appender.cleanerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n log4j.appender.controllerAppender=org.apache.log4j.DailyRollingFileAppender log4j.appender.controllerAppender.DatePattern='.'yyyy-MM-dd-HH -log4j.appender.controllerAppender.File=controller.log +log4j.appender.controllerAppender.File=logs/controller.log log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n diff --git a/config/producer.properties b/config/producer.properties index 162b8a6..52a7611 100644 --- a/config/producer.properties +++ b/config/producer.properties @@ -16,7 +16,7 @@ ############################# Producer Basics ############################# -# list of brokers used for bootstrapping +# list of brokers used for bootstrapping knowledge about the rest of the cluster # format: host1:port1,host2:port2 ... metadata.broker.list=localhost:9092 diff --git a/config/server.properties b/config/server.properties index 7685879..2eccc5e 100644 --- a/config/server.properties +++ b/config/server.properties @@ -47,8 +47,8 @@ socket.request.max.bytes=104857600 ############################# Log Basics ############################# -# The directory under which to store log files -log.dir=/tmp/kafka-logs +# A comma seperated list of directories under which to store log files +log.dirs=/tmp/kafka-logs # The number of logical partitions per topic per server. More partitions allow greater parallelism # for consumption, but also mean more files. @@ -57,8 +57,13 @@ num.partitions=2 ############################# Log Flush Policy ############################# # Messages are immediately written to the filesystem but by default we only fsync() to sync -# the OS cache lazily. The below configuration can enforce a more aggressive application level -# fsync policy. This will have a significant performance impact. +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. # The number of messages to accept before forcing a flush of data to disk #log.flush.interval.messages=10000 @@ -99,11 +104,5 @@ zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper zookeeper.connection.timeout.ms=1000000 -# metrics reporter properties -kafka.metrics.polling.interval.secs=5 -kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter -kafka.csv.metrics.dir=/tmp/kafka_metrics -# Disable csv reporting by default. -kafka.csv.metrics.reporter.enabled=false log.cleanup.policy=delete diff --git a/config/tools-log4j.properties b/config/tools-log4j.properties new file mode 100644 index 0000000..7924049 --- /dev/null +++ b/config/tools-log4j.properties @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +log4j.rootLogger=WARN, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n + + diff --git a/contrib/hadoop-producer/README.md b/contrib/hadoop-producer/README.md index 547c1ef..a5bef73 100644 --- a/contrib/hadoop-producer/README.md +++ b/contrib/hadoop-producer/README.md @@ -77,8 +77,8 @@ turned off by the OutputFormat. What can I tune? ---------------- -* kafka.output.queue.size: Bytes to queue in memory before pushing to the Kafka - producer (i.e., the batch size). Default is 10*1024*1024 (10MB). +* kafka.output.queue.bytes: Bytes to queue in memory before pushing to the Kafka + producer (i.e., the batch size). Default is 1,000,000 (1 million) bytes. Any of Kafka's producer parameters can be changed by prefixing them with "kafka.output" in one's job configuration. For example, to change the 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 0b435b9..417b4b3 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 @@ -40,8 +40,11 @@ public class KafkaOutputFormat extends OutputFormat private Logger log = Logger.getLogger(KafkaOutputFormat.class); public static final String KAFKA_URL = "kafka.output.url"; - /** Bytes to buffer before the OutputFormat does a send (i.e., the amortization window) */ - public static final int KAFKA_QUEUE_SIZE = 10*1024*1024; + /** Bytes to buffer before the OutputFormat does a send (i.e., the amortization window): + * We set the default to a million bytes so that the server will not reject the batch of messages + * with a MessageSizeTooLargeException. The actual size will be smaller after compression. + */ + public static final int KAFKA_QUEUE_BYTES = 1000000; public static final String KAFKA_CONFIG_PREFIX = "kafka.output"; private static final Map kafkaConfigMap; @@ -116,8 +119,7 @@ public class KafkaOutputFormat extends OutputFormat } // KafkaOutputFormat specific parameters - final int queueSize = job.getInt(KAFKA_CONFIG_PREFIX + ".queue.size", KAFKA_QUEUE_SIZE); - job.setInt(KAFKA_CONFIG_PREFIX + ".queue.size", queueSize); + final int queueBytes = job.getInt(KAFKA_CONFIG_PREFIX + ".queue.bytes", KAFKA_QUEUE_BYTES); if (uri.getScheme().equals("kafka")) { // using the direct broker list @@ -137,6 +139,6 @@ public class KafkaOutputFormat extends OutputFormat throw new KafkaException("missing scheme from kafka uri (must be kafka://)"); Producer producer = new Producer(new ProducerConfig(props)); - return new KafkaRecordWriter(producer, topic, queueSize); + return new KafkaRecordWriter(producer, topic, queueBytes); } } 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 6eea635..72c088d 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 @@ -32,14 +32,14 @@ public class KafkaRecordWriter extends RecordWriter protected String topic; protected List> msgList = new LinkedList>(); - protected int totalSize = 0; - protected int queueSize; + protected int totalBytes = 0; + protected int queueBytes; - public KafkaRecordWriter(Producer producer, String topic, int queueSize) + public KafkaRecordWriter(Producer producer, String topic, int queueBytes) { this.producer = producer; this.topic = topic; - this.queueSize = queueSize; + this.queueBytes = queueBytes; } protected void sendMsgList() throws IOException @@ -52,7 +52,7 @@ public class KafkaRecordWriter extends RecordWriter throw new IOException(e); // all Kafka exceptions become IOExceptions } msgList.clear(); - totalSize = 0; + totalBytes = 0; } } @@ -69,12 +69,14 @@ public class KafkaRecordWriter extends RecordWriter else throw new IllegalArgumentException("KafkaRecordWriter expects byte array value to publish"); - msgList.add(new KeyedMessage(this.topic, key, valBytes)); - totalSize += valBytes.length; - // MultiProducerRequest only supports sending up to Short.MAX_VALUE messages in one batch - if (totalSize > queueSize || msgList.size() >= Short.MAX_VALUE) + // If the new message is going to make the message list tip over 1 million bytes, send the + // message list now. + if ((totalBytes + valBytes.length) > queueBytes || msgList.size() >= Short.MAX_VALUE) sendMsgList(); + + msgList.add(new KeyedMessage(this.topic, key, valBytes)); + totalBytes += valBytes.length; } @Override diff --git a/core/src/main/scala/kafka/admin/AddPartitionsCommand.scala b/core/src/main/scala/kafka/admin/AddPartitionsCommand.scala new file mode 100644 index 0000000..fd41661 --- /dev/null +++ b/core/src/main/scala/kafka/admin/AddPartitionsCommand.scala @@ -0,0 +1,127 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.admin + +import joptsimple.OptionParser +import kafka.utils._ +import org.I0Itec.zkclient.ZkClient +import scala.collection.mutable +import kafka.common.TopicAndPartition + +object AddPartitionsCommand extends Logging { + + def main(args: Array[String]): Unit = { + val parser = new OptionParser + val topicOpt = parser.accepts("topic", "REQUIRED: The topic for which partitions need to be added.") + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) + 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 nPartitionsOpt = parser.accepts("partition", "REQUIRED: Number of partitions to add to the topic") + .withRequiredArg + .describedAs("# of partitions") + .ofType(classOf[java.lang.Integer]) + val replicaAssignmentOpt = parser.accepts("replica-assignment-list", "For manually assigning replicas to brokers for the new partitions") + .withRequiredArg + .describedAs("broker_id_for_part1_replica1 : broker_id_for_part1_replica2, " + + "broker_id_for_part2_replica1 : broker_id_for_part2_replica2, ...") + .ofType(classOf[String]) + .defaultsTo("") + + val options = parser.parse(args : _*) + + for(arg <- List(topicOpt, zkConnectOpt, nPartitionsOpt)) { + if(!options.has(arg)) { + System.err.println("***Please note that this tool can only be used to add partitions when data for a topic does not use a key.***\n" + + "Missing required argument. " + " \"" + arg + "\"") + parser.printHelpOn(System.err) + System.exit(1) + } + } + + val topic = options.valueOf(topicOpt) + val zkConnect = options.valueOf(zkConnectOpt) + val nPartitions = options.valueOf(nPartitionsOpt).intValue + val replicaAssignmentStr = options.valueOf(replicaAssignmentOpt) + var zkClient: ZkClient = null + try { + zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr) + println("adding partitions succeeded!") + } catch { + case e => + println("adding partitions failed because of " + e.getMessage) + println(Utils.stackTrace(e)) + } finally { + if (zkClient != null) + zkClient.close() + } + } + + def addPartitions(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "") { + val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) + if (existingPartitionsReplicaList.size == 0) + throw new AdminOperationException("The topic %s does not exist".format(topic)) + + val existingReplicaList = existingPartitionsReplicaList.get(TopicAndPartition(topic,0)).get + + // create the new partition replication list + val brokerList = ZkUtils.getSortedBrokerList(zkClient) + val newPartitionReplicaList = if (replicaAssignmentStr == "") + AdminUtils.assignReplicasToBrokers(brokerList, numPartitions, existingReplicaList.size, existingReplicaList.head, existingPartitionsReplicaList.size) + else + getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet, existingPartitionsReplicaList.size) + + // check if manual assignment has the right replication factor + val unmatchedRepFactorList = newPartitionReplicaList.values.filter(p => (p.size != existingReplicaList.size)) + if (unmatchedRepFactorList.size != 0) + throw new AdminOperationException("The replication factor in manual replication assignment " + + " is not equal to the existing replication factor for the topic " + existingReplicaList.size) + + info("Add partition list for %s is %s".format(topic, newPartitionReplicaList)) + val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2) + // add the new list + partitionReplicaList ++= newPartitionReplicaList + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true) + } + + def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int): Map[Int, List[Int]] = { + val partitionList = replicaAssignmentList.split(",") + val ret = new mutable.HashMap[Int, List[Int]]() + var partitionId = startPartitionId + for (i <- 0 until partitionList.size) { + val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) + if (brokerList.size <= 0) + throw new AdminOperationException("replication factor must be larger than 0") + if (brokerList.size != brokerList.toSet.size) + throw new AdminOperationException("duplicate brokers in replica assignment: " + brokerList) + if (!brokerList.toSet.subsetOf(availableBrokerList)) + throw new AdminOperationException("some specified brokers not available. specified brokers: " + brokerList.toString + + "available broker:" + availableBrokerList.toString) + ret.put(partitionId, brokerList.toList) + if (ret(partitionId).size != ret(startPartitionId).size) + throw new AdminOperationException("partition " + i + " has different replication factor: " + brokerList) + partitionId = partitionId + 1 + } + ret.toMap + } +} diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 9ce89cd..83ba729 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -52,30 +52,33 @@ object AdminUtils extends Logging { * p3 p4 p0 p1 p2 (3nd replica) * p7 p8 p9 p5 p6 (3nd replica) */ - def assignReplicasToBrokers(brokers: Seq[Int], - partitions: Int, + def assignReplicasToBrokers(brokerList: Seq[Int], + nPartitions: Int, replicationFactor: Int, - fixedStartIndex: Int = -1) // for testing only + fixedStartIndex: Int = -1, + startPartitionId: Int = -1) : Map[Int, Seq[Int]] = { - if (partitions <= 0) + if (nPartitions <= 0) throw new AdminOperationException("number of partitions must be larger than 0") if (replicationFactor <= 0) throw new AdminOperationException("replication factor must be larger than 0") - if (replicationFactor > brokers.size) + if (replicationFactor > brokerList.size) throw new AdminOperationException("replication factor: " + replicationFactor + - " larger than available brokers: " + brokers.size) + " larger than available brokers: " + brokerList.size) val ret = new mutable.HashMap[Int, List[Int]]() - val startIndex = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(brokers.size) + val startIndex = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(brokerList.size) + var currentPartitionId = if (startPartitionId >= 0) startPartitionId else 0 - var secondReplicaShift = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(brokers.size) - for (i <- 0 until partitions) { - if (i > 0 && (i % brokers.size == 0)) - secondReplicaShift += 1 - val firstReplicaIndex = (i + startIndex) % brokers.size - var replicaList = List(brokers(firstReplicaIndex)) + var nextReplicaShift = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(brokerList.size) + for (i <- 0 until nPartitions) { + if (currentPartitionId > 0 && (currentPartitionId % brokerList.size == 0)) + nextReplicaShift += 1 + val firstReplicaIndex = (currentPartitionId + startIndex) % brokerList.size + var replicaList = List(brokerList(firstReplicaIndex)) for (j <- 0 until replicationFactor - 1) - replicaList ::= brokers(replicaIndex(firstReplicaIndex, secondReplicaShift, j, brokers.size)) - ret.put(i, replicaList.reverse) + replicaList ::= brokerList(replicaIndex(firstReplicaIndex, nextReplicaShift, j, brokerList.size)) + ret.put(currentPartitionId, replicaList.reverse) + currentPartitionId = currentPartitionId + 1 } ret.toMap } @@ -95,20 +98,21 @@ object AdminUtils extends Logging { topicConfig: Properties = new Properties) { val brokerList = ZkUtils.getSortedBrokerList(zkClient) val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor) - AdminUtils.createTopicWithAssignment(zkClient, topic, replicaAssignment, topicConfig) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig) } - def createTopicWithAssignment(zkClient: ZkClient, - topic: String, - partitionReplicaAssignment: Map[Int, Seq[Int]], - config: Properties = new Properties) { + def createOrUpdateTopicPartitionAssignmentPathInZK(zkClient: ZkClient, + topic: String, + partitionReplicaAssignment: Map[Int, Seq[Int]], + config: Properties = new Properties, + update: Boolean = false) { // validate arguments Topic.validate(topic) LogConfig.validate(config) require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.") val topicPath = ZkUtils.getTopicPath(topic) - if(zkClient.exists(topicPath)) + if(!update && zkClient.exists(topicPath)) throw new TopicExistsException("Topic \"%s\" already exists.".format(topic)) partitionReplicaAssignment.values.foreach(reps => require(reps.size == reps.toSet.size, "Duplicate replica assignment found: " + partitionReplicaAssignment)) @@ -116,14 +120,21 @@ object AdminUtils extends Logging { writeTopicConfig(zkClient, topic, config) // create the partition assignment - writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment) + writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update) } - private def writeTopicPartitionAssignment(zkClient: ZkClient, topic: String, replicaAssignment: Map[Int, Seq[Int]]) { + private def writeTopicPartitionAssignment(zkClient: ZkClient, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) { try { val zkPath = ZkUtils.getTopicPath(topic) val jsonPartitionData = ZkUtils.replicaAssignmentZkdata(replicaAssignment.map(e => (e._1.toString -> e._2))) - ZkUtils.createPersistentPath(zkClient, zkPath, jsonPartitionData) + + if (!update) { + info("Topic creation " + jsonPartitionData.toString) + ZkUtils.createPersistentPath(zkClient, zkPath, jsonPartitionData) + } else { + info("Topic update " + jsonPartitionData.toString) + ZkUtils.updatePersistentPath(zkClient, zkPath, jsonPartitionData) + } debug("Updated path %s with %s for replica assignment".format(zkPath, jsonPartitionData)) } catch { case e: ZkNodeExistsException => throw new TopicExistsException("topic %s already exists".format(topic)) diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 8d287f4..aa61fa1 100644 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -18,6 +18,7 @@ package kafka.admin import joptsimple.OptionParser import kafka.utils._ +import collection._ import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException import kafka.common.{TopicAndPartition, AdminCommandFailedException} @@ -26,21 +27,40 @@ object ReassignPartitionsCommand extends Logging { def main(args: Array[String]): Unit = { val parser = new OptionParser - val jsonFileOpt = parser.accepts("path-to-json-file", "REQUIRED: The JSON file with the list of partitions and the " + - "new replicas they should be reassigned to in the following format - \n" + - "{\"partitions\":\n\t[{\"topic\": \"foo\",\n\t \"partition\": 1,\n\t \"replicas\": [1,2,3] }]\n}") + val topicsToMoveJsonFileOpt = parser.accepts("topics-to-move-json-file", "The JSON file with the list of topics to reassign." + + "This option or manual-assignment-json-file needs to be specified. The format to use is - \n" + + "{\"topics\":\n\t[{\"topic\": \"foo\"},{\"topic\": \"foo1\"}],\n\"version\":1\n}") .withRequiredArg - .describedAs("partition reassignment json file path") + .describedAs("topics to reassign json file path") .ofType(classOf[String]) + + val manualAssignmentJsonFileOpt = parser.accepts("manual-assignment-json-file", "The JSON file with the list of manual reassignments" + + "This option or topics-to-move-json-file needs to be specified. The format to use is - \n" + + "{\"partitions\":\n\t[{\"topic\": \"foo\",\n\t \"partition\": 1,\n\t \"replicas\": [1,2,3] }],\n\"version\":1\n}") + .withRequiredArg + .describedAs("manual assignment json file path") + .ofType(classOf[String]) + + val brokerListOpt = parser.accepts("broker-list", "The list of brokers to which the partitions need to be reassigned" + + " in the form \"0,1,2\". This is required for automatic topic reassignment.") + .withRequiredArg + .describedAs("brokerlist") + .ofType(classOf[String]) + 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 executeOpt = parser.accepts("execute", "This option does the actual reassignment. By default, the tool does a dry run") + .withOptionalArg() + .describedAs("execute") + .ofType(classOf[String]) + val options = parser.parse(args : _*) - for(arg <- List(jsonFileOpt, zkConnectOpt)) { + for(arg <- List(zkConnectOpt)) { if(!options.has(arg)) { System.err.println("Missing required argument \"" + arg + "\"") parser.printHelpOn(System.err) @@ -48,24 +68,56 @@ object ReassignPartitionsCommand extends Logging { } } - val jsonFile = options.valueOf(jsonFileOpt) - val zkConnect = options.valueOf(zkConnectOpt) - val jsonString = Utils.readFileAsString(jsonFile) - var zkClient: ZkClient = null + if (options.has(topicsToMoveJsonFileOpt) && options.has(manualAssignmentJsonFileOpt)) { + System.err.println("Only one of the json files should be specified") + parser.printHelpOn(System.err) + System.exit(1) + } + val zkConnect = options.valueOf(zkConnectOpt) + var zkClient: ZkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) try { - // read the json file into a string - val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentData(jsonString) - if (partitionsToBeReassigned.isEmpty) - throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(jsonFile)) - - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) - val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned) - - if(reassignPartitionsCommand.reassignPartitions()) - println("Successfully started reassignment of partitions %s".format(partitionsToBeReassigned)) - else - println("Failed to reassign partitions %s".format(partitionsToBeReassigned)) + + var partitionsToBeReassigned : Map[TopicAndPartition, Seq[Int]] = new mutable.HashMap[TopicAndPartition, List[Int]]() + + if(options.has(topicsToMoveJsonFileOpt)) { + val topicsToMoveJsonFile = options.valueOf(topicsToMoveJsonFileOpt) + val brokerList = options.valueOf(brokerListOpt) + val topicsToMoveJsonString = Utils.readFileAsString(topicsToMoveJsonFile) + val brokerListToReassign = brokerList.split(',') map (_.toInt) + val topicsToReassign = ZkUtils.parseTopicsData(topicsToMoveJsonString) + val topicPartitionsToReassign = ZkUtils.getReplicaAssignmentForTopics(zkClient, topicsToReassign) + + val groupedByTopic = topicPartitionsToReassign.groupBy(tp => tp._1.topic) + groupedByTopic.foreach { topicInfo => + val assignedReplicas = AdminUtils.assignReplicasToBrokers(brokerListToReassign, topicInfo._2.size, + topicInfo._2.head._2.size) + partitionsToBeReassigned ++= assignedReplicas.map(replicaInfo => (TopicAndPartition(topicInfo._1, replicaInfo._1) -> replicaInfo._2)) + } + + } else if (options.has(manualAssignmentJsonFileOpt)) { + val manualAssignmentJsonFile = options.valueOf(manualAssignmentJsonFileOpt) + val manualAssignmentJsonString = Utils.readFileAsString(manualAssignmentJsonFile) + partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentData(manualAssignmentJsonString) + if (partitionsToBeReassigned.isEmpty) + throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(manualAssignmentJsonFileOpt)) + } else { + System.err.println("Missing json file. One of the file needs to be specified") + parser.printHelpOn(System.err) + System.exit(1) + } + + if (options.has(executeOpt)) { + val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned) + + if(reassignPartitionsCommand.reassignPartitions()) + println("Successfully started reassignment of partitions %s".format(partitionsToBeReassigned)) + else + println("Failed to reassign partitions %s".format(partitionsToBeReassigned)) + } else { + System.out.println("This is a dry run (Use --execute to do the actual reassignment. " + + "The replica assignment is \n" + partitionsToBeReassigned.toString()) + } } catch { case e => println("Partitions reassignment failed due to " + e.getMessage) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index e604233..06bbd37 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -65,7 +65,7 @@ object TopicCommand { for (topic <- topics) { if (opts.options.has(opts.replicaAssignmentOpt)) { val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt)) - AdminUtils.createTopicWithAssignment(zkClient, topic, assignment, configs) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs) } else { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt) val partitions = opts.options.valueOf(opts.partitionsOpt).intValue diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index a474474..981d2bb 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -135,13 +135,13 @@ case class LeaderAndIsrRequest (versionId: Short, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[(String, Int), PartitionStateInfo], - aliveLeaders: Set[Broker]) + leaders: Set[Broker]) extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey), correlationId) { - def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], aliveLeaders: Set[Broker], controllerId: Int, + def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[Broker], controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String) = { this(LeaderAndIsrRequest.CurrentVersion, correlationId, clientId, - controllerId, controllerEpoch, partitionStateInfos, aliveLeaders) + controllerId, controllerEpoch, partitionStateInfos, leaders) } def writeTo(buffer: ByteBuffer) { @@ -156,8 +156,8 @@ case class LeaderAndIsrRequest (versionId: Short, buffer.putInt(key._2) value.writeTo(buffer) } - buffer.putInt(aliveLeaders.size) - aliveLeaders.foreach(_.writeTo(buffer)) + buffer.putInt(leaders.size) + leaders.foreach(_.writeTo(buffer)) } def sizeInBytes(): Int = { @@ -171,7 +171,7 @@ 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 <- aliveLeaders) + for(broker <- leaders) size += broker.sizeInBytes /* broker info */ size } @@ -185,7 +185,7 @@ case class LeaderAndIsrRequest (versionId: Short, leaderAndIsrRequest.append(";CorrelationId:" + correlationId) leaderAndIsrRequest.append(";ClientId:" + clientId) leaderAndIsrRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) - leaderAndIsrRequest.append(";Leaders:" + aliveLeaders.mkString(",")) + leaderAndIsrRequest.append(";Leaders:" + leaders.mkString(",")) leaderAndIsrRequest.toString() } diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 435c473..b03dea2 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -58,8 +58,6 @@ private[kafka] case class Broker(val id: Int, val host: String, val port: Int) { override def toString(): String = new String("id:" + id + ",host:" + host + ",port:" + port) - def getZkString(): String = host + ":" + port - def getConnectionString(): String = host + ":" + port def writeTo(buffer: ByteBuffer) { diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 3323b60..39f51ce 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -173,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, - aliveLeaders: Set[Broker], correlationId: Int): Boolean = { + leaders: Set[Broker], correlationId: Int): Boolean = { leaderIsrUpdateLock synchronized { val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr if (leaderEpoch >= leaderAndIsr.leaderEpoch) { @@ -192,7 +192,8 @@ class Partition(val topic: String, // on the leader val localReplica = getOrCreateReplica() val newLeaderBrokerId: Int = leaderAndIsr.leader - aliveLeaders.find(_.id == newLeaderBrokerId) match { + // TODO: Delete leaders from LeaderAndIsrRequest in 0.8.1 + leaders.find(_.id == newLeaderBrokerId) match { case Some(leaderBroker) => // stop fetcher thread to previous leader replicaFetcherManager.removeFetcher(topic, partitionId) @@ -207,15 +208,15 @@ class Partition(val topic: String, replicaFetcherManager.addFetcher(topic, partitionId, localReplica.logEndOffset, leaderBroker) } else { - stateChangeLogger.trace("Broker %d ignored the become-follower state change with correlation id %d from " + - " controller %d epoch %d since it is shutting down" - .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch)) + stateChangeLogger.trace(("Broker %d ignored the become-follower state change with correlation id %d from " + + "controller %d epoch %d since it is shutting down") + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch)) } - case None => // leader went down - stateChangeLogger.trace("Broker %d aborted the become-follower state change with correlation id %d from " + - " controller %d epoch %d since leader %d for partition [%s,%d] is unavailable during the state change operation" - .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, - newLeaderBrokerId, topic, partitionId)) + case None => // we should not come here + stateChangeLogger.error(("Broker %d aborted the become-follower state change with correlation id %d from " + + "controller %d epoch %d for partition [%s,%d] new leader %d") + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, + topic, partitionId, newLeaderBrokerId)) } true } diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index 71ae640..fa6b213 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -39,7 +39,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, private val config: ConsumerConfig, private val zkClient : ZkClient) extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(SystemTime.milliseconds), - config.groupId, 1) { + config.clientId, 1) { private var partitionMap: immutable.Map[TopicAndPartition, PartitionTopicInfo] = null private var cluster: Cluster = null private val noLeaderPartitionSet = new mutable.HashSet[TopicAndPartition] @@ -92,7 +92,20 @@ class ConsumerFetcherManager(private val consumerIdString: String, leaderForPartitionsMap.foreach { case(topicAndPartition, leaderBroker) => val pti = partitionMap(topicAndPartition) - addFetcher(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset(), leaderBroker) + try { + addFetcher(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset(), leaderBroker) + } catch { + case t => { + if (!isRunning.get()) + throw t /* If this thread is stopped, propagate this exception to kill the thread. */ + else { + warn("Failed to add leader for partition %s; will retry".format(topicAndPartition), t) + lock.lock() + noLeaderPartitionSet += topicAndPartition + lock.unlock() + } + } + } } shutdownIdleFetcherThreads() diff --git a/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala b/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala index 64b702b..9c779ce 100644 --- a/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala +++ b/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala @@ -62,6 +62,8 @@ class PartitionTopicInfo(val topic: String, debug("updated fetch offset of (%s) to %d".format(this, next)) consumerTopicStats.getConsumerTopicStats(topic).byteRate.mark(size) consumerTopicStats.getConsumerAllTopicStats().byteRate.mark(size) + } else if(messages.sizeInBytes > 0) { + chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get)) } } diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index 1c4e1da..77e1ce2 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -73,7 +73,7 @@ class SimpleConsumer(val host: String, response = blockingChannel.receive() } catch { case e : java.io.IOException => - info("Reconnect due to socket error: ", e) + info("Reconnect due to socket error: %s".format(e.getMessage)) // retry once try { reconnect() diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index e3a6420..08c9e4f 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -85,6 +85,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private var fetcher: Option[ConsumerFetcherManager] = None private var zkClient: ZkClient = null private var topicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] + private var checkpointedOffsets = new Pool[TopicAndPartition, Long] private val topicThreadIdAndQueues = new Pool[(String,String), BlockingQueue[FetchedDataChunk]] private val scheduler = new KafkaScheduler(threads = 1, threadNamePrefix = "kafka-consumer-scheduler-") private val messageStreamCreated = new AtomicBoolean(false) @@ -155,31 +156,33 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } def shutdown() { - val canShutdown = isShuttingDown.compareAndSet(false, true); - if (canShutdown) { - info("ZKConsumerConnector shutting down") + rebalanceLock synchronized { + val canShutdown = isShuttingDown.compareAndSet(false, true); + if (canShutdown) { + info("ZKConsumerConnector shutting down") - if (wildcardTopicWatcher != null) - wildcardTopicWatcher.shutdown() - try { - if (config.autoCommitEnable) - scheduler.shutdown() - fetcher match { - case Some(f) => f.stopConnections - case None => - } - sendShutdownToAllQueues() - if (config.autoCommitEnable) - commitOffsets() - if (zkClient != null) { - zkClient.close() - zkClient = null + if (wildcardTopicWatcher != null) + wildcardTopicWatcher.shutdown() + try { + if (config.autoCommitEnable) + scheduler.shutdown() + fetcher match { + case Some(f) => f.stopConnections + case None => + } + sendShutdownToAllQueues() + if (config.autoCommitEnable) + commitOffsets() + if (zkClient != null) { + zkClient.close() + zkClient = null + } + } catch { + case e => + fatal("error during consumer connector shutdown", e) } - } catch { - case e => - fatal("error during consumer connector shutdown", e) + info("ZKConsumerConnector shut down completed") } - info("ZKConsumerConnector shut down completed") } } @@ -213,12 +216,14 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, // this API is used by unit tests only def getTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]] = topicRegistry - private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) = { + private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) { info("begin registering consumer " + consumerIdString + " in ZK") + val timestamp = SystemTime.milliseconds.toString val consumerRegistrationInfo = Utils.mergeJsonFields(Utils.mapToJsonFields(Map("version" -> 1.toString, "subscription" -> topicCount.dbString), valueInQuotes = false) - ++ Utils.mapToJsonFields(Map("pattern" -> topicCount.pattern), valueInQuotes = true)) - createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, consumerRegistrationInfo) + ++ Utils.mapToJsonFields(Map("pattern" -> topicCount.pattern, "timestamp" -> timestamp), valueInQuotes = true)) + + createEphemeralPathExpectConflictHandleZKBug(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, consumerRegistrationInfo, null, (consumerZKString, consumer) => true, config.zkSessionTimeoutMs) info("end registering consumer " + consumerIdString + " in ZK") } @@ -252,15 +257,17 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) for (info <- infos.values) { val newOffset = info.getConsumeOffset - try { - updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partitionId, - newOffset.toString) - } catch { - case t: Throwable => - // log it and let it go - warn("exception during commitOffsets", t) + if (newOffset != checkpointedOffsets.get(TopicAndPartition(topic, info.partitionId))) { + try { + updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partitionId, newOffset.toString) + checkpointedOffsets.put(TopicAndPartition(topic, info.partitionId), newOffset) + } catch { + case t: Throwable => + // log it and let it go + warn("exception during commitOffsets", t) + } + debug("Committed offset " + newOffset + " for topic " + info) } - debug("Committed offset " + newOffset + " for topic " + info) } } } @@ -369,31 +376,36 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def syncedRebalance() { rebalanceLock synchronized { - for (i <- 0 until config.rebalanceMaxRetries) { - info("begin rebalancing consumer " + consumerIdString + " try #" + i) - var done = false - val cluster = getCluster(zkClient) - try { - done = rebalance(cluster) - } catch { - case e => - /** occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating. - * For example, a ZK node can disappear between the time we get all children and the time we try to get - * the value of a child. Just let this go since another rebalance will be triggered. - **/ - info("exception during rebalance ", e) - } - info("end rebalancing consumer " + consumerIdString + " try #" + i) - if (done) { - return - } else { + if(isShuttingDown.get()) { + return + } else { + for (i <- 0 until config.rebalanceMaxRetries) { + info("begin rebalancing consumer " + consumerIdString + " try #" + i) + var done = false + var cluster: Cluster = null + try { + cluster = getCluster(zkClient) + done = rebalance(cluster) + } catch { + case e => + /** occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating. + * For example, a ZK node can disappear between the time we get all children and the time we try to get + * the value of a child. Just let this go since another rebalance will be triggered. + **/ + info("exception during rebalance ", e) + } + info("end rebalancing consumer " + consumerIdString + " try #" + i) + if (done) { + return + } else { /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should * clear the cache */ info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") + } + // stop all fetchers and clear all the queues to avoid data duplication + closeFetchersForQueues(cluster, kafkaMessageAndMetadataStreams, topicThreadIdAndQueues.map(q => q._2)) + Thread.sleep(config.rebalanceBackoffMs) } - // stop all fetchers and clear all the queues to avoid data duplication - closeFetchersForQueues(cluster, kafkaMessageAndMetadataStreams, topicThreadIdAndQueues.map(q => q._2)) - Thread.sleep(config.rebalanceBackoffMs) } } @@ -404,81 +416,91 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic val consumersPerTopicMap = getConsumersPerTopic(zkClient, group) val brokers = getAllBrokersInCluster(zkClient) - val topicsMetadata = ClientUtils.fetchTopicMetadata(myTopicThreadIdsMap.keySet, - brokers, - config.clientId, - config.socketTimeoutMs, - correlationId.getAndIncrement).topicsMetadata - val partitionsPerTopicMap = new mutable.HashMap[String, Seq[Int]] - topicsMetadata.foreach(m => { - val topic = m.topic - val partitions = m.partitionsMetadata.map(m1 => m1.partitionId) - partitionsPerTopicMap.put(topic, partitions) - }) - - /** - * fetchers must be stopped to avoid data duplication, since if the current - * rebalancing attempt fails, the partitions that are released could be owned by another consumer. - * But if we don't stop the fetchers first, this consumer would continue returning data for released - * partitions in parallel. So, not stopping the fetchers leads to duplicate data. - */ - closeFetchers(cluster, kafkaMessageAndMetadataStreams, myTopicThreadIdsMap) - - releasePartitionOwnership(topicRegistry) - - var partitionOwnershipDecision = new collection.mutable.HashMap[(String, Int), String]() - val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] - - for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) { - currentTopicRegistry.put(topic, new Pool[Int, PartitionTopicInfo]) - - val topicDirs = new ZKGroupTopicDirs(group, topic) - val curConsumers = consumersPerTopicMap.get(topic).get - val curPartitions: Seq[Int] = partitionsPerTopicMap.get(topic).get - - val nPartsPerConsumer = curPartitions.size / curConsumers.size - val nConsumersWithExtraPart = curPartitions.size % curConsumers.size - - info("Consumer " + consumerIdString + " rebalancing the following partitions: " + curPartitions + - " for topic " + topic + " with consumers: " + curConsumers) - - for (consumerThreadId <- consumerThreadIdSet) { - val myConsumerPosition = curConsumers.findIndexOf(_ == consumerThreadId) - assert(myConsumerPosition >= 0) - val startPart = nPartsPerConsumer*myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) - val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1) - - /** - * Range-partition the sorted partitions to consumers for better locality. - * The first few consumers pick up an extra partition, if any. - */ - if (nParts <= 0) - warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic) - else { - for (i <- startPart until startPart + nParts) { - val partition = curPartitions(i) - info(consumerThreadId + " attempting to claim partition " + partition) - addPartitionTopicInfo(currentTopicRegistry, topicDirs, partition, topic, consumerThreadId) - // record the partition ownership decision - partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) + if (brokers.size == 0) { + // This can happen in a rare case when there are no brokers available in the cluster when the consumer is started. + // We log an warning and register for child changes on brokers/id so that rebalance can be triggered when the brokers + // are up. + warn("no brokers found when trying to rebalance.") + zkClient.subscribeChildChanges(ZkUtils.BrokerIdsPath, loadBalancerListener) + true + } + else { + val topicsMetadata = ClientUtils.fetchTopicMetadata(myTopicThreadIdsMap.keySet, + brokers, + config.clientId, + config.socketTimeoutMs, + correlationId.getAndIncrement).topicsMetadata + val partitionsPerTopicMap = new mutable.HashMap[String, Seq[Int]] + topicsMetadata.foreach(m => { + val topic = m.topic + val partitions = m.partitionsMetadata.map(m1 => m1.partitionId) + partitionsPerTopicMap.put(topic, partitions) + }) + + /** + * fetchers must be stopped to avoid data duplication, since if the current + * rebalancing attempt fails, the partitions that are released could be owned by another consumer. + * But if we don't stop the fetchers first, this consumer would continue returning data for released + * partitions in parallel. So, not stopping the fetchers leads to duplicate data. + */ + closeFetchers(cluster, kafkaMessageAndMetadataStreams, myTopicThreadIdsMap) + + releasePartitionOwnership(topicRegistry) + + var partitionOwnershipDecision = new collection.mutable.HashMap[(String, Int), String]() + val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] + + for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) { + currentTopicRegistry.put(topic, new Pool[Int, PartitionTopicInfo]) + + val topicDirs = new ZKGroupTopicDirs(group, topic) + val curConsumers = consumersPerTopicMap.get(topic).get + val curPartitions: Seq[Int] = partitionsPerTopicMap.get(topic).get + + val nPartsPerConsumer = curPartitions.size / curConsumers.size + val nConsumersWithExtraPart = curPartitions.size % curConsumers.size + + info("Consumer " + consumerIdString + " rebalancing the following partitions: " + curPartitions + + " for topic " + topic + " with consumers: " + curConsumers) + + for (consumerThreadId <- consumerThreadIdSet) { + val myConsumerPosition = curConsumers.findIndexOf(_ == consumerThreadId) + assert(myConsumerPosition >= 0) + val startPart = nPartsPerConsumer*myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) + val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1) + + /** + * Range-partition the sorted partitions to consumers for better locality. + * The first few consumers pick up an extra partition, if any. + */ + if (nParts <= 0) + warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic) + else { + for (i <- startPart until startPart + nParts) { + val partition = curPartitions(i) + info(consumerThreadId + " attempting to claim partition " + partition) + addPartitionTopicInfo(currentTopicRegistry, topicDirs, partition, topic, consumerThreadId) + // record the partition ownership decision + partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) + } } } } - } - /** - * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt - * A rebalancing attempt is completed successfully only after the fetchers have been started correctly - */ - if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) { - info("Updating the cache") - debug("Partitions per topic cache " + partitionsPerTopicMap) - debug("Consumers per topic cache " + consumersPerTopicMap) - topicRegistry = currentTopicRegistry - updateFetcher(cluster) - true - } else { - false + /** + * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt + * A rebalancing attempt is completed successfully only after the fetchers have been started correctly + */ + if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) { + info("Updating the cache") + debug("Partitions per topic cache " + partitionsPerTopicMap) + debug("Consumers per topic cache " + consumersPerTopicMap) + topicRegistry = currentTopicRegistry + updateFetcher(cluster) + true + } else { + false + } } } @@ -600,6 +622,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, config.clientId) partTopicInfoMap.put(partition, partTopicInfo) debug(partTopicInfo + " selected new offset " + offset) + checkpointedOffsets.put(TopicAndPartition(topic, partition), offset) } } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 38b8674..ed1ce0b 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -229,14 +229,13 @@ class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendReq } } - def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int, liveBrokers: Set[Broker]) { + def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int) { leaderAndIsrRequestMap.foreach { m => val broker = m._1 val partitionStateInfos = m._2.toMap val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet - val aliveLeaders = liveBrokers.filter(b => leaderIds.contains(b.id)) - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, aliveLeaders, controllerId, controllerEpoch, correlationId, - clientId) + val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)) + val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, 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 5ac38fd..ab18b7a 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -28,7 +28,7 @@ import kafka.common._ import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} import kafka.server.{ZookeeperLeaderElector, KafkaConfig} import kafka.utils.ZkUtils._ -import kafka.utils.{Utils, ZkUtils, Logging} +import kafka.utils.{Json, Utils, ZkUtils, Logging} import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} @@ -37,6 +37,7 @@ import scala.Some import kafka.common.TopicAndPartition class ControllerContext(val zkClient: ZkClient, + val zkSessionTimeout: Int, var controllerChannelManager: ControllerChannelManager = null, val controllerLock: Object = new Object, var shuttingDownBrokerIds: mutable.Set[Int] = mutable.Set.empty, @@ -73,17 +74,38 @@ trait KafkaControllerMBean { def shutdownBroker(id: Int): Set[TopicAndPartition] } -object KafkaController { +object KafkaController extends Logging { val MBeanName = "kafka.controller:type=KafkaController,name=ControllerOps" val stateChangeLogger = "state.change.logger" val InitialControllerEpoch = 1 val InitialControllerEpochZkVersion = 1 + + def parseControllerId(controllerInfoString: String): Int = { + try { + Json.parseFull(controllerInfoString) match { + case Some(m) => + val controllerInfo = m.asInstanceOf[Map[String, Any]] + return controllerInfo.get("brokerid").get.asInstanceOf[Int] + case None => throw new KafkaException("Failed to parse the controller info json [%s].".format(controllerInfoString)) + } + } catch { + case t => + // It may be due to an incompatible controller register version + warn("Failed to parse the controller info as json. " + + "Probably this controller is still using the old format [%s] to store the broker id in zookeeper".format(controllerInfoString)) + try { + return controllerInfoString.toInt + } catch { + case t => throw new KafkaException("Failed to parse the controller info: " + controllerInfoString + ". This is neither the new or the old format.", t) + } + } + } } class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logging with KafkaMetricsGroup with KafkaControllerMBean { this.logIdent = "[Controller " + config.brokerId + "]: " private var isRunning = true - val controllerContext = new ControllerContext(zkClient) + val controllerContext = new ControllerContext(zkClient, config.zkSessionTimeoutMs) private val partitionStateMachine = new PartitionStateMachine(this) private val replicaStateMachine = new ReplicaStateMachine(this) private val controllerElector = new ZookeeperLeaderElector(controllerContext, ZkUtils.ControllerPath, onControllerFailover, @@ -168,7 +190,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // before which the stop replica request should be completed (in most cases) brokerRequestBatch.newBatch() brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, topicAndPartition.partition, deletePartition = false) - brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) + brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) // If the broker is a follower, updates the isr in ZK and notifies the current leader replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic, @@ -215,6 +237,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg initializeControllerContext() replicaStateMachine.startup() partitionStateMachine.startup() + // register the partition change listeners for all existing topics on failover + controllerContext.allTopics.foreach(topic => partitionStateMachine.registerPartitionChangeListener(topic)) Utils.registerMBean(this, KafkaController.MBeanName) info("Broker %d is ready to serve as the new controller with epoch %d".format(config.brokerId, epoch)) initializeAndMaybeTriggerPartitionReassignment() @@ -332,13 +356,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * Reassigning replicas for a partition goes through a few stages - * RAR = Reassigned replicas * AR = Original list of replicas for partition - * 1. Register listener for ISR changes to detect when the RAR is a subset of the ISR - * 2. Start new replicas RAR - AR. - * 3. Wait until new replicas are in sync with the leader - * 4. If the leader is not in RAR, elect a new leader from RAR - * 5. Stop old replicas AR - RAR - * 6. Write new AR - * 7. Remove partition from the /admin/reassign_partitions path + * 1. Start new replicas RAR - AR. + * 2. Wait until new replicas are in sync with the leader + * 3. If the leader is not in RAR, elect a new leader from RAR + * 4. Stop old replicas AR - RAR + * 5. Write new AR + * 6. Remove partition from the /admin/reassign_partitions path */ def onPartitionReassignment(topicAndPartition: TopicAndPartition, reassignedPartitionContext: ReassignedPartitionsContext) { val reassignedReplicas = reassignedPartitionContext.newReplicas @@ -371,6 +394,54 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } + private def watchIsrChangesForReassignedPartition(topic: String, + partition: Int, + reassignedPartitionContext: ReassignedPartitionsContext) { + val reassignedReplicas = reassignedPartitionContext.newReplicas + val isrChangeListener = new ReassignedPartitionsIsrChangeListener(this, topic, partition, + reassignedReplicas.toSet) + reassignedPartitionContext.isrChangeListener = isrChangeListener + // register listener on the leader and isr path to wait until they catch up with the current leader + zkClient.subscribeDataChanges(ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition), isrChangeListener) + } + + def initiateReassignReplicasForTopicPartition(topicAndPartition: TopicAndPartition, + reassignedPartitionContext: ReassignedPartitionsContext) { + val newReplicas = reassignedPartitionContext.newReplicas + val topic = topicAndPartition.topic + val partition = topicAndPartition.partition + val aliveNewReplicas = newReplicas.filter(r => controllerContext.liveBrokerIds.contains(r)) + try { + val assignedReplicasOpt = controllerContext.partitionReplicaAssignment.get(topicAndPartition) + assignedReplicasOpt match { + case Some(assignedReplicas) => + if(assignedReplicas == newReplicas) { + throw new KafkaException("Partition %s to be reassigned is already assigned to replicas".format(topicAndPartition) + + " %s. Ignoring request for partition reassignment".format(newReplicas.mkString(","))) + } else { + if(aliveNewReplicas == newReplicas) { + info("Handling reassignment of partition %s to new replicas %s".format(topicAndPartition, newReplicas.mkString(","))) + // first register ISR change listener + watchIsrChangesForReassignedPartition(topic, partition, reassignedPartitionContext) + controllerContext.partitionsBeingReassigned.put(topicAndPartition, reassignedPartitionContext) + onPartitionReassignment(topicAndPartition, reassignedPartitionContext) + } else { + // some replica in RAR is not alive. Fail partition reassignment + throw new KafkaException("Only %s replicas out of the new set of replicas".format(aliveNewReplicas.mkString(",")) + + " %s for partition %s to be reassigned are alive. ".format(newReplicas.mkString(","), topicAndPartition) + + "Failing partition reassignment") + } + } + case None => throw new KafkaException("Attempt to reassign partition %s that doesn't exist" + .format(topicAndPartition)) + } + } catch { + case e => error("Error completing reassignment of partition %s".format(topicAndPartition), e) + // remove the partition from the admin path to unblock the admin client + removePartitionFromReassignedPartitions(topicAndPartition) + } + } + def onPreferredReplicaElection(partitions: Set[TopicAndPartition]) { info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(","))) try { @@ -477,12 +548,17 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg val reassignedPartitions = partitionsBeingReassigned.filter(partition => controllerContext.partitionReplicaAssignment(partition._1) == partition._2.newReplicas).map(_._1) reassignedPartitions.foreach(p => removePartitionFromReassignedPartitions(p)) - controllerContext.partitionsBeingReassigned ++= partitionsBeingReassigned - controllerContext.partitionsBeingReassigned --= reassignedPartitions + var partitionsToReassign: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap + partitionsToReassign ++= partitionsBeingReassigned + partitionsToReassign --= reassignedPartitions + info("Partitions being reassigned: %s".format(partitionsBeingReassigned.toString())) info("Partitions already reassigned: %s".format(reassignedPartitions.toString())) - info("Resuming reassignment of partitions: %s".format(controllerContext.partitionsBeingReassigned.toString())) - controllerContext.partitionsBeingReassigned.foreach(partition => onPartitionReassignment(partition._1, partition._2)) + info("Resuming reassignment of partitions: %s".format(partitionsToReassign.toString())) + + partitionsToReassign.foreach { topicPartitionToReassign => + initiateReassignReplicasForTopicPartition(topicPartitionToReassign._1, topicPartitionToReassign._2) + } } private def initializeAndMaybeTriggerPreferredReplicaElection() { @@ -571,8 +647,6 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // stop watching the ISR changes for this partition zkClient.unsubscribeDataChanges(ZkUtils.getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition), controllerContext.partitionsBeingReassigned(topicAndPartition).isrChangeListener) - // update the assigned replica list - controllerContext.partitionReplicaAssignment.put(topicAndPartition, reassignedReplicas) } private def startNewReplicasForReassignedPartition(topicAndPartition: TopicAndPartition, @@ -654,7 +728,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) { brokerRequestBatch.newBatch() brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) - brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) + brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) } /** @@ -771,39 +845,8 @@ class PartitionsReassignedListener(controller: KafkaController) extends IZkDataL val newPartitions = partitionsReassignmentData.filterNot(p => controllerContext.partitionsBeingReassigned.contains(p._1)) newPartitions.foreach { partitionToBeReassigned => controllerContext.controllerLock synchronized { - val topic = partitionToBeReassigned._1.topic - val partition = partitionToBeReassigned._1.partition - val newReplicas = partitionToBeReassigned._2 - val topicAndPartition = partitionToBeReassigned._1 - val aliveNewReplicas = newReplicas.filter(r => controllerContext.liveBrokerIds.contains(r)) - try { - val assignedReplicasOpt = controllerContext.partitionReplicaAssignment.get(topicAndPartition) - assignedReplicasOpt match { - case Some(assignedReplicas) => - if(assignedReplicas == newReplicas) { - throw new KafkaException("Partition %s to be reassigned is already assigned to replicas".format(topicAndPartition) + - " %s. Ignoring request for partition reassignment".format(newReplicas.mkString(","))) - } else { - if(aliveNewReplicas == newReplicas) { - info("Handling reassignment of partition %s to new replicas %s".format(topicAndPartition, newReplicas.mkString(","))) - val context = createReassignmentContextForPartition(topic, partition, newReplicas) - controllerContext.partitionsBeingReassigned.put(topicAndPartition, context) - controller.onPartitionReassignment(topicAndPartition, context) - } else { - // some replica in RAR is not alive. Fail partition reassignment - throw new KafkaException("Only %s replicas out of the new set of replicas".format(aliveNewReplicas.mkString(",")) + - " %s for partition %s to be reassigned are alive. ".format(newReplicas.mkString(","), topicAndPartition) + - "Failing partition reassignment") - } - } - case None => throw new KafkaException("Attempt to reassign partition %s that doesn't exist" - .format(topicAndPartition)) - } - } catch { - case e => error("Error completing reassignment of partition %s".format(topicAndPartition), e) - // remove the partition from the admin path to unblock the admin client - controller.removePartitionFromReassignedPartitions(topicAndPartition) - } + val context = new ReassignedPartitionsContext(partitionToBeReassigned._2) + controller.initiateReassignReplicasForTopicPartition(partitionToBeReassigned._1, context) } } } @@ -816,25 +859,6 @@ class PartitionsReassignedListener(controller: KafkaController) extends IZkDataL @throws(classOf[Exception]) def handleDataDeleted(dataPath: String) { } - - private def createReassignmentContextForPartition(topic: String, - partition: Int, - newReplicas: Seq[Int]): ReassignedPartitionsContext = { - val context = new ReassignedPartitionsContext(newReplicas) - // first register ISR change listener - watchIsrChangesForReassignedPartition(topic, partition, context) - context - } - - private def watchIsrChangesForReassignedPartition(topic: String, partition: Int, - reassignedPartitionContext: ReassignedPartitionsContext) { - val reassignedReplicas = reassignedPartitionContext.newReplicas - val isrChangeListener = new ReassignedPartitionsIsrChangeListener(controller, topic, partition, - reassignedReplicas.toSet) - reassignedPartitionContext.isrChangeListener = isrChangeListener - // register listener on the leader and isr path to wait until they catch up with the current leader - zkClient.subscribeDataChanges(ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition), isrChangeListener) - } } class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic: String, partition: Int, diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index deebed0..a084830 100644 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.api.LeaderAndIsr import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException} import kafka.utils.{Logging, ZkUtils} -import org.I0Itec.zkclient.IZkChildListener +import org.I0Itec.zkclient.{IZkDataListener, IZkChildListener} import org.I0Itec.zkclient.exception.ZkNodeExistsException import org.apache.log4j.Logger @@ -89,7 +89,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { if(partitionState.equals(OfflinePartition) || partitionState.equals(NewPartition)) handleStateChange(topicAndPartition.topic, topicAndPartition.partition, OnlinePartition, controller.offlinePartitionSelector) } - brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) + brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) } catch { case e => error("Error while moving some partitions to the online state", e) // TODO: It is not enough to bail out and log an error, it is important to trigger leader election for those partitions @@ -109,7 +109,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { partitions.foreach { topicAndPartition => handleStateChange(topicAndPartition.topic, topicAndPartition.partition, targetState, leaderSelector) } - brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) + brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) }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 @@ -334,7 +334,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } def registerPartitionChangeListener(topic: String) = { - zkClient.subscribeChildChanges(ZkUtils.getTopicPath(topic), new PartitionChangeListener(topic)) + zkClient.subscribeDataChanges(ZkUtils.getTopicPath(topic), new AddPartitionsListener(topic)) } private def getLeaderIsrAndEpochOrThrowException(topic: String, partition: Int): LeaderIsrAndControllerEpoch = { @@ -383,15 +383,31 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } } - class PartitionChangeListener(topic: String) extends IZkChildListener with Logging { - this.logIdent = "[Controller " + controller.config.brokerId + "]: " + + class AddPartitionsListener(topic: String) extends IZkDataListener with Logging { + + this.logIdent = "[AddPartitionsListener on " + controller.config.brokerId + "]: " @throws(classOf[Exception]) - def handleChildChange(parentPath : String, children : java.util.List[String]) { + def handleDataChange(dataPath : String, data: Object) { controllerContext.controllerLock synchronized { - // TODO: To be completed as part of KAFKA-41 + try { + info("Add Partition triggered " + data.toString + " for path " + dataPath) + val partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) + val partitionsRemainingToBeAdded = partitionReplicaAssignment.filter(p => + !controllerContext.partitionReplicaAssignment.contains(p._1)) + info("New partitions to be added [%s]".format(partitionsRemainingToBeAdded)) + controller.onNewPartitionCreation(partitionsRemainingToBeAdded.keySet.toSet) + } catch { + case e => error("Error while handling add partitions for data path " + dataPath, e ) + } } } + + @throws(classOf[Exception]) + def handleDataDeleted(parentPath : String) { + // this is not implemented for partition change + } } } diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 0c9d436..c964857 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -87,7 +87,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { try { brokerRequestBatch.newBatch() replicas.foreach(r => handleStateChange(r.topic, r.partition, r.replica, targetState)) - brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement, controllerContext.liveBrokers) + brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) }catch { case e => error("Error while moving some replicas to %s state".format(targetState), e) } @@ -182,7 +182,12 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case None => true } - else false + else { + replicaState.put((topic, partition, replicaId), OfflineReplica) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OfflineReplica" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + false + } case None => true } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 1437496..77d7ec0 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -84,12 +84,20 @@ object RequestChannel extends Logging { } } - case class Response(processor: Int, request: Request, responseSend: Send) { + case class Response(processor: Int, request: Request, responseSend: Send, responseAction: ResponseAction) { request.responseCompleteTimeMs = SystemTime.milliseconds + def this(processor: Int, request: Request, responseSend: Send) = + this(processor, request, responseSend, if (responseSend == null) NoOpAction else SendAction) + def this(request: Request, send: Send) = this(request.processor, request, send) } + + trait ResponseAction + case object SendAction extends ResponseAction + case object NoOpAction extends ResponseAction + case object CloseConnectionAction extends ResponseAction } class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMetricsGroup { @@ -127,6 +135,20 @@ class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMe onResponse(response.processor) } + /** No operation to take for the request, need to read more over the network */ + def noOperation(processor: Int, request: RequestChannel.Request) { + responseQueues(processor).put(new RequestChannel.Response(processor, request, null, RequestChannel.NoOpAction)) + for(onResponse <- responseListeners) + onResponse(processor) + } + + /** Close the connection for the request */ + def closeConnection(processor: Int, request: RequestChannel.Request) { + responseQueues(processor).put(new RequestChannel.Response(processor, request, null, RequestChannel.CloseConnectionAction)) + for(onResponse <- responseListeners) + onResponse(processor) + } + /** Get the next request or block until there is one */ def receiveRequest(): RequestChannel.Request = requestQueue.take() diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 9767f4d..216245d 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -274,18 +274,26 @@ private[kafka] class Processor(val id: Int, while(curr != null) { val key = curr.request.requestKey.asInstanceOf[SelectionKey] try { - if(curr.responseSend == null) { - // a null response send object indicates that there is no response to send to the client. - // In this case, we just want to turn the interest ops to READ to be able to read more pipelined requests - // that are sitting in the server's socket buffer - trace("Socket server received empty response to send, registering for read: " + curr) - key.interestOps(SelectionKey.OP_READ) - key.attach(null) - curr.request.updateRequestMetrics - } else { - trace("Socket server received response to send, registering for write: " + curr) - key.interestOps(SelectionKey.OP_WRITE) - key.attach(curr) + curr.responseAction match { + case RequestChannel.NoOpAction => { + // There is no response to send to the client, we need to read more pipelined requests + // that are sitting in the server's socket buffer + curr.request.updateRequestMetrics + trace("Socket server received empty response to send, registering for read: " + curr) + key.interestOps(SelectionKey.OP_READ) + key.attach(null) + } + case RequestChannel.SendAction => { + trace("Socket server received response to send, registering for write: " + curr) + key.interestOps(SelectionKey.OP_WRITE) + key.attach(curr) + } + case RequestChannel.CloseConnectionAction => { + curr.request.updateRequestMetrics + trace("Closing socket connection actively according to the response code.") + close(key) + } + case responseCode => throw new KafkaException("No mapping found for response code " + responseCode) } } catch { case e: CancelledKeyException => { diff --git a/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala b/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala new file mode 100644 index 0000000..988e437 --- /dev/null +++ b/core/src/main/scala/kafka/producer/ByteArrayPartitioner.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.producer + + +import kafka.utils._ + +private class ByteArrayPartitioner(props: VerifiableProperties = null) extends Partitioner { + def partition(key: Any, numPartitions: Int): Int = { + Utils.abs(java.util.Arrays.hashCode(key.asInstanceOf[Array[Byte]])) % numPartitions + } +} diff --git a/core/src/main/scala/kafka/producer/ConsoleProducer.scala b/core/src/main/scala/kafka/producer/ConsoleProducer.scala index 59222a2..28de573 100644 --- a/core/src/main/scala/kafka/producer/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/producer/ConsoleProducer.scala @@ -44,6 +44,14 @@ object ConsoleProducer { .describedAs("size") .ofType(classOf[java.lang.Integer]) .defaultsTo(200) + val messageSendMaxRetriesOpt = parser.accepts("message-send-max-retries", "Brokers can fail receiving the message for multiple reasons, and being unavailable transiently is just one of them. This property specifies the number of retires before the producer give up and drop this message.") + .withRequiredArg + .ofType(classOf[java.lang.Integer]) + .defaultsTo(3) + val retryBackoffMsOpt = parser.accepts("retry-backoff-ms", "Before each retry, the producer refreshes the metadata of relevant topics. Since leader election takes a bit of time, this property specifies the amount of time that the producer waits before refreshing the metadata.") + .withRequiredArg + .ofType(classOf[java.lang.Long]) + .defaultsTo(100) val sendTimeoutOpt = parser.accepts("timeout", "If set and the producer is running in asynchronous mode, this gives the maximum amount of time" + " a message will queue awaiting suffient batch size. The value is given in ms.") .withRequiredArg @@ -97,7 +105,7 @@ object ConsoleProducer { .withRequiredArg .describedAs("prop") .ofType(classOf[String]) - + val options = parser.parse(args : _*) for(arg <- List(topicOpt, brokerListOpt)) { @@ -132,6 +140,9 @@ object ConsoleProducer { props.put("producer.type", if(sync) "sync" else "async") if(options.has(batchSizeOpt)) props.put("batch.num.messages", batchSize.toString) + + props.put("message.send.max.retries", options.valueOf(messageSendMaxRetriesOpt).toString) + props.put("retry.backoff.ms", options.valueOf(retryBackoffMsOpt).toString) props.put("queue.buffering.max.ms", sendTimeout.toString) props.put("queue.buffering.max.messages", queueSize.toString) props.put("queue.enqueue.timeout.ms", queueEnqueueTimeoutMs.toString) diff --git a/core/src/main/scala/kafka/producer/DefaultPartitioner.scala b/core/src/main/scala/kafka/producer/DefaultPartitioner.scala index 37ddd55..3afb22e 100644 --- a/core/src/main/scala/kafka/producer/DefaultPartitioner.scala +++ b/core/src/main/scala/kafka/producer/DefaultPartitioner.scala @@ -20,7 +20,7 @@ package kafka.producer import kafka.utils._ -private class DefaultPartitioner(props: VerifiableProperties = null) extends Partitioner { +class DefaultPartitioner(props: VerifiableProperties = null) extends Partitioner { private val random = new java.util.Random def partition(key: Any, numPartitions: Int): Int = { diff --git a/core/src/main/scala/kafka/producer/Producer.scala b/core/src/main/scala/kafka/producer/Producer.scala index ba94e87..4798481 100644 --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -33,16 +33,17 @@ class Producer[K,V](val config: ProducerConfig, private val hasShutdown = new AtomicBoolean(false) private val queue = new LinkedBlockingQueue[KeyedMessage[K,V]](config.queueBufferingMaxMessages) - private val random = new Random private var sync: Boolean = true private var producerSendThread: ProducerSendThread[K,V] = null + private val lock = new Object() + config.producerType match { case "sync" => case "async" => sync = false producerSendThread = new ProducerSendThread[K,V]("ProducerSendThread-" + config.clientId, queue, - eventHandler, + eventHandler, config.queueBufferingMaxMs, config.batchNumMessages, config.clientId) @@ -67,12 +68,14 @@ class Producer[K,V](val config: ProducerConfig, * @param messages the producer data object that encapsulates the topic, key and message data */ def send(messages: KeyedMessage[K,V]*) { - if (hasShutdown.get) - throw new ProducerClosedException - recordStats(messages) - sync match { - case true => eventHandler.handle(messages) - case false => asyncSend(messages) + lock synchronized { + if (hasShutdown.get) + throw new ProducerClosedException + recordStats(messages) + sync match { + case true => eventHandler.handle(messages) + case false => asyncSend(messages) + } } } @@ -119,12 +122,14 @@ class Producer[K,V](val config: ProducerConfig, * the zookeeper client connection if one exists */ def close() = { - val canShutdown = hasShutdown.compareAndSet(false, true) - if(canShutdown) { - info("Shutting down producer") - if (producerSendThread != null) - producerSendThread.shutdown - eventHandler.close + lock synchronized { + val canShutdown = hasShutdown.compareAndSet(false, true) + if(canShutdown) { + info("Shutting down producer") + if (producerSendThread != null) + producerSendThread.shutdown + eventHandler.close + } } } } diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index 48ddb6a..65613ce 100644 --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -40,56 +40,55 @@ class DefaultEventHandler[K,V](config: ProducerConfig, val correlationId = new AtomicInteger(0) val brokerPartitionInfo = new BrokerPartitionInfo(config, producerPool, topicPartitionInfos) - private val lock = new Object() - private val topicMetadataRefreshInterval = config.topicMetadataRefreshIntervalMs private var lastTopicMetadataRefreshTime = 0L private val topicMetadataToRefresh = Set.empty[String] + private val sendPartitionPerTopicCache = HashMap.empty[String, Int] private val producerStats = ProducerStatsRegistry.getProducerStats(config.clientId) private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(config.clientId) def handle(events: Seq[KeyedMessage[K,V]]) { - lock synchronized { - val serializedData = serialize(events) - serializedData.foreach { - keyed => - val dataSize = keyed.message.payloadSize - producerTopicStats.getProducerTopicStats(keyed.topic).byteRate.mark(dataSize) - producerTopicStats.getProducerAllTopicsStats.byteRate.mark(dataSize) - } - var outstandingProduceRequests = serializedData - var remainingRetries = config.messageSendMaxRetries + 1 - val correlationIdStart = correlationId.get() - debug("Handling %d events".format(events.size)) - while (remainingRetries > 0 && outstandingProduceRequests.size > 0) { - topicMetadataToRefresh ++= outstandingProduceRequests.map(_.topic) - if (topicMetadataRefreshInterval >= 0 && - SystemTime.milliseconds - lastTopicMetadataRefreshTime > topicMetadataRefreshInterval) { - Utils.swallowError(brokerPartitionInfo.updateInfo(topicMetadataToRefresh.toSet, correlationId.getAndIncrement)) - topicMetadataToRefresh.clear - lastTopicMetadataRefreshTime = SystemTime.milliseconds - } - outstandingProduceRequests = dispatchSerializedData(outstandingProduceRequests) - if (outstandingProduceRequests.size > 0) { - info("Back off for %d ms before retrying send. Remaining retries = %d".format(config.retryBackoffMs, remainingRetries-1)) - // back off and update the topic metadata cache before attempting another send operation - Thread.sleep(config.retryBackoffMs) - // get topics of the outstanding produce requests and refresh metadata for those - Utils.swallowError(brokerPartitionInfo.updateInfo(outstandingProduceRequests.map(_.topic).toSet, correlationId.getAndIncrement)) - remainingRetries -= 1 - producerStats.resendRate.mark() - } + val serializedData = serialize(events) + serializedData.foreach { + keyed => + val dataSize = keyed.message.payloadSize + producerTopicStats.getProducerTopicStats(keyed.topic).byteRate.mark(dataSize) + producerTopicStats.getProducerAllTopicsStats.byteRate.mark(dataSize) + } + var outstandingProduceRequests = serializedData + var remainingRetries = config.messageSendMaxRetries + 1 + val correlationIdStart = correlationId.get() + debug("Handling %d events".format(events.size)) + while (remainingRetries > 0 && outstandingProduceRequests.size > 0) { + topicMetadataToRefresh ++= outstandingProduceRequests.map(_.topic) + if (topicMetadataRefreshInterval >= 0 && + SystemTime.milliseconds - lastTopicMetadataRefreshTime > topicMetadataRefreshInterval) { + Utils.swallowError(brokerPartitionInfo.updateInfo(topicMetadataToRefresh.toSet, correlationId.getAndIncrement)) + sendPartitionPerTopicCache.clear() + topicMetadataToRefresh.clear + lastTopicMetadataRefreshTime = SystemTime.milliseconds } - if(outstandingProduceRequests.size > 0) { - producerStats.failedSendRate.mark() - val correlationIdEnd = correlationId.get() - error("Failed to send requests for topics %s with correlation ids in [%d,%d]" - .format(outstandingProduceRequests.map(_.topic).toSet.mkString(","), - correlationIdStart, correlationIdEnd-1)) - throw new FailedToSendMessageException("Failed to send messages after " + config.messageSendMaxRetries + " tries.", null) + outstandingProduceRequests = dispatchSerializedData(outstandingProduceRequests) + if (outstandingProduceRequests.size > 0) { + info("Back off for %d ms before retrying send. Remaining retries = %d".format(config.retryBackoffMs, remainingRetries-1)) + // back off and update the topic metadata cache before attempting another send operation + Thread.sleep(config.retryBackoffMs) + // get topics of the outstanding produce requests and refresh metadata for those + Utils.swallowError(brokerPartitionInfo.updateInfo(outstandingProduceRequests.map(_.topic).toSet, correlationId.getAndIncrement)) + sendPartitionPerTopicCache.clear() + remainingRetries -= 1 + producerStats.resendRate.mark() } } + if(outstandingProduceRequests.size > 0) { + producerStats.failedSendRate.mark() + val correlationIdEnd = correlationId.get() + error("Failed to send requests for topics %s with correlation ids in [%d,%d]" + .format(outstandingProduceRequests.map(_.topic).toSet.mkString(","), + correlationIdStart, correlationIdEnd-1)) + throw new FailedToSendMessageException("Failed to send messages after " + config.messageSendMaxRetries + " tries.", null) + } } private def dispatchSerializedData(messages: Seq[KeyedMessage[K,Message]]): Seq[KeyedMessage[K, Message]] = { @@ -207,18 +206,29 @@ class DefaultEventHandler[K,V](config: ProducerConfig, throw new UnknownTopicOrPartitionException("Topic " + topic + " doesn't exist") val partition = if(key == null) { - // If the key is null, we don't really need a partitioner so we just send to the next - // available partition - val availablePartitions = topicPartitionList.filter(_.leaderBrokerIdOpt.isDefined) - if (availablePartitions.isEmpty) - throw new LeaderNotAvailableException("No leader for any partition in topic " + topic) - val index = Utils.abs(partitionCounter.getAndIncrement()) % availablePartitions.size - availablePartitions(index).partitionId + // If the key is null, we don't really need a partitioner + // So we look up in the send partition cache for the topic to decide the target partition + val id = sendPartitionPerTopicCache.get(topic) + id match { + case Some(partitionId) => + // directly return the partitionId without checking availability of the leader, + // since we want to postpone the failure until the send operation anyways + partitionId + case None => + val availablePartitions = topicPartitionList.filter(_.leaderBrokerIdOpt.isDefined) + if (availablePartitions.isEmpty) + throw new LeaderNotAvailableException("No leader for any partition in topic " + topic) + val index = Utils.abs(partitionCounter.getAndIncrement()) % availablePartitions.size + val partitionId = availablePartitions(index).partitionId + sendPartitionPerTopicCache.put(topic, partitionId) + partitionId + } } else partitioner.partition(key, numPartitions) if(partition < 0 || partition >= numPartitions) throw new UnknownTopicOrPartitionException("Invalid partition id: " + partition + " for topic " + topic + "; Valid values are in the inclusive range of [0, " + (numPartitions-1) + "]") + trace("Assigning message of topic %s and key %s to a selected partition %d".format(topic, if (key == null) "[none]" else key.toString, partition)) partition } diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 7663fac..d5addb3 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -144,17 +144,17 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke try { val newOffset = handleOffsetOutOfRange(topicAndPartition) partitionMap.put(topicAndPartition, newOffset) - warn("current offset %d for partition [%s,%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 partition [%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 _ => if (isRunning.get) { - warn("error for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), - ErrorMapping.exceptionFor(partitionData.error)) + warn("Error for partition [%s,%d] to broker %d:%s".format(topic, partitionId, sourceBroker.id, + ErrorMapping.exceptionFor(partitionData.error).getClass)) partitionsWithError += topicAndPartition } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 0ec031a..0287f87 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -56,8 +56,7 @@ class KafkaApis(val requestChannel: RequestChannel, * and is queried by the topic metadata request. */ var leaderCache: mutable.Map[TopicAndPartition, PartitionStateInfo] = new mutable.HashMap[TopicAndPartition, PartitionStateInfo]() -// private var allBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]() - private var aliveBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]() + private val aliveBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]() private val partitionMetadataLock = new Object this.logIdent = "[KafkaApi-%d] ".format(brokerId) @@ -177,9 +176,17 @@ class KafkaApis(val requestChannel: RequestChannel, !produceRequest.data.keySet.exists( m => replicaManager.getReplicationFactorForPartition(m.topic, m.partition) != 1) if(produceRequest.requiredAcks == 0) { - // send a fake producer response if producer request.required.acks = 0. This mimics the behavior of a 0.7 producer - // and is tuned for very high throughput - requestChannel.sendResponse(new RequestChannel.Response(request.processor, request, null)) + // no operation needed if producer request.required.acks = 0; however, if there is any exception in handling the request, since + // no response is expected by the producer the handler will send a close connection response to the socket server + // to close the socket so that the producer client will know that some exception has happened and will refresh its metadata + if (numPartitionsInError != 0) { + info(("Send the close connection response due to error handling produce request " + + "[clientId = %s, correlationId = %s, topicAndPartition = %s] with Ack=0") + .format(produceRequest.clientId, produceRequest.correlationId, produceRequest.topicPartitionMessageSizeMap.keySet.mkString(","))) + requestChannel.closeConnection(request.processor, request) + } else { + requestChannel.noOperation(request.processor, request) + } } else if (produceRequest.requiredAcks == 1 || produceRequest.numPartitions <= 0 || allPartitionHaveReplicationFactorOne || @@ -363,8 +370,7 @@ class KafkaApis(val requestChannel: RequestChannel, BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark() 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) + .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) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index ebbbdea..ebd171f 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -18,7 +18,7 @@ package kafka.server import java.util.Properties -import kafka.message.Message +import kafka.message.{MessageSet, Message} import kafka.consumer.ConsumerConfig import kafka.utils.{VerifiableProperties, ZKConfig, Utils} @@ -38,7 +38,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val brokerId: Int = props.getIntInRange("broker.id", (0, Int.MaxValue)) /* the maximum size of message that the server can receive */ - val messageMaxBytes = props.getIntInRange("message.max.bytes", 1000000, (0, Int.MaxValue)) + val messageMaxBytes = props.getIntInRange("message.max.bytes", 1000000 + MessageSet.LogOverhead, (0, Int.MaxValue)) /* the number of network threads that the server uses for handling network requests */ val numNetworkThreads = props.getIntInRange("num.network.threads", 3, (1, Int.MaxValue)) diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index a078707..84ea17a 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -34,7 +34,8 @@ import java.net.InetAddress */ class KafkaHealthcheck(private val brokerId: Int, private val host: String, - private val port: Int, + private val port: Int, + private val zkSessionTimeoutMs: Int, private val zkClient: ZkClient) extends Logging { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId @@ -54,7 +55,7 @@ class KafkaHealthcheck(private val brokerId: Int, else host val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt - ZkUtils.registerBrokerInZk(zkClient, brokerId, hostName, port, jmxPort) + ZkUtils.registerBrokerInZk(zkClient, brokerId, hostName, port, zkSessionTimeoutMs, jmxPort) } /** diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index 34342f5..6d562c2 100644 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -41,7 +41,7 @@ class KafkaRequestHandler(id: Int, brokerId: Int, val requestChannel: RequestCha trace("Kafka request handler %d on broker %d handling request %s".format(id, brokerId, req)) apis.handle(req) } catch { - case e: Throwable => error("Exception when handling request") + case e: Throwable => error("Exception when handling request", e) } } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index a925ae1..c148fdf 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -99,7 +99,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg topicConfigManager.startup() /* tell everyone we are alive */ - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.hostName, config.port, zkClient) + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.hostName, config.port, config.zkSessionTimeoutMs, zkClient) kafkaHealthcheck.startup() diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 73c87c6..3ca4419 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -222,7 +222,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.aliveLeaders, + makeFollower(controllerId, controllerEpoch, topic, partitionId, partitionStateInfo, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId) } catch { case e => @@ -266,14 +266,14 @@ class ReplicaManager(val config: KafkaConfig, } private def makeFollower(controllerId: Int, epoch: Int, topic: String, partitionId: Int, - partitionStateInfo: PartitionStateInfo, aliveLeaders: Set[Broker], correlationId: Int) { + partitionStateInfo: PartitionStateInfo, leaders: Set[Broker], correlationId: Int) { val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch 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, aliveLeaders, correlationId)) { + if (partition.makeFollower(controllerId, topic, partitionId, leaderIsrAndControllerEpoch, leaders, correlationId)) { // remove this replica's partition from the ISR expiration queue leaderPartitionsLock synchronized { leaderPartitions -= partition diff --git a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala index 574922b..f1f0625 100644 --- a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala +++ b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala @@ -17,10 +17,12 @@ package kafka.server import kafka.utils.ZkUtils._ -import kafka.utils.Logging +import kafka.utils.{Utils, SystemTime, Logging} import org.I0Itec.zkclient.exception.ZkNodeExistsException import org.I0Itec.zkclient.IZkDataListener import kafka.controller.ControllerContext +import kafka.controller.KafkaController +import kafka.common.KafkaException /** * This class handles zookeeper based leader election based on an ephemeral path. The election module does not handle @@ -40,29 +42,41 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, electionPath: def startup { controllerContext.controllerLock synchronized { + controllerContext.zkClient.subscribeDataChanges(electionPath, leaderChangeListener) elect } } def elect: Boolean = { - controllerContext.zkClient.subscribeDataChanges(electionPath, leaderChangeListener) + val timestamp = SystemTime.milliseconds.toString + val electString = + Utils.mergeJsonFields(Utils.mapToJsonFields(Map("version" -> 1.toString, "brokerid" -> brokerId.toString), valueInQuotes = false) + ++ Utils.mapToJsonFields(Map("timestamp" -> timestamp), valueInQuotes = true)) + try { - createEphemeralPathExpectConflict(controllerContext.zkClient, electionPath, brokerId.toString) + createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, electionPath, electString, brokerId, + (controllerString : String, leaderId : Any) => KafkaController.parseControllerId(controllerString) == leaderId.asInstanceOf[Int], + controllerContext.zkSessionTimeout) info(brokerId + " successfully elected as leader") leaderId = brokerId onBecomingLeader() - } catch { - case e: ZkNodeExistsException => - // If someone else has written the path, then - val data: String = controllerContext.zkClient.readData(electionPath, true) - debug("Broker %d was elected as leader instead of broker %d".format(data.toInt, brokerId)) - if (data != null) { - leaderId = data.toInt - } - case e2 => - error("Error while electing or becoming leader on broker %d".format(brokerId), e2) - resign() + } catch { + case e: ZkNodeExistsException => + // If someone else has written the path, then + leaderId = readDataMaybeNull(controllerContext.zkClient, electionPath)._1 match { + case Some(controller) => KafkaController.parseControllerId(controller) + case None => { + warn("A leader has been elected but just resigned, this will result in another round of election") + -1 + } + } + if (leaderId != -1) + debug("Broker %d was elected as leader instead of broker %d".format(leaderId, brokerId)) + case e2 => + error("Error while electing or becoming leader on broker %d".format(brokerId), e2) + leaderId = -1 } + amILeader } @@ -73,6 +87,7 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, electionPath: def amILeader : Boolean = leaderId == brokerId def resign() = { + leaderId = -1 deletePath(controllerContext.zkClient, electionPath) } @@ -88,7 +103,7 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, electionPath: @throws(classOf[Exception]) def handleDataChange(dataPath: String, data: Object) { controllerContext.controllerLock synchronized { - leaderId = data.toString.toInt + leaderId = KafkaController.parseControllerId(data.toString) info("New leader is %d".format(leaderId)) } } diff --git a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala index 63519e1..55709b5 100644 --- a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala +++ b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala @@ -96,9 +96,6 @@ object ImportZkOffsets extends Logging { } private def updateZkOffsets(zkClient: ZkClient, partitionOffsets: Map[String,String]): Unit = { - val cluster = ZkUtils.getCluster(zkClient) - var partitions: List[String] = Nil - for ((partition, offset) <- partitionOffsets) { debug("updating [" + partition + "] with offset [" + offset + "]") diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 2d93947..6fb545a 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -22,6 +22,7 @@ import kafka.utils.{Utils, CommandLineUtils, Logging} import kafka.producer.{KeyedMessage, ProducerConfig, Producer} import scala.collection.JavaConversions._ import java.util.concurrent.CountDownLatch +import java.nio.ByteBuffer import kafka.consumer._ import kafka.serializer._ import collection.mutable.ListBuffer @@ -30,6 +31,10 @@ import kafka.javaapi object MirrorMaker extends Logging { + private var connectors: Seq[ZookeeperConsumerConnector] = null + private var consumerThreads: Seq[MirrorMakerThread] = null + private var producerThreads: ListBuffer[ProducerThread] = null + def main(args: Array[String]) { info ("Starting mirror maker") @@ -99,12 +104,19 @@ object MirrorMaker extends Logging { val bufferSize = options.valueOf(bufferSizeOpt).intValue() val producers = (1 to options.valueOf(numProducersOpt).intValue()).map(_ => { - val config = new ProducerConfig( - Utils.loadProps(options.valueOf(producerConfigOpt))) + val props = Utils.loadProps(options.valueOf(producerConfigOpt)) + val config = props.getProperty("partitioner.class") match { + case null => + new ProducerConfig(props) { + override val partitionerClass = "kafka.producer.ByteArrayPartitioner" + } + case pClass : String => + new ProducerConfig(props) + } new Producer[Array[Byte], Array[Byte]](config) }) - val connectors = options.valuesOf(consumerConfigOpt).toList + connectors = options.valuesOf(consumerConfigOpt).toList .map(cfg => new ConsumerConfig(Utils.loadProps(cfg.toString))) .map(new ZookeeperConsumerConnector(_)) @@ -124,18 +136,9 @@ object MirrorMaker extends Logging { val producerDataChannel = new ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]](bufferSize); - val consumerThreads = - streams.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producerDataChannel, streamAndIndex._2)) - - val producerThreads = new ListBuffer[ProducerThread]() + consumerThreads = streams.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producerDataChannel, producers, streamAndIndex._2)) - def cleanShutdown() { - connectors.foreach(_.shutdown) - consumerThreads.foreach(_.awaitShutdown) - producerThreads.foreach(_.shutdown) - producerThreads.foreach(_.awaitShutdown) - info("Kafka mirror maker shutdown successfully") - } + producerThreads = new ListBuffer[ProducerThread]() Runtime.getRuntime.addShutdownHook(new Thread() { override def run() { @@ -160,13 +163,25 @@ object MirrorMaker extends Logging { cleanShutdown() } + def cleanShutdown() { + if (connectors != null) connectors.foreach(_.shutdown) + if (consumerThreads != null) consumerThreads.foreach(_.awaitShutdown) + if (producerThreads != null) { + producerThreads.foreach(_.shutdown) + producerThreads.foreach(_.awaitShutdown) + } + info("Kafka mirror maker shutdown successfully") + } + class MirrorMakerThread(stream: KafkaStream[Array[Byte], Array[Byte]], producerDataChannel: ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]], + producers: Seq[Producer[Array[Byte], Array[Byte]]], threadId: Int) extends Thread with Logging { private val shutdownLatch = new CountDownLatch(1) private val threadName = "mirrormaker-" + threadId + this.logIdent = "[%s] ".format(threadName) this.setName(threadName) @@ -174,15 +189,26 @@ object MirrorMaker extends Logging { info("Starting mirror maker thread " + threadName) try { for (msgAndMetadata <- stream) { - val pd = new KeyedMessage[Array[Byte], Array[Byte]](msgAndMetadata.topic, msgAndMetadata.message) - producerDataChannel.sendRequest(pd) + // If the key of the message is empty, put it into the universal channel + // Otherwise use a pre-assigned producer to send the message + if (msgAndMetadata.key == null) { + trace("Send the non-keyed message the producer channel.") + val pd = new KeyedMessage[Array[Byte], Array[Byte]](msgAndMetadata.topic, msgAndMetadata.message) + producerDataChannel.sendRequest(pd) + } else { + val producerId = Utils.abs(java.util.Arrays.hashCode(msgAndMetadata.key)) % producers.size() + trace("Send message with key %s to producer %d.".format(java.util.Arrays.toString(msgAndMetadata.key), producerId)) + val producer = producers(producerId) + val pd = new KeyedMessage[Array[Byte], Array[Byte]](msgAndMetadata.topic, msgAndMetadata.key, msgAndMetadata.message) + producer.send(pd) + } } } catch { case e => - fatal("%s stream unexpectedly exited.", e) + fatal("Stream unexpectedly exited.", e) } finally { shutdownLatch.countDown() - info("Stopped thread %s.".format(threadName)) + info("Stopped thread.") } } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 84744eb..4094dcb 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -31,6 +31,7 @@ import kafka.admin._ import kafka.common.{KafkaException, NoEpochForPartitionException} import kafka.controller.ReassignedPartitionsContext import kafka.controller.PartitionAndReplica +import kafka.controller.KafkaController import scala.Some import kafka.controller.LeaderIsrAndControllerEpoch import kafka.common.TopicAndPartition @@ -59,7 +60,7 @@ object ZkUtils extends Logging { def getController(zkClient: ZkClient): Int= { readDataMaybeNull(zkClient, ControllerPath)._1 match { - case Some(controller) => controller.toInt + case Some(controller) => KafkaController.parseControllerId(controller) case None => throw new KafkaException("Controller doesn't exist") } } @@ -187,18 +188,27 @@ object ZkUtils extends Logging { debug("The list of replicas for partition [%s,%d] is %s".format(topic, partition, replicas)) replicas.contains(brokerId.toString) } - - def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, jmxPort: Int) { + + def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id + val timestamp = "\"" + SystemTime.milliseconds.toString + "\"" val brokerInfo = Utils.mergeJsonFields(Utils.mapToJsonFields(Map("host" -> host), valueInQuotes = true) ++ - Utils.mapToJsonFields(Map("version" -> 1.toString, "jmx_port" -> jmxPort.toString, "port" -> port.toString), + Utils.mapToJsonFields(Map("version" -> 1.toString, "jmx_port" -> jmxPort.toString, "port" -> port.toString, "timestamp" -> timestamp), valueInQuotes = false)) + val expectedBroker = new Broker(id, host, port) + try { - createEphemeralPathExpectConflict(zkClient, brokerIdPath, brokerInfo) + createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, + (brokerString: String, broker: Any) => Broker.createBroker(broker.asInstanceOf[Broker].id, brokerString).equals(broker.asInstanceOf[Broker]), + timeout) + } catch { case e: ZkNodeExistsException => - throw new RuntimeException("A broker is already registered on the path " + brokerIdPath + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " + "else you have shutdown this broker and restarted it faster than the zookeeper " + "timeout so it appears to be re-registering.") + throw new RuntimeException("A broker is already registered on the path " + brokerIdPath + + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " + + "else you have shutdown this broker and restarted it faster than the zookeeper " + + "timeout so it appears to be re-registering.") } info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, host, port)) } @@ -283,6 +293,47 @@ object ZkUtils extends Logging { } /** + * Create an ephemeral node with the given path and data. + * Throw NodeExistsException if node already exists. + * Handles the following ZK session timeout bug: + * + * https://issues.apache.org/jira/browse/ZOOKEEPER-1740 + * + * Upon receiving a NodeExistsException, read the data from the conflicted path and + * trigger the checker function comparing the read data and the expected data, + * If the checker function returns true then the above bug might be encountered, back off and retry; + * otherwise re-throw the exception + */ + def createEphemeralPathExpectConflictHandleZKBug(zkClient: ZkClient, path: String, data: String, expectedCallerData: Any, checker: (String, Any) => Boolean, backoffTime: Int): Unit = { + while (true) { + try { + createEphemeralPathExpectConflict(zkClient, path, data) + return + } catch { + case e: ZkNodeExistsException => { + // An ephemeral node may still exist even after its corresponding session has expired + // due to a Zookeeper bug, in this case we need to retry writing until the previous node is deleted + // and hence the write succeeds without ZkNodeExistsException + ZkUtils.readDataMaybeNull(zkClient, path)._1 match { + case Some(writtenData) => { + if (checker(writtenData, expectedCallerData)) { + info("I wrote this conflicted ephemeral node [%s] at %s a while back in a different session, ".format(data, path) + + "hence I will backoff for this node to be deleted by Zookeeper and retry") + + Thread.sleep(backoffTime) + } else { + throw e + } + } + case None => // the node disappeared; retry creating the ephemeral node immediately + } + } + case e2 => throw e2 + } + } + } + + /** * Create an persistent node with the given path and data. Create parents if necessary. */ def createPersistentPath(client: ZkClient, path: String, data: String = ""): Unit = { @@ -585,6 +636,24 @@ object ZkUtils extends Logging { reassignedPartitions } + def parseTopicsData(jsonData: String): Seq[String] = { + var topics = List.empty[String] + Json.parseFull(jsonData) match { + case Some(m) => + m.asInstanceOf[Map[String, Any]].get("topics") match { + case Some(partitionsSeq) => + val mapPartitionSeq = partitionsSeq.asInstanceOf[Seq[Map[String, Any]]] + mapPartitionSeq.foreach(p => { + val topic = p.get("topic").get.asInstanceOf[String] + topics ++= List(topic) + }) + case None => + } + case None => + } + topics + } + def getPartitionReassignmentZkData(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]): String = { var jsonPartitionsData: mutable.ListBuffer[String] = ListBuffer[String]() for (p <- partitionsToBeReassigned) { diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala new file mode 100644 index 0000000..abcbed8 --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -0,0 +1,251 @@ +/** + * 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 org.scalatest.junit.JUnit3Suite +import kafka.zk.ZooKeeperTestHarness +import kafka.utils.TestUtils._ +import junit.framework.Assert._ +import kafka.utils.{ZkUtils, Utils, TestUtils} +import kafka.cluster.Broker +import kafka.client.ClientUtils +import kafka.server.{KafkaConfig, KafkaServer} + +class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { + val brokerId1 = 0 + val brokerId2 = 1 + val brokerId3 = 2 + val brokerId4 = 3 + + val port1 = TestUtils.choosePort() + val port2 = TestUtils.choosePort() + val port3 = TestUtils.choosePort() + val port4 = TestUtils.choosePort() + + val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) + val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) + val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3) + val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4) + + var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] + var brokers: Seq[Broker] = Seq.empty[Broker] + + val partitionId = 0 + + val topic1 = "new-topic1" + val topic2 = "new-topic2" + val topic3 = "new-topic3" + val topic4 = "new-topic4" + + override def setUp() { + super.setUp() + // start all the servers + val server1 = TestUtils.createServer(new KafkaConfig(configProps1)) + val server2 = TestUtils.createServer(new KafkaConfig(configProps2)) + val server3 = TestUtils.createServer(new KafkaConfig(configProps3)) + val server4 = TestUtils.createServer(new KafkaConfig(configProps4)) + + servers ++= List(server1, server2, server3, server4) + brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.config.port)) + + // create topics with 1 partition, 2 replicas, one on each broker + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic1, Map(0->Seq(0,1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic2, Map(0->Seq(1,2))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic3, Map(0->Seq(2,3,0,1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic4, Map(0->Seq(0,3))) + + + // wait until leader is elected + var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partitionId, 500) + var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, partitionId, 500) + var leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, partitionId, 500) + var leader4 = waitUntilLeaderIsElectedOrChanged(zkClient, topic4, partitionId, 500) + + debug("Leader for " + topic1 + " is elected to be: %s".format(leader1.getOrElse(-1))) + debug("Leader for " + topic2 + " is elected to be: %s".format(leader1.getOrElse(-1))) + debug("Leader for " + topic3 + "is elected to be: %s".format(leader1.getOrElse(-1))) + debug("Leader for " + topic4 + "is elected to be: %s".format(leader1.getOrElse(-1))) + + assertTrue("Leader should get elected", leader1.isDefined) + assertTrue("Leader should get elected", leader2.isDefined) + assertTrue("Leader should get elected", leader3.isDefined) + assertTrue("Leader should get elected", leader4.isDefined) + + assertTrue("Leader could be broker 0 or broker 1 for " + topic1, (leader1.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 1)) + assertTrue("Leader could be broker 1 or broker 2 for " + topic2, (leader2.getOrElse(-1) == 1) || (leader1.getOrElse(-1) == 2)) + assertTrue("Leader could be broker 2 or broker 3 for " + topic3, (leader3.getOrElse(-1) == 2) || (leader1.getOrElse(-1) == 3)) + assertTrue("Leader could be broker 3 or broker 4 for " + topic4, (leader4.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 3)) + } + + override def tearDown() { + servers.map(server => server.shutdown()) + servers.map(server => Utils.rm(server.config.logDirs)) + super.tearDown() + } + + def testTopicDoesNotExist { + try { + AddPartitionsCommand.addPartitions(zkClient, "Blah", 1) + fail("Topic should not exist") + } catch { + case e: AdminOperationException => //this is good + case e2 => throw e2 + } + } + + def testWrongReplicaCount { + try { + AddPartitionsCommand.addPartitions(zkClient, topic1, 2, "0:1:2") + fail("Add partitions should fail") + } catch { + case e: AdminOperationException => //this is good + case e2 => throw e2 + } + } + + def testIncrementPartitions { + AddPartitionsCommand.addPartitions(zkClient, topic1, 2) + // wait until leader is elected + var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1, 500) + var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2, 500) + val leader1FromZk = ZkUtils.getLeaderForPartition(zkClient, topic1, 1).get + val leader2FromZk = ZkUtils.getLeaderForPartition(zkClient, topic1, 2).get + assertEquals(leader1.get, leader1FromZk) + assertEquals(leader2.get, leader2FromZk) + + // read metadata from a broker and verify the new topic partitions exist + TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 1, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 2, 1000) + val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers, "AddPartitionsTest-testIncrementPartitions", + 2000,0).topicsMetadata + val metaDataForTopic1 = metadata.filter(p => p.topic.equals(topic1)) + val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata + assertEquals(partitionDataForTopic1.size, 3) + assertEquals(partitionDataForTopic1(1).partitionId, 1) + assertEquals(partitionDataForTopic1(2).partitionId, 2) + val replicas = partitionDataForTopic1(1).replicas + assertEquals(replicas.size, 2) + assert(replicas.contains(partitionDataForTopic1(1).leader.get)) + } + + def testManualAssignmentOfReplicas { + AddPartitionsCommand.addPartitions(zkClient, topic2, 2, "0:1,2:3") + // wait until leader is elected + var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 1, 500) + var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 2, 500) + val leader1FromZk = ZkUtils.getLeaderForPartition(zkClient, topic2, 1).get + val leader2FromZk = ZkUtils.getLeaderForPartition(zkClient, topic2, 2).get + assertEquals(leader1.get, leader1FromZk) + assertEquals(leader2.get, leader2FromZk) + + // read metadata from a broker and verify the new topic partitions exist + TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 1, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 2, 1000) + val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers, "AddPartitionsTest-testManualAssignmentOfReplicas", + 2000,0).topicsMetadata + val metaDataForTopic2 = metadata.filter(p => p.topic.equals(topic2)) + val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata + assertEquals(partitionDataForTopic2.size, 3) + assertEquals(partitionDataForTopic2(1).partitionId, 1) + assertEquals(partitionDataForTopic2(2).partitionId, 2) + val replicas = partitionDataForTopic2(1).replicas + assertEquals(replicas.size, 2) + assert(replicas(0).id == 0 || replicas(0).id == 1) + assert(replicas(1).id == 0 || replicas(1).id == 1) + } + + def testReplicaPlacement { + AddPartitionsCommand.addPartitions(zkClient, topic3, 6) + // wait until leader is elected + var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 1, 500) + var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 2, 500) + var leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 3, 500) + var leader4 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 4, 500) + var leader5 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 5, 500) + var leader6 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 6, 500) + + val leader1FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 1).get + val leader2FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 2).get + val leader3FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 3).get + val leader4FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 4).get + val leader5FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 5).get + val leader6FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 6).get + + assertEquals(leader1.get, leader1FromZk) + assertEquals(leader2.get, leader2FromZk) + assertEquals(leader3.get, leader3FromZk) + assertEquals(leader4.get, leader4FromZk) + assertEquals(leader5.get, leader5FromZk) + assertEquals(leader6.get, leader6FromZk) + + // read metadata from a broker and verify the new topic partitions exist + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 1, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 2, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 3, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 4, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 5, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 6, 1000) + + val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers, "AddPartitionsTest-testReplicaPlacement", + 2000,0).topicsMetadata + + val metaDataForTopic3 = metadata.filter(p => p.topic.equals(topic3)).head + val partition1DataForTopic3 = metaDataForTopic3.partitionsMetadata(1) + val partition2DataForTopic3 = metaDataForTopic3.partitionsMetadata(2) + val partition3DataForTopic3 = metaDataForTopic3.partitionsMetadata(3) + val partition4DataForTopic3 = metaDataForTopic3.partitionsMetadata(4) + val partition5DataForTopic3 = metaDataForTopic3.partitionsMetadata(5) + val partition6DataForTopic3 = metaDataForTopic3.partitionsMetadata(6) + + assertEquals(partition1DataForTopic3.replicas.size, 4) + assertEquals(partition1DataForTopic3.replicas(0).id, 3) + assertEquals(partition1DataForTopic3.replicas(1).id, 2) + assertEquals(partition1DataForTopic3.replicas(2).id, 0) + assertEquals(partition1DataForTopic3.replicas(3).id, 1) + + assertEquals(partition2DataForTopic3.replicas.size, 4) + assertEquals(partition2DataForTopic3.replicas(0).id, 0) + assertEquals(partition2DataForTopic3.replicas(1).id, 3) + assertEquals(partition2DataForTopic3.replicas(2).id, 1) + assertEquals(partition2DataForTopic3.replicas(3).id, 2) + + assertEquals(partition3DataForTopic3.replicas.size, 4) + assertEquals(partition3DataForTopic3.replicas(0).id, 1) + assertEquals(partition3DataForTopic3.replicas(1).id, 0) + assertEquals(partition3DataForTopic3.replicas(2).id, 2) + assertEquals(partition3DataForTopic3.replicas(3).id, 3) + + assertEquals(partition4DataForTopic3.replicas.size, 4) + assertEquals(partition4DataForTopic3.replicas(0).id, 2) + assertEquals(partition4DataForTopic3.replicas(1).id, 3) + assertEquals(partition4DataForTopic3.replicas(2).id, 0) + assertEquals(partition4DataForTopic3.replicas(3).id, 1) + + assertEquals(partition5DataForTopic3.replicas.size, 4) + assertEquals(partition5DataForTopic3.replicas(0).id, 3) + assertEquals(partition5DataForTopic3.replicas(1).id, 0) + assertEquals(partition5DataForTopic3.replicas(2).id, 1) + assertEquals(partition5DataForTopic3.replicas(3).id, 2) + + assertEquals(partition6DataForTopic3.replicas.size, 4) + assertEquals(partition6DataForTopic3.replicas(0).id, 0) + assertEquals(partition6DataForTopic3.replicas(1).id, 1) + assertEquals(partition6DataForTopic3.replicas(2).id, 2) + assertEquals(partition6DataForTopic3.replicas(3).id, 3) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index cc394a3..b4f4c58 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -69,18 +69,18 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // duplicate brokers intercept[IllegalArgumentException] { - AdminUtils.createTopicWithAssignment(zkClient, "test", Map(0->Seq(0,0))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,0))) } // inconsistent replication factor intercept[IllegalArgumentException] { - AdminUtils.createTopicWithAssignment(zkClient, "test", Map(0->Seq(0,1), 1->Seq(0))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,1), 1->Seq(0))) } // good assignment val assignment = Map(0 -> List(0, 1, 2), 1 -> List(1, 2, 3)) - AdminUtils.createTopicWithAssignment(zkClient, "test", assignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", assignment) val found = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq("test")) assertEquals(assignment, found("test")) } @@ -118,7 +118,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val topic = "test" TestUtils.createBrokersInZk(zkClient, List(0, 1, 2, 3, 4)) // create the topic - AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // create leaders for all partitions TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1) val actualReplicaList = leaderForPartitionMap.keys.toArray.map(p => (p -> ZkUtils.getReplicasForPartition(zkClient, topic, p))).toMap @@ -128,7 +128,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { intercept[TopicExistsException] { // shouldn't be able to create a topic that already exists - AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) } } @@ -139,7 +139,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) // create the topic - AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 val newReplicas = Seq(0, 2, 3) val partitionToBeReassigned = 0 @@ -164,7 +164,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) // create the topic - AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 val newReplicas = Seq(1, 2, 3) val partitionToBeReassigned = 0 @@ -190,7 +190,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) // create the topic - AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 val newReplicas = Seq(2, 3) val partitionToBeReassigned = 0 @@ -231,7 +231,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1)) val topic = "test" // create the topic - AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // put the partition in the reassigned path as well // reassign partition 0 val newReplicas = Seq(0, 1) @@ -270,7 +270,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val serverConfigs = TestUtils.createBrokerConfigs(3).map(new KafkaConfig(_)) // create the topic - AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) // broker 2 should be the leader since it was started first val currentLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, 1000, None).get @@ -291,7 +291,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val serverConfigs = TestUtils.createBrokerConfigs(3).map(new KafkaConfig(_)) val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) // create the topic - AdminUtils.createTopicWithAssignment(zkClient, topic, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) TestUtils.waitUntilMetadataIsPropagated(servers, topic, partition, 1000) val controllerId = ZkUtils.getController(zkClient) diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index b43b063..ef1de83 100644 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -61,7 +61,7 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { override def setUp() { super.setUp - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0 -> Seq(configs.head.brokerId)), new Properties) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(configs.head.brokerId)), new Properties) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) } diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index b1d56d6..47130d3 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -55,7 +55,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { override def setUp() { super.setUp - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0 -> Seq(configs.head.brokerId))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(configs.head.brokerId))) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient) fetcher.stopConnections() diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala index cb0e349..b585f0e 100644 --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -79,10 +79,10 @@ class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { val topic4 = "new-topic4" // create topics with 1 partition, 2 replicas, one on each broker - AdminUtils.createTopicWithAssignment(zkClient, topic1, Map(0->Seq(0,1))) - AdminUtils.createTopicWithAssignment(zkClient, topic2, Map(0->Seq(1,2))) - AdminUtils.createTopicWithAssignment(zkClient, topic3, Map(0->Seq(2,3))) - AdminUtils.createTopicWithAssignment(zkClient, topic4, Map(0->Seq(0,3))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic1, Map(0->Seq(0,1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic2, Map(0->Seq(1,2))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic3, Map(0->Seq(2,3))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic4, Map(0->Seq(0,3))) // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partitionId, 500) diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 72eccc1..f546c15 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -17,6 +17,7 @@ package kafka.producer +import org.scalatest.TestFailedException import org.scalatest.junit.JUnit3Suite import kafka.consumer.SimpleConsumer import kafka.message.Message @@ -206,7 +207,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val topic = "new-topic" // create topic - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0))) // waiting for 1 partition is enough TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) @@ -233,7 +234,8 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ producer.send(new KeyedMessage[String, String](topic, "test", "test1")) fail("Should fail since no leader exists for the partition.") } catch { - case e => // success + case e : TestFailedException => throw e // catch and re-throw the failure message + case e2 => // otherwise success } // restart server 1 @@ -268,7 +270,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val topic = "new-topic" // create topics in ZK - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(0,1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index bbf0406..6fa1abc 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -114,6 +114,33 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { } @Test + def testMessageSizeTooLargeWithAckZero() { + val server = servers.head + + val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + props.put("request.required.acks", "0") + + val producer = new SyncProducer(new SyncProducerConfig(props)) + AdminUtils.createTopic(zkClient, "test", 1, 1) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "test", 0, 500) + + // This message will be dropped silently since message size too large. + producer.send(TestUtils.produceRequest("test", 0, + new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(new Array[Byte](configs(0).messageMaxBytes + 1))), acks = 0)) + + // Send another message whose size is large enough to exceed the buffer size so + // the socket buffer will be flushed immediately; + // this send should fail since the socket has been closed + try { + producer.send(TestUtils.produceRequest("test", 0, + new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(new Array[Byte](configs(0).messageMaxBytes + 1))), acks = 0)) + } catch { + case e : java.io.IOException => // success + case e2 => throw e2 + } + } + + @Test def testProduceCorrectlyReceivesResponse() { val server = servers.head val props = TestUtils.getSyncProducerConfig(server.socketServer.port) diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index bb6fbdf..38e3ae7 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -61,7 +61,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val partitionId = 0 // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0 -> Seq(0, 1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(0, 1))) // wait until leader is elected val leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) @@ -108,7 +108,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val partitionId = 0 // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0 -> Seq(0, 1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(0, 1))) // wait until leader is elected val leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) @@ -124,7 +124,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val controllerId = 2 val controllerConfig = new KafkaConfig(TestUtils.createBrokerConfig(controllerId, TestUtils.choosePort())) val brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.config.port)) - val controllerContext = new ControllerContext(zkClient) + val controllerContext = new ControllerContext(zkClient, 6000) controllerContext.liveBrokers = brokers.toSet val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) controllerChannelManager.startup() diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index d2650e3..34e39e7 100644 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -71,7 +71,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { producer = new Producer[Int, String](new ProducerConfig(producerProps)) // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(0,1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) // wait until leader is elected var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) @@ -104,7 +104,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { producer = new Producer[Int, String](new ProducerConfig(producerProps)) // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(0,1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) // wait until leader is elected var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) @@ -168,7 +168,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { producer = new Producer[Int, String](new ProducerConfig(producerProps)) // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(0,1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) // wait until leader is elected var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) @@ -201,7 +201,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { producer = new Producer[Int, String](new ProducerConfig(producerProps)) // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createTopicWithAssignment(zkClient, topic, Map(0->Seq(server1.config.brokerId, server2.config.brokerId))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(server1.config.brokerId, server2.config.brokerId))) // wait until leader is elected var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 10712e2..2375758 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -346,7 +346,7 @@ object TestUtils extends Logging { def createBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { val brokers = ids.map(id => new Broker(id, "localhost", 6667)) - brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.host, b.port, jmxPort = -1)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.host, b.port, 6000, jmxPort = -1)) brokers } diff --git a/project/Build.scala b/project/Build.scala index bad93db..b3858f3 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -52,7 +52,7 @@ object KafkaBuild extends Build { javacOptions ++= Seq("-Xlint:unchecked", "-source", "1.5"), parallelExecution in Test := false, // Prevent tests from overrunning each other libraryDependencies ++= Seq( - "log4j" % "log4j" % "1.2.15", + "log4j" % "log4j" % "1.2.15" exclude("javax.jms", "jms"), "net.sf.jopt-simple" % "jopt-simple" % "3.2", "org.slf4j" % "slf4j-simple" % "1.6.4" ), diff --git a/system_test/migration_tool_testsuite/0.7/lib/kafka-perf-0.7.0.jar b/system_test/migration_tool_testsuite/0.7/lib/kafka-perf-0.7.0.jar index b1d7c7e..d4f89d5 100644 Binary files a/system_test/migration_tool_testsuite/0.7/lib/kafka-perf-0.7.0.jar and b/system_test/migration_tool_testsuite/0.7/lib/kafka-perf-0.7.0.jar differ diff --git a/system_test/migration_tool_testsuite/config/migration_producer.properties b/system_test/migration_tool_testsuite/config/migration_producer.properties index 1750807..17b5928 100644 --- a/system_test/migration_tool_testsuite/config/migration_producer.properties +++ b/system_test/migration_tool_testsuite/config/migration_producer.properties @@ -61,3 +61,6 @@ serializer.class=kafka.serializer.DefaultEncoder # the number of messages batched at the producer #batch.num.messages= + +message.send.max.retries=3 +request.required.acks=1 diff --git a/system_test/migration_tool_testsuite/migration_tool_test.py b/system_test/migration_tool_testsuite/migration_tool_test.py index 5d22f83..ce6f4f6 100644 --- a/system_test/migration_tool_testsuite/migration_tool_test.py +++ b/system_test/migration_tool_testsuite/migration_tool_test.py @@ -76,6 +76,8 @@ class MigrationToolTest(ReplicationUtils, SetupUtils): self.testSuiteAbsPathName, SystemTestEnv.SYSTEM_TEST_CASE_PREFIX) testCasePathNameList.sort() + replicationUtils = ReplicationUtils(self) + # ============================================================= # launch each testcase one by one: testcase_1, testcase_2, ... # ============================================================= @@ -284,7 +286,7 @@ class MigrationToolTest(ReplicationUtils, SetupUtils): # validate the data matched and checksum # ============================================= self.log_message("validating data matched") - kafka_system_test_utils.validate_07_08_migrated_data_matched(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) # ============================================= 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 1904ab5..608e3bd 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 @@ -100,7 +100,6 @@ "group.id": "mytestgroup", "consumer-timeout-ms": "10000", "log_filename": "console_consumer_8.log", - "formatter": "kafka.consumer.ChecksumMessageFormatter", "config_filename": "console_consumer_8.properties" }, { 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 8cacc69..333256c 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 @@ -101,7 +101,6 @@ "group.id": "mytestgroup", "consumer-timeout-ms": "10000", "log_filename": "console_consumer_8.log", - "formatter": "kafka.consumer.ChecksumMessageFormatter", "config_filename": "console_consumer_8.properties" }, { 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 4dbd80b..b2a6e85 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 @@ -101,7 +101,6 @@ "group.id": "mytestgroup", "consumer-timeout-ms": "10000", "log_filename": "console_consumer_8.log", - "formatter": "kafka.consumer.ChecksumMessageFormatter", "config_filename": "console_consumer_8.properties" }, { 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 e46b453..ddbc905 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 @@ -128,7 +128,6 @@ "group.id": "mytestgroup", "consumer-timeout-ms": "10000", "log_filename": "console_consumer_10.log", - "formatter": "kafka.consumer.ChecksumMessageFormatter", "config_filename": "console_consumer_10.properties" }, { @@ -137,7 +136,6 @@ "group.id": "mytestgroup", "consumer-timeout-ms": "10000", "log_filename": "console_consumer_11.log", - "formatter": "kafka.consumer.ChecksumMessageFormatter", "config_filename": "console_consumer_11.properties" }, 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 10f5955..21b4c40 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 @@ -128,7 +128,6 @@ "group.id": "mytestgroup", "consumer-timeout-ms": "10000", "log_filename": "console_consumer_10.log", - "formatter": "kafka.consumer.ChecksumMessageFormatter", "config_filename": "console_consumer_10.properties" }, { @@ -137,7 +136,6 @@ "group.id": "mytestgroup", "consumer-timeout-ms": "10000", "log_filename": "console_consumer_11.log", - "formatter": "kafka.consumer.ChecksumMessageFormatter", "config_filename": "console_consumer_11.properties" }, diff --git a/system_test/mirror_maker_testsuite/mirror_maker_test.py b/system_test/mirror_maker_testsuite/mirror_maker_test.py index 4dc3cdf..dfffb4e 100644 --- a/system_test/mirror_maker_testsuite/mirror_maker_test.py +++ b/system_test/mirror_maker_testsuite/mirror_maker_test.py @@ -165,17 +165,17 @@ class MirrorMakerTest(ReplicationUtils, SetupUtils): 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) + self.log_message("starting mirror makers") kafka_system_test_utils.start_mirror_makers(self.systemTestEnv, self.testcaseEnv) 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) - # ============================================= # starting producer diff --git a/system_test/replication_testsuite/replica_basic_test.py b/system_test/replication_testsuite/replica_basic_test.py index 40c1157..e20130b 100644 --- a/system_test/replication_testsuite/replica_basic_test.py +++ b/system_test/replication_testsuite/replica_basic_test.py @@ -231,7 +231,7 @@ class ReplicaBasicTest(ReplicationUtils, SetupUtils): # ============================================== if brokerType == "leader" or brokerType == "follower": self.log_message("looking up leader") - leaderDict = kafka_system_test_utils.get_leader_elected_log_line(self.systemTestEnv, self.testcaseEnv, self.leaderAttributesDict) + leaderDict = kafka_system_test_utils.get_leader_attributes(self.systemTestEnv, self.testcaseEnv) # ========================== # leaderDict looks like this: @@ -285,10 +285,10 @@ class ReplicaBasicTest(ReplicationUtils, SetupUtils): kafka_system_test_utils.validate_leader_election_successful(self.testcaseEnv, leaderDict, self.testcaseEnv.validationStatusDict) # trigger leader re-election by stopping leader to get re-election latency - reelectionLatency = kafka_system_test_utils.get_reelection_latency(self.systemTestEnv, self.testcaseEnv, leaderDict, self.leaderAttributesDict) - latencyKeyName = "Leader Election Latency - iter " + str(i) + " brokerid " + leaderDict["brokerid"] - self.testcaseEnv.validationStatusDict[latencyKeyName] = str("{0:.2f}".format(reelectionLatency * 1000)) + " ms" - self.testcaseEnv.userDefinedEnvVarDict["leaderElectionLatencyList"].append("{0:.2f}".format(reelectionLatency * 1000)) + #reelectionLatency = kafka_system_test_utils.get_reelection_latency(self.systemTestEnv, self.testcaseEnv, leaderDict, self.leaderAttributesDict) + #latencyKeyName = "Leader Election Latency - iter " + str(i) + " brokerid " + leaderDict["brokerid"] + #self.testcaseEnv.validationStatusDict[latencyKeyName] = str("{0:.2f}".format(reelectionLatency * 1000)) + " ms" + #self.testcaseEnv.userDefinedEnvVarDict["leaderElectionLatencyList"].append("{0:.2f}".format(reelectionLatency * 1000)) elif brokerType == "follower": # stopping Follower @@ -324,25 +324,25 @@ class ReplicaBasicTest(ReplicationUtils, SetupUtils): pass - self.anonLogger.info("sleeping for 15s") - time.sleep(15) + self.anonLogger.info("sleeping for 60s") + time.sleep(60) i += 1 # while loop # update Leader Election Latency MIN/MAX to testcaseEnv.validationStatusDict - self.testcaseEnv.validationStatusDict["Leader Election Latency MIN"] = None - try: - self.testcaseEnv.validationStatusDict["Leader Election Latency MIN"] = \ - min(self.testcaseEnv.userDefinedEnvVarDict["leaderElectionLatencyList"]) - except: - pass - - self.testcaseEnv.validationStatusDict["Leader Election Latency MAX"] = None - try: - self.testcaseEnv.validationStatusDict["Leader Election Latency MAX"] = \ - max(self.testcaseEnv.userDefinedEnvVarDict["leaderElectionLatencyList"]) - except: - pass + #self.testcaseEnv.validationStatusDict["Leader Election Latency MIN"] = None + #try: + # self.testcaseEnv.validationStatusDict["Leader Election Latency MIN"] = \ + # min(self.testcaseEnv.userDefinedEnvVarDict["leaderElectionLatencyList"]) + #except: + # pass + # + #self.testcaseEnv.validationStatusDict["Leader Election Latency MAX"] = None + #try: + # self.testcaseEnv.validationStatusDict["Leader Election Latency MAX"] = \ + # max(self.testcaseEnv.userDefinedEnvVarDict["leaderElectionLatencyList"]) + #except: + # pass # ============================================= # tell producer to stop diff --git a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json index a3e94cc..9bc164b 100644 --- a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json +++ b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json index 721f14f..c90d753 100644 --- a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json +++ b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json index be2bbb1..b62b8aa 100644 --- a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json +++ b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json index d51b988..b91cef8 100644 --- a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json +++ b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json index ebf88c7..4b3f76a 100644 --- a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json +++ b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json index b4044ad..b9b3485 100644 --- a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json +++ b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json index 3702a47..5c4351f 100644 --- a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json +++ b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json index 3f0cfa9..79cfed8 100644 --- a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json +++ b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json index 1f78162..a52b709 100644 --- a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json +++ b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json b/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json index 5b00112..8d4b5fe 100644 --- a/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json +++ b/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json @@ -66,6 +66,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0011/testcase_0011_properties.json b/system_test/replication_testsuite/testcase_0011/testcase_0011_properties.json index ad93419..b03f9cf 100644 --- a/system_test/replication_testsuite/testcase_0011/testcase_0011_properties.json +++ b/system_test/replication_testsuite/testcase_0011/testcase_0011_properties.json @@ -67,6 +67,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json b/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json index 82787a7..40c2f8d 100644 --- a/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json +++ b/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json @@ -65,6 +65,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -77,6 +78,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json b/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json index f8c884a..7a30460 100644 --- a/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json +++ b/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json @@ -65,6 +65,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -77,6 +78,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json b/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json index 9d71e30..d921f01 100644 --- a/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json +++ b/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json @@ -65,6 +65,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -77,6 +78,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json b/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json index 8d66c5d..fff0d68 100644 --- a/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json +++ b/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json b/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json index 3522f87..636f02a 100644 --- a/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json +++ b/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json b/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json index 58a98d0..bdb885e 100644 --- a/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json +++ b/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json b/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json index f3f9621..ca51c5f 100644 --- a/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json +++ b/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json b/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json index ba78605..459becf 100644 --- a/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json +++ b/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json b/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json index db4aea2..c9471bc 100644 --- a/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json +++ b/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json b/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json index b0353e6..8159464 100644 --- a/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json +++ b/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json b/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json index 6c9b214..f73f0e4 100644 --- a/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json +++ b/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json b/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json index d1dcbdb..606aad3 100644 --- a/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json +++ b/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json @@ -69,6 +69,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -81,6 +82,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json b/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json index 52a6cd3..ea47536 100644 --- a/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json +++ b/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json @@ -69,6 +69,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -81,6 +82,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json b/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json index 49d4b75..f9c9273 100644 --- a/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json +++ b/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json @@ -69,6 +69,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -81,6 +82,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json b/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json index e3e0b15..02f8506 100644 --- a/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json +++ b/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json b/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json index 0489d8e..0048279 100644 --- a/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json +++ b/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json b/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json index 676dd63..5119e61 100644 --- a/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json +++ b/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json b/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json index 8d8bb61..8b53fa7 100644 --- a/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json +++ b/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json b/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json index 7193294..a140882 100644 --- a/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json +++ b/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json @@ -72,6 +72,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -84,6 +85,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json b/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json index 77d6c2f..48b30c7 100644 --- a/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json +++ b/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json @@ -69,6 +69,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -81,6 +82,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json b/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json index 482ca59..8276aae 100644 --- a/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json +++ b/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json @@ -69,6 +69,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -81,6 +82,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json b/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json index 095a7b5..c6f8a23 100644 --- a/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json +++ b/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json b/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json index 557c081..7b1a4c2 100644 --- a/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json +++ b/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json b/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json index 751efc5..47276a8 100644 --- a/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json +++ b/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json b/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json index 761385f..3742cfa 100644 --- a/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json +++ b/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json b/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json index 772b310..d41672d 100644 --- a/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json +++ b/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json b/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json index b7759fe..a32d888 100644 --- a/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json +++ b/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json b/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json index a4f414b..5737870 100644 --- a/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json +++ b/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json b/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json index 62f97d7..c3d1d2c 100644 --- a/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json +++ b/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json b/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json index 47b857f..76ea677 100644 --- a/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json +++ b/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json b/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json index 37e96a6..672b3ff 100644 --- a/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json +++ b/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json b/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json index bec4fc4..19bd096 100644 --- a/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json +++ b/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json b/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json index 80fc064..8d6add0 100644 --- a/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json +++ b/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json b/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json index 5adfefb..2a7e777 100644 --- a/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json +++ b/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json b/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json index b8ef881..91fbc9d 100644 --- a/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json +++ b/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json b/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json index 6ea8cbf..5594a9b 100644 --- a/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json +++ b/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json b/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json index c1ca6b2..6eabd37 100644 --- a/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json +++ b/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json @@ -70,6 +70,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json b/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json index f87074d..496340c 100644 --- a/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json +++ b/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json @@ -68,6 +68,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json b/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json index cb186dd..beabc86 100644 --- a/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json +++ b/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json @@ -68,6 +68,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json b/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json index 6a87d14..0e8f5a4 100644 --- a/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json +++ b/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json @@ -68,6 +68,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json b/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json index fbe342e..98a3855 100644 --- a/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json +++ b/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json @@ -68,6 +68,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json b/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json index 51a7af4..e09ac9c 100644 --- a/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json +++ b/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json @@ -68,6 +68,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json b/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json index 3a7a2ac..bcfe91b 100644 --- a/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json +++ b/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json @@ -68,6 +68,7 @@ "message-size": "500", "message": "100", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json b/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json index 065cdb8..da07a9e 100644 --- a/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json +++ b/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json @@ -68,6 +68,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json b/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json index 1d1fb96..0fecafc 100644 --- a/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json +++ b/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json @@ -68,6 +68,7 @@ "message-size": "500", "message": "100", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance.log", "config_filename": "producer_performance.properties" diff --git a/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json b/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json index 3d0e8b7..d2ffd95 100644 --- a/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json +++ b/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json b/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json index 7fb20b7..c86525d 100644 --- a/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json +++ b/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json b/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json index d50cdcb..b77e4fd 100644 --- a/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json +++ b/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json b/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json index c0423f1..e753327 100644 --- a/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json +++ b/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json b/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json index a9155eb..5468401 100644 --- a/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json +++ b/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json b/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json index a6c65b7..e5ab0a0 100644 --- a/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json +++ b/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json b/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json index 9d351ee..7aa6e9a 100644 --- a/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json +++ b/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json b/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json index c3aa78d..08aa108 100644 --- a/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json +++ b/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json b/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json index 5c5161f..512fafb 100644 --- a/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json +++ b/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json b/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json index 286d46e..9b711af 100644 --- a/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json +++ b/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json b/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json index 70b14d5..3836366 100644 --- a/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json +++ b/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json b/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json index 5ea6368..86ab75a 100644 --- a/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json +++ b/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"true", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json b/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json index 6ac49c5..82d51b6 100644 --- a/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json +++ b/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json b/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json index fe4ec5f..31c1be0 100644 --- a/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json +++ b/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json b/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json index 3b85302..72f78b0 100644 --- a/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json +++ b/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json b/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json index 8f6ee50..ee459f4 100644 --- a/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json +++ b/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json @@ -73,6 +73,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_4.log", "config_filename": "producer_performance_4.properties" @@ -85,6 +86,7 @@ "message-size": "500", "message": "500", "request-num-acks": "1", + "producer-retry-backoff-ms": "300", "sync":"false", "log_filename": "producer_performance_5.log", "config_filename": "producer_performance_5.properties" diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index ae393bc..35fc383 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -1047,6 +1047,7 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk "JMX_PORT=" + jmxPort, kafkaRunClassBin + " kafka.perf.ProducerPerformance", "--brokerinfo " + brokerInfoStr, + "--initial-message-id " + str(initMsgId), "--messages " + noMsgPerBatch, "--topic " + topic, "--threads " + threads, @@ -2211,3 +2212,60 @@ def validate_index_log(systemTestEnv, testcaseEnv, clusterName="source"): else: validationStatusDict["Validate index log in cluster [" + clusterName + "]"] = "FAILED" +def get_leader_attributes(systemTestEnv, testcaseEnv): + + logger.info("Querying Zookeeper for leader info ...", extra=d) + + # keep track of leader data in this dict such as broker id & entity id + leaderDict = {} + + clusterConfigsList = systemTestEnv.clusterEntityConfigDictList + tcConfigsList = testcaseEnv.testcaseConfigsList + + zkDictList = system_test_utils.get_dict_from_list_of_dicts(clusterConfigsList, "role", "zookeeper") + firstZkDict = zkDictList[0] + hostname = firstZkDict["hostname"] + zkEntityId = firstZkDict["entity_id"] + clientPort = system_test_utils.get_data_by_lookup_keyval(tcConfigsList, "entity_id", zkEntityId, "clientPort") + kafkaHome = system_test_utils.get_data_by_lookup_keyval(clusterConfigsList, "entity_id", zkEntityId, "kafka_home") + javaHome = system_test_utils.get_data_by_lookup_keyval(clusterConfigsList, "entity_id", zkEntityId, "java_home") + kafkaRunClassBin = kafkaHome + "/bin/kafka-run-class.sh" + + # this should have been updated in start_producer_in_thread + producerTopicsString = testcaseEnv.producerTopicsString + topics = producerTopicsString.split(',') + zkQueryStr = "get /brokers/topics/" + topics[0] + "/partitions/0/state" + brokerid = '' + + cmdStrList = ["ssh " + hostname, + "\"JAVA_HOME=" + javaHome, + kafkaRunClassBin + " org.apache.zookeeper.ZooKeeperMain", + "-server " + testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"], + zkQueryStr + " 2> /dev/null | tail -1\""] + cmdStr = " ".join(cmdStrList) + logger.debug("executing command [" + cmdStr + "]", extra=d) + + subproc = system_test_utils.sys_call_return_subproc(cmdStr) + for line in subproc.stdout.readlines(): + logger.debug("zk returned : " + line, extra=d) + if "\"leader\"" in line: + line = line.rstrip('\n') + json_data = json.loads(line) + for key,val in json_data.items(): + if key == 'leader': + brokerid = str(val) + + leaderDict["brokerid"] = brokerid + leaderDict["topic"] = topics[0] + leaderDict["partition"] = '0' + leaderDict["entity_id"] = system_test_utils.get_data_by_lookup_keyval( + tcConfigsList, "broker.id", brokerid, "entity_id") + leaderDict["hostname"] = system_test_utils.get_data_by_lookup_keyval( + clusterConfigsList, "entity_id", leaderDict["entity_id"], "hostname") + break + + print leaderDict + return leaderDict + + +