diff --git a/DISCLAIMER b/DISCLAIMER deleted file mode 100644 index 950e15d67781b..0000000000000 --- a/DISCLAIMER +++ /dev/null @@ -1,15 +0,0 @@ -Apache Kafka is an effort undergoing incubation at the Apache Software -Foundation (ASF), sponsored by the Apache Incubator PMC. - -Incubation is required of all newly accepted projects until a further review -indicates that the infrastructure, communications, and decision making process -have stabilized in a manner consistent with other successful ASF projects. - -While incubation status is not necessarily a reflection of the completeness -or stability of the code, it does indicate that the project has yet to be -fully endorsed by the ASF. - -For more information about the incubation status of the Kafka project you -can go to the following page: - -http://incubator.apache.org/kafka/ \ No newline at end of file diff --git a/bin/kafka-add-partitions.sh b/bin/kafka-add-partitions.sh deleted file mode 100755 index c08b837cd0dcb..0000000000000 --- a/bin/kafka-add-partitions.sh +++ /dev/null @@ -1,18 +0,0 @@ -#!/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-console-consumer.sh b/bin/kafka-console-consumer.sh index 95edc9e5bff12..b86ea56c5c01f 100755 --- a/bin/kafka-console-consumer.sh +++ b/bin/kafka-console-consumer.sh @@ -15,4 +15,4 @@ # limitations under the License. export KAFKA_HEAP_OPTS="-Xmx512M" -$(dirname $0)/kafka-run-class.sh kafka.consumer.ConsoleConsumer $@ +exec $(dirname $0)/kafka-run-class.sh kafka.consumer.ConsoleConsumer $@ diff --git a/bin/kafka-console-producer.sh b/bin/kafka-console-producer.sh index f102c2239a55c..ec0f21f9a2e64 100755 --- a/bin/kafka-console-producer.sh +++ b/bin/kafka-console-producer.sh @@ -15,4 +15,4 @@ # limitations under the License. export KAFKA_HEAP_OPTS="-Xmx512M" -$(dirname $0)/kafka-run-class.sh kafka.producer.ConsoleProducer $@ +exec $(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 39434d162e540..38c56c78304c2 100755 --- a/bin/kafka-consumer-perf-test.sh +++ b/bin/kafka-consumer-perf-test.sh @@ -15,4 +15,4 @@ # limitations under the License. export KAFKA_HEAP_OPTS="-Xmx512M" -$(dirname $0)/kafka-run-class.sh kafka.perf.ConsumerPerformance $@ +exec $(dirname $0)/kafka-run-class.sh kafka.perf.ConsumerPerformance $@ diff --git a/bin/kafka-create-topic.sh b/bin/kafka-create-topic.sh deleted file mode 100755 index 7a5f29735fceb..0000000000000 --- a/bin/kafka-create-topic.sh +++ /dev/null @@ -1,17 +0,0 @@ -#!/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. - -$(dirname $0)/kafka-run-class.sh kafka.admin.CreateTopicCommand $@ diff --git a/bin/kafka-preferred-replica-election.sh b/bin/kafka-preferred-replica-election.sh index 6416dc1614002..ed167c26ce957 100755 --- a/bin/kafka-preferred-replica-election.sh +++ b/bin/kafka-preferred-replica-election.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -$(dirname $0)/kafka-run-class.sh kafka.admin.PreferredReplicaLeaderElectionCommand $@ +exec $(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 1935522079cdd..d75ab7d1f303e 100755 --- a/bin/kafka-producer-perf-test.sh +++ b/bin/kafka-producer-perf-test.sh @@ -15,4 +15,4 @@ # limitations under the License. export KAFKA_HEAP_OPTS="-Xmx512M" -$(dirname $0)/kafka-run-class.sh kafka.perf.ProducerPerformance $@ +exec $(dirname $0)/kafka-run-class.sh kafka.perf.ProducerPerformance $@ diff --git a/bin/kafka-reassign-partitions.sh b/bin/kafka-reassign-partitions.sh index a6bd148752bb8..95b4ae0e4f001 100755 --- a/bin/kafka-reassign-partitions.sh +++ b/bin/kafka-reassign-partitions.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -$(dirname $0)/kafka-run-class.sh kafka.admin.ReassignPartitionsCommand $@ +exec $(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 86943eed22e3a..8e2e7139f71e7 100755 --- a/bin/kafka-replay-log-producer.sh +++ b/bin/kafka-replay-log-producer.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -$(dirname $0)/kafka-run-class.sh kafka.tools.ReplayLogProducer $@ +exec $(dirname $0)/kafka-run-class.sh kafka.tools.ReplayLogProducer $@ diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index eb6ff1b12e538..416ecadf27333 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -16,7 +16,7 @@ if [ $# -lt 1 ]; then - echo "USAGE: $0 classname [opts]" + echo "USAGE: $0 [-daemon] [-name servicename] [-loggc] classname [opts]" exit 1 fi @@ -69,6 +69,8 @@ if [ -z "$KAFKA_LOG4J_OPTS" ]; then KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/config/tools-log4j.properties" fi +KAFKA_LOG4J_OPTS="-Dkafka.logs.dir=$LOG_DIR $KAFKA_LOG4J_OPTS" + # Generic jvm settings you want to add if [ -z "$KAFKA_OPTS" ]; then KAFKA_OPTS="" @@ -88,33 +90,48 @@ 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" + KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseCompressedOops -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC -Djava.awt.headless=true" fi + +while [ $# -gt 0 ]; do + COMMAND=$1 + case $COMMAND in + -name) + DAEMON_NAME=$2 + CONSOLE_OUTPUT_FILE=$LOG_DIR/$DAEMON_NAME.out + shift 2 + ;; + -loggc) + if [ -z "$KAFKA_GC_LOG_OPTS"] ; then + GC_LOG_ENABLED="true" + fi + shift + ;; + -daemon) + DAEMON_MODE="true" + shift + ;; + *) + break + ;; + esac +done + # 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 +if [ "x$GC_LOG_ENABLED" = "xtrue" ]; then + GC_LOG_FILE_NAME=$DAEMON_NAME$GC_FILE_SUFFIX 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_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)" - match=`echo $exception | grep -E "$pattern"` - if [[ -n "$match" ]]; then - echo $noBuildMessage - fi - rm exception.txt +# Launch mode +if [ "x$DAEMON_MODE" = "xtrue" ]; then + nohup $JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_OPTS -cp $CLASSPATH $KAFKA_OPTS "$@" > "$CONSOLE_OUTPUT_FILE" 2>&1 < /dev/null & +else + exec $JAVA $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_OPTS -cp $CLASSPATH $KAFKA_OPTS "$@" fi + diff --git a/bin/kafka-server-start.sh b/bin/kafka-server-start.sh index 4005289428d5d..7050649e6795b 100755 --- a/bin/kafka-server-start.sh +++ b/bin/kafka-server-start.sh @@ -16,10 +16,23 @@ if [ $# -lt 1 ]; then - echo "USAGE: $0 server.properties" + echo "USAGE: $0 [-daemon] server.properties" exit 1 fi 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 $@ + +EXTRA_ARGS="-name kafkaServer -loggc" + +COMMAND=$1 +case $COMMAND in + -daemon) + EXTRA_ARGS="-daemon "$EXTRA_ARGS + shift + ;; + *) + ;; +esac + +exec $base_dir/kafka-run-class.sh $EXTRA_ARGS kafka.Kafka $@ diff --git a/bin/kafka-simple-consumer-perf-test.sh b/bin/kafka-simple-consumer-perf-test.sh index 94c9ed0bfc283..c466cc8886a25 100755 --- a/bin/kafka-simple-consumer-perf-test.sh +++ b/bin/kafka-simple-consumer-perf-test.sh @@ -15,4 +15,4 @@ # limitations under the License. export KAFKA_HEAP_OPTS="-Xmx512M" -$(dirname $0)/kafka-run-class.sh kafka.perf.SimpleConsumerPerformance $@ +exec $(dirname $0)/kafka-run-class.sh kafka.perf.SimpleConsumerPerformance $@ diff --git a/bin/kafka-simple-consumer-shell.sh b/bin/kafka-simple-consumer-shell.sh index 8d69357838f1b..9316f7908dfe3 100755 --- a/bin/kafka-simple-consumer-shell.sh +++ b/bin/kafka-simple-consumer-shell.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -$(dirname $0)/kafka-run-class.sh kafka.tools.SimpleConsumerShell $@ +exec $(dirname $0)/kafka-run-class.sh kafka.tools.SimpleConsumerShell $@ diff --git a/bin/kafka-list-topic.sh b/bin/kafka-topics.sh similarity index 92% rename from bin/kafka-list-topic.sh rename to bin/kafka-topics.sh index bb2dde52c0d57..b39b2729170d6 100755 --- a/bin/kafka-list-topic.sh +++ b/bin/kafka-topics.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -$(dirname $0)/kafka-run-class.sh kafka.admin.ListTopicCommand $@ +exec $(dirname $0)/kafka-run-class.sh kafka.admin.TopicCommand $@ diff --git a/bin/zookeeper-server-start.sh b/bin/zookeeper-server-start.sh index e4746a84b3b95..2e7be7486d20a 100755 --- a/bin/zookeeper-server-start.sh +++ b/bin/zookeeper-server-start.sh @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -if [ $# -ne 1 ]; +if [ $# -lt 1 ]; then echo "USAGE: $0 zookeeper.properties" exit 1 @@ -22,5 +22,18 @@ 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 $@ + +EXTRA_ARGS="-name zookeeper -loggc" + +COMMAND=$1 +case $COMMAND in + -daemon) + EXTRA_ARGS="-daemon "$EXTRA_ARGS + shift + ;; + *) + ;; +esac + +exec $base_dir/kafka-run-class.sh $EXTRA_ARGS org.apache.zookeeper.server.quorum.QuorumPeerMain $@ diff --git a/bin/zookeeper-shell.sh b/bin/zookeeper-shell.sh index e0de33f2e16e5..92cce7600efa6 100755 --- a/bin/zookeeper-shell.sh +++ b/bin/zookeeper-shell.sh @@ -20,4 +20,4 @@ then exit 1 fi -$(dirname $0)/kafka-run-class.sh org.apache.zookeeper.ZooKeeperMain -server $1 \ No newline at end of file +exec $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.ZooKeeperMain -server $1 \ No newline at end of file diff --git a/config/log4j.properties b/config/log4j.properties index dcf48f56d920e..1ab850772a965 100644 --- a/config/log4j.properties +++ b/config/log4j.properties @@ -12,6 +12,9 @@ # 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. + +kafka.logs.dir=logs + log4j.rootLogger=INFO, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender @@ -20,25 +23,31 @@ 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=logs/server.log +log4j.appender.kafkaAppender.File=${kafka.logs.dir}/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=logs/state-change.log +log4j.appender.stateChangeAppender.File=${kafka.logs.dir}/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=logs/kafka-request.log +log4j.appender.requestAppender.File=${kafka.logs.dir}/kafka-request.log log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n +log4j.appender.cleanerAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.cleanerAppender.DatePattern='.'yyyy-MM-dd-HH +log4j.appender.cleanerAppender.File=log-cleaner.log +log4j.appender.cleanerAppender.layout=org.apache.log4j.PatternLayout +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=logs/controller.log +log4j.appender.controllerAppender.File=${kafka.logs.dir}/controller.log log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n @@ -50,19 +59,22 @@ log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n #log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG log4j.logger.kafka=INFO, kafkaAppender -log4j.logger.kafka.network.RequestChannel$=TRACE, requestAppender +log4j.logger.kafka.network.RequestChannel$=WARN, requestAppender log4j.additivity.kafka.network.RequestChannel$=false #log4j.logger.kafka.network.Processor=TRACE, requestAppender #log4j.logger.kafka.server.KafkaApis=TRACE, requestAppender #log4j.additivity.kafka.server.KafkaApis=false -log4j.logger.kafka.request.logger=TRACE, requestAppender +log4j.logger.kafka.request.logger=WARN, requestAppender log4j.additivity.kafka.request.logger=false log4j.logger.kafka.controller=TRACE, controllerAppender log4j.additivity.kafka.controller=false +log4j.logger.kafka.log.LogCleaner=INFO, cleanerAppender +log4j.additivity.kafka.log.LogCleaner=false +log4j.logger.kafka.log.Cleaner=INFO, cleanerAppender +log4j.additivity.kafka.log.Cleaner=false + log4j.logger.state.change.logger=TRACE, stateChangeAppender log4j.additivity.state.change.logger=false - - diff --git a/config/server.properties b/config/server.properties index 977b8da470bc6..2ffe0ebccf109 100644 --- a/config/server.properties +++ b/config/server.properties @@ -24,11 +24,18 @@ broker.id=0 # The port the socket server listens on port=9092 -# Hostname the broker will bind to and advertise to producers and consumers. -# If not set, the server will bind to all interfaces and advertise the value returned from -# from java.net.InetAddress.getCanonicalHostName(). +# Hostname the broker will bind to. If not set, the server will bind to all interfaces #host.name=localhost +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +#advertised.host.name= + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + # The number of threads handling network requests num.network.threads=2 @@ -50,14 +57,15 @@ socket.request.max.bytes=104857600 # 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. +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. num.partitions=2 ############################# Log Flush Policy ############################# -# The following configurations control the flush of data to disk. This is among the most -# important performance knob in kafka. +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# 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. @@ -66,13 +74,10 @@ num.partitions=2 # 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 +#log.flush.interval.messages=10000 # The maximum amount of time a message can sit in a log before we force a flush -log.flush.interval.ms=1000 - -# Per-topic overrides for log.flush.interval.ms -#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000 +#log.flush.interval.ms=1000 ############################# Log Retention Policy ############################# @@ -93,7 +98,7 @@ log.segment.bytes=536870912 # The interval at which log segments are checked to see if they can be deleted according # to the retention policies -log.cleanup.interval.mins=1 +log.retention.check.interval.ms=60000 ############################# Zookeeper ############################# @@ -108,3 +113,4 @@ zookeeper.connect=localhost:2181 zookeeper.connection.timeout.ms=1000000 +log.cleanup.policy=delete diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java index 4b1d117462d10..f3fb3fd99869e 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java @@ -86,8 +86,7 @@ public void run() throws Exception { Long timestamp = RANDOM.nextLong(); if (timestamp < 0) timestamp = -timestamp; byte[] bytes = timestamp.toString().getBytes("UTF8"); - Message message = new Message(bytes); - list.add(new KeyedMessage(_topic, null, message)); + list.add(new KeyedMessage(_topic, null, bytes)); } // send events System.out.println(" send " + list.size() + " " + _topic + " count events to " + _uri); diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/SimpleKafkaETLMapper.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/SimpleKafkaETLMapper.java index 45cc92181014a..0fea5db0bd3d8 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/SimpleKafkaETLMapper.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/SimpleKafkaETLMapper.java @@ -42,6 +42,8 @@ public class SimpleKafkaETLMapper implements protected Text getData(Message message) throws IOException { ByteBuffer buf = message.payload(); + if(buf == null) + return new Text(); byte[] array = new byte[buf.limit()]; buf.get(array); diff --git a/core/src/main/scala/kafka/admin/AddPartitionsCommand.scala b/core/src/main/scala/kafka/admin/AddPartitionsCommand.scala deleted file mode 100644 index 7f037081304c5..0000000000000 --- a/core/src/main/scala/kafka/admin/AddPartitionsCommand.scala +++ /dev/null @@ -1,127 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.admin - -import joptsimple.OptionParser -import 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: Throwable => - 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 AdministrationException("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 AdministrationException("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(topic, partitionReplicaList, zkClient, 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 AdministrationException("replication factor must be larger than 0") - if (brokerList.size != brokerList.toSet.size) - throw new AdministrationException("duplicate brokers in replica assignment: " + brokerList) - if (!brokerList.toSet.subsetOf(availableBrokerList)) - throw new AdministrationException("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 AdministrationException("partition " + i + " has different replication factor: " + brokerList) - partitionId = partitionId + 1 - } - ret.toMap - } -} diff --git a/core/src/main/scala/kafka/admin/AdminOperationException.scala b/core/src/main/scala/kafka/admin/AdminOperationException.scala new file mode 100644 index 0000000000000..a45b3f7e93a08 --- /dev/null +++ b/core/src/main/scala/kafka/admin/AdminOperationException.scala @@ -0,0 +1,23 @@ +/** + * 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 + +class AdminOperationException(val error: String, cause: Throwable) extends RuntimeException(error, cause) { + def this(error: Throwable) = this(error.getMessage, error) + def this(msg: String) = this(msg, null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index d6ab275d6ed03..9d142a5c20d64 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -18,16 +18,25 @@ package kafka.admin import java.util.Random -import kafka.utils.{Logging, ZkUtils} +import java.util.Properties +import kafka.api.{TopicMetadata, PartitionMetadata} +import kafka.cluster.{Broker, Cluster} +import kafka.log.LogConfig +import kafka.utils.{Logging, ZkUtils, Json} import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException import scala.collection._ +import mutable.ListBuffer import scala.collection.mutable import kafka.common._ +import scala.Predef._ +import collection.Map +import scala.Some +import collection.Set object AdminUtils extends Logging { val rand = new Random - val AdminEpoch = -1 + val TopicConfigChangeZnodePrefix = "config_change_" /** * There are 2 goals of replica assignment: @@ -47,15 +56,20 @@ object AdminUtils extends Logging { * p3 p4 p0 p1 p2 (3nd replica) * p7 p8 p9 p5 p6 (3nd replica) */ - def assignReplicasToBrokers(brokerList: Seq[Int], nPartitions: Int, replicationFactor: Int, - fixedStartIndex: Int = -1, startPartitionId: Int = -1) + def assignReplicasToBrokers(brokerList: Seq[Int], + filteredCluster: Cluster, + nPartitions: Int, + replicationFactor: Int, + fixedStartIndex: Int = -1, + startPartitionId: Int = -1, + maxReplicaPerRack: Int = -1) : Map[Int, Seq[Int]] = { if (nPartitions <= 0) - throw new AdministrationException("number of partitions must be larger than 0") + throw new AdminOperationException("number of partitions must be larger than 0") if (replicationFactor <= 0) - throw new AdministrationException("replication factor must be larger than 0") + throw new AdminOperationException("replication factor must be larger than 0") if (replicationFactor > brokerList.size) - throw new AdministrationException("replication factor: " + replicationFactor + + throw new AdminOperationException("replication factor: " + replicationFactor + " larger than available brokers: " + brokerList.size) val ret = new mutable.HashMap[Int, List[Int]]() val startIndex = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(brokerList.size) @@ -67,18 +81,139 @@ object AdminUtils extends Logging { nextReplicaShift += 1 val firstReplicaIndex = (currentPartitionId + startIndex) % brokerList.size var replicaList = List(brokerList(firstReplicaIndex)) - for (j <- 0 until replicationFactor - 1) - replicaList ::= brokerList(getWrappedIndex(firstReplicaIndex, nextReplicaShift, j, brokerList.size)) + if (maxReplicaPerRack <= 0) { + for (j <- 0 until replicationFactor - 1) + replicaList ::= brokerList(replicaIndex(firstReplicaIndex, nextReplicaShift, j, brokerList.size)) + } else { + var rackReplicaCount: mutable.Map[Int, Int] = mutable.Map(filteredCluster.getBroker(brokerList(firstReplicaIndex)).get.rack -> 1) + var k = 0 + for (j <- 0 until replicationFactor - 1) { + var done = false; + while (!done && k < brokerList.size) { + val broker = brokerList(replicaIndex(firstReplicaIndex, nextReplicaShift, k, brokerList.size)) + val rack = filteredCluster.getBroker(broker).get.rack + if (!(rackReplicaCount contains rack)) { + replicaList ::= broker + rackReplicaCount += (rack -> 1) + done = true; + } else if (rackReplicaCount(rack) < maxReplicaPerRack) { + rackReplicaCount(rack) = rackReplicaCount(rack) + 1 + replicaList ::= broker + done = true; + } + k = k + 1 + } + if (!done) { + throw new AdminOperationException("not enough brokers available in unique racks to meet maxReplicaPerRack limit of " + maxReplicaPerRack) + } + } + } ret.put(currentPartitionId, replicaList.reverse) currentPartitionId = currentPartitionId + 1 } ret.toMap } - def createOrUpdateTopicPartitionAssignmentPathInZK(topic: String, replicaAssignment: Map[Int, Seq[Int]], zkClient: ZkClient, update: Boolean = false) { + def addPartitions(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "") { + val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) + val existingMaxRackReplication = ZkUtils.getMaxRackReplicationForTopics(zkClient, List(topic)) + if (existingPartitionsReplicaList.size == 0 || existingMaxRackReplication.size == 0) + throw new AdminOperationException("The topic %s does not exist".format(topic)) + + val existingReplicaList = existingPartitionsReplicaList.head._2 + val maxReplicaPerRack = existingMaxRackReplication.head._2 + val partitionsToAdd = numPartitions - existingPartitionsReplicaList.size + if (partitionsToAdd <= 0) + throw new AdminOperationException("The number of partitions for a topic can only be increased") + + // create the new partition replication list + val brokerList = ZkUtils.getSortedBrokerList(zkClient) + val newPartitionReplicaList = if (replicaAssignmentStr == null || replicaAssignmentStr == "") + AdminUtils.assignReplicasToBrokers(brokerList, ZkUtils.getFilteredCluster(zkClient, brokerList), partitionsToAdd, existingReplicaList.size, existingReplicaList.head, existingPartitionsReplicaList.size, maxReplicaPerRack) + 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, maxReplicaPerRack = maxReplicaPerRack, update = true) + } + + def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int): Map[Int, List[Int]] = { + var partitionList = replicaAssignmentList.split(",") + val ret = new mutable.HashMap[Int, List[Int]]() + var partitionId = startPartitionId + partitionList = partitionList.takeRight(partitionList.size - partitionId) + 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 + } + + def deleteTopic(zkClient: ZkClient, topic: String) { + zkClient.deleteRecursive(ZkUtils.getTopicPath(topic)) + zkClient.deleteRecursive(ZkUtils.getTopicConfigPath(topic)) + } + + def topicExists(zkClient: ZkClient, topic: String): Boolean = + zkClient.exists(ZkUtils.getTopicPath(topic)) + + def createTopic(zkClient: ZkClient, + topic: String, + partitions: Int, + replicationFactor: Int, + maxReplicaPerRack: Int = -1, + topicConfig: Properties = new Properties) { + val brokerList = ZkUtils.getSortedBrokerList(zkClient) + val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, ZkUtils.getFilteredCluster(zkClient, brokerList), partitions, replicationFactor, -1, -1, maxReplicaPerRack) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig, maxReplicaPerRack) + } + + def createOrUpdateTopicPartitionAssignmentPathInZK(zkClient: ZkClient, + topic: String, + partitionReplicaAssignment: Map[Int, Seq[Int]], + config: Properties = new Properties, + maxReplicaPerRack: Int = -1, + 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(!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)) + + // write out the config if there is any, this isn't transactional with the partition assignments + writeTopicConfig(zkClient, topic, config) + + // create the partition assignment + writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, maxReplicaPerRack, update) + } + + private def writeTopicPartitionAssignment(zkClient: ZkClient, topic: String, replicaAssignment: Map[Int, Seq[Int]], maxReplicaPerRack: Int, update: Boolean) { try { val zkPath = ZkUtils.getTopicPath(topic) - val jsonPartitionData = ZkUtils.replicaAssignmentZkdata(replicaAssignment.map(e => (e._1.toString -> e._2))) + val jsonPartitionData = ZkUtils.replicaAssignmentZkData(replicaAssignment.map(e => (e._1.toString -> e._2)), maxReplicaPerRack) if (!update) { info("Topic creation " + jsonPartitionData.toString) @@ -90,16 +225,154 @@ object AdminUtils extends Logging { 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)) - case e2: Throwable => throw new AdministrationException(e2.toString) + case e2: Throwable => throw new AdminOperationException(e2.toString) } } + + /** + * Update the config for an existing topic and create a change notification so the change will propagate to other brokers + * @param zkClient: The ZkClient handle used to write the new config to zookeeper + * @param topic: The topic for which configs are being changed + * @param configs: The final set of configs that will be applied to the topic. If any new configs need to be added or + * existing configs need to be deleted, it should be done prior to invoking this API + * + */ + def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties) { + if(!topicExists(zkClient, topic)) + throw new AdminOperationException("Topic \"%s\" does not exist.".format(topic)) + + // remove the topic overrides + LogConfig.validate(configs) - private def getWrappedIndex(firstReplicaIndex: Int, secondReplicaShift: Int, replicaIndex: Int, nBrokers: Int): Int = { + // write the new config--may not exist if there were previously no overrides + writeTopicConfig(zkClient, topic, configs) + + // create the change notification + zkClient.createPersistentSequential(ZkUtils.TopicConfigChangesPath + "/" + TopicConfigChangeZnodePrefix, Json.encode(topic)) + } + + /** + * Write out the topic config to zk, if there is any + */ + private def writeTopicConfig(zkClient: ZkClient, topic: String, config: Properties) { + val configMap: mutable.Map[String, String] = { + import JavaConversions._ + config + } + val map = Map("version" -> 1, "config" -> configMap) + ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicConfigPath(topic), Json.encode(map)) + } + + /** + * Read the topic config (if any) from zk + */ + def fetchTopicConfig(zkClient: ZkClient, topic: String): Properties = { + val str: String = zkClient.readData(ZkUtils.getTopicConfigPath(topic), true) + val props = new Properties() + if(str != null) { + Json.parseFull(str) match { + case None => // there are no config overrides + case Some(map: Map[String, _]) => + require(map("version") == 1) + map.get("config") match { + case Some(config: Map[String, String]) => + for((k,v) <- config) + props.setProperty(k, v) + case _ => throw new IllegalArgumentException("Invalid topic config: " + str) + } + + case o => throw new IllegalArgumentException("Unexpected value in config: " + str) + } + } + props + } + + def fetchAllTopicConfigs(zkClient: ZkClient): Map[String, Properties] = + ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchTopicConfig(zkClient, topic))).toMap + + def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient): TopicMetadata = + fetchTopicMetadataFromZk(topic, zkClient, new mutable.HashMap[Int, Broker]) + + def fetchTopicMetadataFromZk(topics: Set[String], zkClient: ZkClient): Set[TopicMetadata] = { + val cachedBrokerInfo = new mutable.HashMap[Int, Broker]() + topics.map(topic => fetchTopicMetadataFromZk(topic, zkClient, cachedBrokerInfo)) + } + + private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker]): TopicMetadata = { + if(ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) { + val topicPartitionAssignment = ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic).get + val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) + val partitionMetadata = sortedPartitions.map { partitionMap => + val partition = partitionMap._1 + val replicas = partitionMap._2 + val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partition) + val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition) + debug("replicas = " + replicas + ", in sync replicas = " + inSyncReplicas + ", leader = " + leader) + + var leaderInfo: Option[Broker] = None + var replicaInfo: Seq[Broker] = Nil + var isrInfo: Seq[Broker] = Nil + try { + leaderInfo = leader match { + case Some(l) => + try { + Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head) + } catch { + case e: Throwable => throw new LeaderNotAvailableException("Leader not available for partition [%s,%d]".format(topic, partition), e) + } + case None => throw new LeaderNotAvailableException("No leader exists for partition " + partition) + } + try { + replicaInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, replicas.map(id => id.toInt)) + isrInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, inSyncReplicas) + } catch { + case e: Throwable => throw new ReplicaNotAvailableException(e) + } + if(replicaInfo.size < replicas.size) + throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + + replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) + if(isrInfo.size < inSyncReplicas.size) + throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + + inSyncReplicas.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) + new PartitionMetadata(partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) + } catch { + case e: Throwable => + debug("Error while fetching metadata for partition [%s,%d]".format(topic, partition), e) + new PartitionMetadata(partition, leaderInfo, replicaInfo, isrInfo, + ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) + } + } + new TopicMetadata(topic, partitionMetadata) + } else { + // topic doesn't exist, send appropriate error code + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode) + } + } + + private def getBrokerInfoFromCache(zkClient: ZkClient, + cachedBrokerInfo: scala.collection.mutable.Map[Int, Broker], + brokerIds: Seq[Int]): Seq[Broker] = { + var failedBrokerIds: ListBuffer[Int] = new ListBuffer() + val brokerMetadata = brokerIds.map { id => + val optionalBrokerInfo = cachedBrokerInfo.get(id) + optionalBrokerInfo match { + case Some(brokerInfo) => Some(brokerInfo) // return broker info from the cache + case None => // fetch it from zookeeper + ZkUtils.getBrokerInfo(zkClient, id) match { + case Some(brokerInfo) => + cachedBrokerInfo += (id -> brokerInfo) + Some(brokerInfo) + case None => + failedBrokerIds += id + None + } + } + } + brokerMetadata.filter(_.isDefined).map(_.get) + } + + private def replicaIndex(firstReplicaIndex: Int, secondReplicaShift: Int, replicaIndex: Int, nBrokers: Int): Int = { val shift = 1 + (secondReplicaShift + replicaIndex) % (nBrokers - 1) (firstReplicaIndex + shift) % nBrokers } } - -class AdministrationException(val errorMessage: String) extends RuntimeException(errorMessage) { - def this() = this(null) -} diff --git a/core/src/main/scala/kafka/admin/CreateTopicCommand.scala b/core/src/main/scala/kafka/admin/CreateTopicCommand.scala deleted file mode 100644 index 84c20950fbf39..0000000000000 --- a/core/src/main/scala/kafka/admin/CreateTopicCommand.scala +++ /dev/null @@ -1,117 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.admin - -import joptsimple.OptionParser -import kafka.utils._ -import org.I0Itec.zkclient.ZkClient -import scala.collection.mutable -import kafka.common.Topic - -object CreateTopicCommand extends Logging { - - def main(args: Array[String]): Unit = { - val parser = new OptionParser - val topicOpt = parser.accepts("topic", "REQUIRED: The topic to be created.") - .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", "number of partitions in the topic") - .withRequiredArg - .describedAs("# of partitions") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) - val replicationFactorOpt = parser.accepts("replica", "replication factor for each partitions in the topic") - .withRequiredArg - .describedAs("replication factor") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) - val replicaAssignmentOpt = parser.accepts("replica-assignment-list", "for manually assigning replicas to brokers") - .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)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } - - val topic = options.valueOf(topicOpt) - val zkConnect = options.valueOf(zkConnectOpt) - val nPartitions = options.valueOf(nPartitionsOpt).intValue - val replicationFactor = options.valueOf(replicationFactorOpt).intValue - val replicaAssignmentStr = options.valueOf(replicaAssignmentOpt) - var zkClient: ZkClient = null - try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) - createTopic(zkClient, topic, nPartitions, replicationFactor, replicaAssignmentStr) - println("creation succeeded!") - } catch { - case e: Throwable => - println("creation failed because of " + e.getMessage) - println(Utils.stackTrace(e)) - } finally { - if (zkClient != null) - zkClient.close() - } - } - - def createTopic(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicationFactor: Int = 1, replicaAssignmentStr: String = "") { - Topic.validate(topic) - - val brokerList = ZkUtils.getSortedBrokerList(zkClient) - - val partitionReplicaAssignment = if (replicaAssignmentStr == "") - AdminUtils.assignReplicasToBrokers(brokerList, numPartitions, replicationFactor) - else - getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet) - debug("Replica assignment list for %s is %s".format(topic, partitionReplicaAssignment)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(topic, partitionReplicaAssignment, zkClient) - } - - def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int]): Map[Int, List[Int]] = { - val partitionList = replicaAssignmentList.split(",") - val ret = new mutable.HashMap[Int, List[Int]]() - for (i <- 0 until partitionList.size) { - val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) - if (brokerList.size <= 0) - throw new AdministrationException("replication factor must be larger than 0") - if (brokerList.size != brokerList.toSet.size) - throw new AdministrationException("duplicate brokers in replica assignment: " + brokerList) - if (!brokerList.toSet.subsetOf(availableBrokerList)) - throw new AdministrationException("some specified brokers not available. specified brokers: " + brokerList.toString + - "available broker:" + availableBrokerList.toString) - ret.put(i, brokerList.toList) - if (ret(i).size != ret(0).size) - throw new AdministrationException("partition " + i + " has different replication factor: " + brokerList) - } - ret.toMap - } -} diff --git a/core/src/main/scala/kafka/admin/ListTopicCommand.scala b/core/src/main/scala/kafka/admin/ListTopicCommand.scala deleted file mode 100644 index eed49e1642e5c..0000000000000 --- a/core/src/main/scala/kafka/admin/ListTopicCommand.scala +++ /dev/null @@ -1,107 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.admin - -import joptsimple.OptionParser -import org.I0Itec.zkclient.ZkClient -import kafka.utils.{Utils, ZKStringSerializer, ZkUtils} - -object ListTopicCommand { - - def main(args: Array[String]): Unit = { - val parser = new OptionParser - val topicOpt = parser.accepts("topic", "REQUIRED: The topic to be listed. Defaults to all existing topics.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - .defaultsTo("") - val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + - "Multiple URLS can be given to allow fail-over.") - .withRequiredArg - .describedAs("urls") - .ofType(classOf[String]) - val reportUnderReplicatedPartitionsOpt = parser.accepts("under-replicated-partitions", - "if set, only show under replicated partitions") - val reportUnavailablePartitionsOpt = parser.accepts("unavailable-partitions", - "if set, only show partitions whose leader is not available") - - val options = parser.parse(args : _*) - - for(arg <- List(zkConnectOpt)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } - - val topic = options.valueOf(topicOpt) - val zkConnect = options.valueOf(zkConnectOpt) - val reportUnderReplicatedPartitions = if (options.has(reportUnderReplicatedPartitionsOpt)) true else false - val reportUnavailablePartitions = if (options.has(reportUnavailablePartitionsOpt)) true else false - var zkClient: ZkClient = null - try { - var topicList: Seq[String] = Nil - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) - - if (topic == "") - topicList = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath).sorted - else - topicList = List(topic) - - if (topicList.size <= 0) - println("no topics exist!") - - val liveBrokers = ZkUtils.getAllBrokersInCluster(zkClient).map(_.id).toSet - for (t <- topicList) - showTopic(t, zkClient, reportUnderReplicatedPartitions, reportUnavailablePartitions, liveBrokers) - } - catch { - case e: Throwable => - println("list topic failed because of " + e.getMessage) - println(Utils.stackTrace(e)) - } - finally { - if (zkClient != null) - zkClient.close() - } - } - - def showTopic(topic: String, zkClient: ZkClient, reportUnderReplicatedPartitions: Boolean, - reportUnavailablePartitions: Boolean, liveBrokers: Set[Int]) { - ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic) match { - case Some(topicPartitionAssignment) => - val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) - for ((partitionId, assignedReplicas) <- sortedPartitions) { - val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionId) - val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitionId) - if ((!reportUnderReplicatedPartitions && !reportUnavailablePartitions) || - (reportUnderReplicatedPartitions && inSyncReplicas.size < assignedReplicas.size) || - (reportUnavailablePartitions && (!leader.isDefined || !liveBrokers.contains(leader.get)))) { - print("topic: " + topic) - print("\tpartition: " + partitionId) - print("\tleader: " + (if(leader.isDefined) leader.get else "none")) - print("\treplicas: " + assignedReplicas.mkString(",")) - println("\tisr: " + inSyncReplicas.mkString(",")) - } - } - case None => - println("topic " + topic + " doesn't exist!") - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala index 34ed7aaa866e5..9b3c6aeaf77db 100644 --- a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala +++ b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala @@ -80,31 +80,27 @@ object PreferredReplicaLeaderElectionCommand extends Logging { val partition = p.get("partition").get.asInstanceOf[Int] TopicAndPartition(topic, partition) }.toSet - case None => throw new AdministrationException("Preferred replica election data is empty") + case None => throw new AdminOperationException("Preferred replica election data is empty") } - case None => throw new AdministrationException("Preferred replica election data is empty") + case None => throw new AdminOperationException("Preferred replica election data is empty") } } def writePreferredReplicaElectionData(zkClient: ZkClient, partitionsUndergoingPreferredReplicaElection: scala.collection.Set[TopicAndPartition]) { val zkPath = ZkUtils.PreferredReplicaLeaderElectionPath - var partitionsData: mutable.ListBuffer[String] = ListBuffer[String]() - for (p <- partitionsUndergoingPreferredReplicaElection) { - partitionsData += Utils.mergeJsonFields(Utils.mapToJsonFields(Map("topic" -> p.topic), valueInQuotes = true) ++ - Utils.mapToJsonFields(Map("partition" -> p.partition.toString), valueInQuotes = false)) - } - val jsonPartitionsData = Utils.seqToJson(partitionsData, valueInQuotes = false) - val jsonData = Utils.mapToJson(Map("version" -> 1.toString, "partitions" -> jsonPartitionsData), valueInQuotes = false) + val partitionsList = partitionsUndergoingPreferredReplicaElection.map(e => Map("topic" -> e.topic, "partition" -> e.partition)) + val jsonData = Json.encode(Map("version" -> 1, "partitions" -> partitionsList)) try { ZkUtils.createPersistentPath(zkClient, zkPath, jsonData) info("Created preferred replica election path with %s".format(jsonData)) } catch { case nee: ZkNodeExistsException => - val partitionsUndergoingPreferredReplicaElection = parsePreferredReplicaElectionData(ZkUtils.readData(zkClient, zkPath)._1) - throw new AdministrationException("Preferred replica leader election currently in progress for " + + val partitionsUndergoingPreferredReplicaElection = + PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(ZkUtils.readData(zkClient, zkPath)._1) + throw new AdminOperationException("Preferred replica leader election currently in progress for " + "%s. Aborting operation".format(partitionsUndergoingPreferredReplicaElection)) - case e2: Throwable => throw new AdministrationException(e2.toString) + case e2: Throwable => throw new AdminOperationException(e2.toString) } } } diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 2f706c94d340f..ac12aa6748158 100644 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -26,124 +26,27 @@ import kafka.common.{TopicAndPartition, AdminCommandFailedException} object ReassignPartitionsCommand extends Logging { def main(args: Array[String]): Unit = { - val parser = new OptionParser - 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("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 statusCheckJsonFileOpt = parser.accepts("status-check-json-file", "REQUIRED: The JSON file with the list of partitions and the " + - "new replicas they should be reassigned to, which can be obtained from the output of a dry run.") - .withRequiredArg - .describedAs("partition reassignment json file path") - .ofType(classOf[String]) - val options = parser.parse(args : _*) + val opts = new ReassignPartitionsCommandOptions(args) - for(arg <- List(zkConnectOpt)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } + // should have exactly one action + val actions = Seq(opts.generateOpt, opts.executeOpt, opts.verifyOpt).count(opts.options.has _) + if(actions != 1) { + opts.parser.printHelpOn(System.err) + Utils.croak("Command must include exactly one action: --generate, --execute or --verify") } - 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) - } + CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt) - val zkConnect = options.valueOf(zkConnectOpt) + val zkConnect = opts.options.valueOf(opts.zkConnectOpt) var zkClient: ZkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) try { - - var partitionsToBeReassigned : Map[TopicAndPartition, Seq[Int]] = new mutable.HashMap[TopicAndPartition, List[Int]]() - - if(options.has(statusCheckJsonFileOpt)) { - val jsonFile = options.valueOf(statusCheckJsonFileOpt) - val jsonString = Utils.readFileAsString(jsonFile) - val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentData(jsonString) - - println("Status of partition reassignment:") - val reassignedPartitionsStatus = checkIfReassignmentSucceeded(zkClient, partitionsToBeReassigned) - reassignedPartitionsStatus.foreach { partition => - partition._2 match { - case ReassignmentCompleted => - println("Reassignment of partition %s completed successfully".format(partition._1)) - case ReassignmentFailed => - println("Reassignment of partition %s failed".format(partition._1)) - case ReassignmentInProgress => - println("Reassignment of partition %s is still in progress".format(partition._1)) - } - } - } else 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(topicsToMoveJsonFileOpt) || options.has(manualAssignmentJsonFileOpt)) { - 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 following is the replica assignment. Save it for the status check option.\n" + - ZkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned)) - } - } + if(opts.options.has(opts.verifyOpt)) + verifyAssignment(zkClient, opts) + else if(opts.options.has(opts.generateOpt)) + generateAssignment(zkClient, opts) + else if (opts.options.has(opts.executeOpt)) + executeAssignment(zkClient, opts) } catch { case e: Throwable => println("Partitions reassignment failed due to " + e.getMessage) @@ -154,6 +57,76 @@ object ReassignPartitionsCommand extends Logging { } } + def verifyAssignment(zkClient: ZkClient, opts: ReassignPartitionsCommandOptions) { + if(!opts.options.has(opts.reassignmentJsonFileOpt)) { + opts.parser.printHelpOn(System.err) + Utils.croak("If --verify option is used, command must include --reassignment-json-file that was used during the --execute option") + } + val jsonFile = opts.options.valueOf(opts.reassignmentJsonFileOpt) + val jsonString = Utils.readFileAsString(jsonFile) + val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentData(jsonString) + + println("Status of partition reassignment:") + val reassignedPartitionsStatus = checkIfReassignmentSucceeded(zkClient, partitionsToBeReassigned) + reassignedPartitionsStatus.foreach { partition => + partition._2 match { + case ReassignmentCompleted => + println("Reassignment of partition %s completed successfully".format(partition._1)) + case ReassignmentFailed => + println("Reassignment of partition %s failed".format(partition._1)) + case ReassignmentInProgress => + println("Reassignment of partition %s is still in progress".format(partition._1)) + } + } + } + + def generateAssignment(zkClient: ZkClient, opts: ReassignPartitionsCommandOptions) { + if(!(opts.options.has(opts.topicsToMoveJsonFileOpt) && opts.options.has(opts.brokerListOpt))) { + opts.parser.printHelpOn(System.err) + Utils.croak("If --generate option is used, command must include both --topics-to-move-json-file and --broker-list options") + } + val topicsToMoveJsonFile = opts.options.valueOf(opts.topicsToMoveJsonFileOpt) + val brokerListToReassign = opts.options.valueOf(opts.brokerListOpt).split(',').map(_.toInt) + val topicsToMoveJsonString = Utils.readFileAsString(topicsToMoveJsonFile) + val topicsToReassign = ZkUtils.parseTopicsData(topicsToMoveJsonString) + val topicPartitionsToReassign = ZkUtils.getReplicaAssignmentForTopics(zkClient, topicsToReassign) + + var partitionsToBeReassigned : Map[TopicAndPartition, Seq[Int]] = new mutable.HashMap[TopicAndPartition, List[Int]]() + val groupedByTopic = topicPartitionsToReassign.groupBy(tp => tp._1.topic) + groupedByTopic.foreach { topicInfo => + val assignedReplicas = AdminUtils.assignReplicasToBrokers(brokerListToReassign, ZkUtils.getFilteredCluster(zkClient, brokerListToReassign), topicInfo._2.size, + topicInfo._2.head._2.size) + partitionsToBeReassigned ++= assignedReplicas.map(replicaInfo => (TopicAndPartition(topicInfo._1, replicaInfo._1) -> replicaInfo._2)) + } + val currentPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, partitionsToBeReassigned.map(_._1.topic).toSeq) + println("Current partition replica assignment\n\n%s" + .format(ZkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment))) + println("Proposed partition reassignment configuration\n\n%s".format(ZkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned))) + } + + def executeAssignment(zkClient: ZkClient, opts: ReassignPartitionsCommandOptions) { + if(!opts.options.has(opts.reassignmentJsonFileOpt)) { + opts.parser.printHelpOn(System.err) + Utils.croak("If --execute option is used, command must include --reassignment-json-file that was output " + + "during the --generate option") + } + val reassignmentJsonFile = opts.options.valueOf(opts.reassignmentJsonFileOpt) + val reassignmentJsonString = Utils.readFileAsString(reassignmentJsonFile) + val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentData(reassignmentJsonString) + if (partitionsToBeReassigned.isEmpty) + throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(reassignmentJsonFile)) + val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned) + // before starting assignment, output the current replica assignment to facilitate rollback + val currentPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, partitionsToBeReassigned.map(_._1.topic).toSeq) + println("Current partition replica assignment\n\n%s\n\nSave this to use as the --reassignment-json-file option during rollback" + .format(ZkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment))) + // start the reassignment + if(reassignPartitionsCommand.reassignPartitions()) + println("Successfully started reassignment of partitions %s".format(ZkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned))) + else + println("Failed to reassign partitions %s".format(partitionsToBeReassigned)) + } + private def checkIfReassignmentSucceeded(zkClient: ZkClient, partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]) :Map[TopicAndPartition, ReassignmentStatus] = { val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas) @@ -175,10 +148,46 @@ object ReassignPartitionsCommand extends Logging { val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition) if(assignedReplicas == newReplicas) ReassignmentCompleted - else + else { + println(("ERROR: Assigned replicas (%s) don't match the list of replicas for reassignment (%s)" + + " for partition %s").format(assignedReplicas.mkString(","), newReplicas.mkString(","), topicAndPartition)) ReassignmentFailed + } } } + + class ReassignPartitionsCommandOptions(args: Array[String]) { + val parser = new OptionParser + + 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 generateOpt = parser.accepts("generate", "Generate a candidate partition reassignment configuration." + + " Note that this only generates a candidate assignment, it does not execute it.") + val executeOpt = parser.accepts("execute", "Kick off the reassignment as specified by the --reassignment-json-file option.") + val verifyOpt = parser.accepts("verify", "Verify if the reassignment completed as specified by the --reassignment-json-file option.") + val reassignmentJsonFileOpt = parser.accepts("reassignment-json-file", "The JSON file with the partition reassignment configuration" + + "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 topicsToMoveJsonFileOpt = parser.accepts("topics-to-move-json-file", "Generate a reassignment configuration to move the partitions" + + " of the specified topics to the list of brokers specified by the --broker-list option. The format to use is - \n" + + "{\"topics\":\n\t[{\"topic\": \"foo\"},{\"topic\": \"foo1\"}],\n\"version\":1\n}") + .withRequiredArg + .describedAs("topics to reassign 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 if --topics-to-move-json-file is used to generate reassignment configuration") + .withRequiredArg + .describedAs("brokerlist") + .ofType(classOf[String]) + + val options = parser.parse(args : _*) + } } class ReassignPartitionsCommand(zkClient: ZkClient, partitions: collection.Map[TopicAndPartition, collection.Seq[Int]]) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala new file mode 100644 index 0000000000000..d27e91f621e87 --- /dev/null +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -0,0 +1,272 @@ +/** + * 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._ +import java.util.Properties +import kafka.utils._ +import org.I0Itec.zkclient.ZkClient +import scala.collection._ +import scala.collection.JavaConversions._ +import kafka.cluster.Broker +import kafka.log.LogConfig +import kafka.consumer.Whitelist + +object TopicCommand { + + def main(args: Array[String]): Unit = { + + val opts = new TopicCommandOptions(args) + + // should have exactly one action + val actions = Seq(opts.createOpt, opts.deleteOpt, opts.listOpt, opts.alterOpt, opts.describeOpt).count(opts.options.has _) + if(actions != 1) { + System.err.println("Command must include exactly one action: --list, --describe, --create, --delete, or --alter") + opts.parser.printHelpOn(System.err) + System.exit(1) + } + + CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt) + if (!opts.options.has(opts.listOpt)) CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.topicOpt) + + val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) + + try { + if(opts.options.has(opts.createOpt)) + createTopic(zkClient, opts) + else if(opts.options.has(opts.alterOpt)) + alterTopic(zkClient, opts) + else if(opts.options.has(opts.deleteOpt)) + deleteTopic(zkClient, opts) + else if(opts.options.has(opts.listOpt)) + listTopics(zkClient, opts) + else if(opts.options.has(opts.describeOpt)) + describeTopic(zkClient, opts) + } catch { + case e => + println("Error while executing topic command " + e.getMessage) + println(Utils.stackTrace(e)) + } finally { + zkClient.close() + } + } + + private def getTopics(zkClient: ZkClient, opts: TopicCommandOptions): Seq[String] = { + val topicsSpec = opts.options.valueOf(opts.topicOpt) + val topicsFilter = new Whitelist(topicsSpec) + val allTopics = ZkUtils.getAllTopics(zkClient) + allTopics.filter(topicsFilter.isTopicAllowed).sorted + } + + def createTopic(zkClient: ZkClient, opts: TopicCommandOptions) { + val topic = opts.options.valueOf(opts.topicOpt) + val configs = parseTopicConfigsToBeAdded(opts) + val maxReplicaPerRack = if (opts.options.has(opts.maxRackReplicationOpt)) opts.options.valueOf(opts.maxRackReplicationOpt).intValue else -1 + if (opts.options.has(opts.replicaAssignmentOpt)) { + val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt)) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs, maxReplicaPerRack) + } else { + CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt) + val partitions = opts.options.valueOf(opts.partitionsOpt).intValue + val replicas = opts.options.valueOf(opts.replicationFactorOpt).intValue + AdminUtils.createTopic(zkClient, topic, partitions, replicas, maxReplicaPerRack, configs) + } + println("Created topic \"%s\".".format(topic)) + } + + def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) { + val topics = getTopics(zkClient, opts) + topics.foreach { topic => + if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) { + val configsToBeAdded = parseTopicConfigsToBeAdded(opts) + val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts) + // compile the final set of configs + val configs = AdminUtils.fetchTopicConfig(zkClient, topic) + configs.putAll(configsToBeAdded) + configsToBeDeleted.foreach(config => configs.remove(config)) + AdminUtils.changeTopicConfig(zkClient, topic, configs) + println("Updated config for topic \"%s\".".format(topic)) + } + if(opts.options.has(opts.maxRackReplicationOpt)) { + Utils.croak("Changing the max-rack-replication is not supported.") + } + if(opts.options.has(opts.partitionsOpt)) { + println("WARNING: If partitions are increased for a topic that has a key, the partition " + + "logic or ordering of the messages will be affected") + val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue + val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt) + AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr) + println("adding partitions succeeded!") + } + if(opts.options.has(opts.replicationFactorOpt)) + Utils.croak("Changing the replication factor is not supported.") + } + } + + def deleteTopic(zkClient: ZkClient, opts: TopicCommandOptions) { + val topics = getTopics(zkClient, opts) + topics.foreach { topic => + AdminUtils.deleteTopic(zkClient, topic) + println("Topic \"%s\" deleted.".format(topic)) + } + } + + def listTopics(zkClient: ZkClient, opts: TopicCommandOptions) { + if(opts.options.has(opts.topicsWithOverridesOpt)) { + ZkUtils.getAllTopics(zkClient).sorted.foreach { topic => + val configs = AdminUtils.fetchTopicConfig(zkClient, topic) + if(configs.size() != 0) { + val replicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) + val numPartitions = replicaAssignment.size + val replicationFactor = replicaAssignment.head._2.size + println("\nTopic:%s\tPartitionCount:%d\tReplicationFactor:%d\tConfigs:%s".format(topic, numPartitions, + replicationFactor, configs.map(kv => kv._1 + "=" + kv._2).mkString(","))) + } + } + } else { + for(topic <- ZkUtils.getAllTopics(zkClient).sorted) + println(topic) + } + } + + def describeTopic(zkClient: ZkClient, opts: TopicCommandOptions) { + val topics = getTopics(zkClient, opts) + val reportUnderReplicatedPartitions = if (opts.options.has(opts.reportUnderReplicatedPartitionsOpt)) true else false + val reportUnavailablePartitions = if (opts.options.has(opts.reportUnavailablePartitionsOpt)) true else false + val liveBrokers = ZkUtils.getAllBrokersInCluster(zkClient).map(_.id).toSet + for (topic <- topics) { + ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic) match { + case Some(topicPartitionAssignment) => + val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) + if (!reportUnavailablePartitions && !reportUnderReplicatedPartitions) { + println(topic) + val config = AdminUtils.fetchTopicConfig(zkClient, topic) + println("\tconfigs: " + config.map(kv => kv._1 + " = " + kv._2).mkString(", ")) + println("\tpartitions: " + sortedPartitions.size) + } + for ((partitionId, assignedReplicas) <- sortedPartitions) { + val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionId) + val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitionId) + if ((!reportUnderReplicatedPartitions && !reportUnavailablePartitions) || + (reportUnderReplicatedPartitions && inSyncReplicas.size < assignedReplicas.size) || + (reportUnavailablePartitions && (!leader.isDefined || !liveBrokers.contains(leader.get)))) { + print("\t\ttopic: " + topic) + print("\tpartition: " + partitionId) + print("\tleader: " + (if(leader.isDefined) leader.get else "none")) + print("\treplicas: " + assignedReplicas.mkString(",")) + println("\tisr: " + inSyncReplicas.mkString(",")) + } + } + case None => + println("topic " + topic + " doesn't exist!") + } + } + } + + def formatBroker(broker: Broker) = broker.id + " (" + broker.host + ":" + broker.port + ")" + + def parseTopicConfigsToBeAdded(opts: TopicCommandOptions): Properties = { + val configsToBeAdded = opts.options.valuesOf(opts.configOpt).map(_.split("""\s*=\s*""")) + require(configsToBeAdded.forall(config => config.length == 2), + "Invalid topic config: all configs to be added must be in the format \"key=val\".") + val props = new Properties + configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim)) + LogConfig.validate(props) + props + } + + def parseTopicConfigsToBeDeleted(opts: TopicCommandOptions): Seq[String] = { + val configsToBeDeleted = opts.options.valuesOf(opts.deleteConfigOpt).map(_.split("""\s*=\s*""")) + if(opts.options.has(opts.createOpt)) + require(configsToBeDeleted.size == 0, "Invalid topic config: all configs on create topic must be in the format \"key=val\".") + require(configsToBeDeleted.forall(config => config.length == 1), + "Invalid topic config: all configs to be deleted must be in the format \"key\".") + val propsToBeDeleted = new Properties + configsToBeDeleted.foreach(pair => propsToBeDeleted.setProperty(pair(0).trim, "")) + LogConfig.validateNames(propsToBeDeleted) + configsToBeDeleted.map(pair => pair(0)) + } + + def parseReplicaAssignment(replicaAssignmentList: String): Map[Int, List[Int]] = { + val partitionList = replicaAssignmentList.split(",") + val ret = new mutable.HashMap[Int, List[Int]]() + for (i <- 0 until partitionList.size) { + val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) + ret.put(i, brokerList.toList) + if (ret(i).size != ret(0).size) + throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList) + } + ret.toMap + } + + class TopicCommandOptions(args: Array[String]) { + val parser = new OptionParser + 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 listOpt = parser.accepts("list", "List all available topics.") + val createOpt = parser.accepts("create", "Create a new topic.") + val alterOpt = parser.accepts("alter", "Alter the configuration for the topic.") + val deleteOpt = parser.accepts("delete", "Delete the topic.") + val describeOpt = parser.accepts("describe", "List details for the given topics.") + val helpOpt = parser.accepts("help", "Print usage information.") + val topicOpt = parser.accepts("topic", "The topic to be create, alter, delete, or describe. Can also accept a regular " + + "expression except for --create option") + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) + val configOpt = parser.accepts("config", "A topic configuration override for the topic being created or altered.") + .withRequiredArg + .describedAs("name=value") + .ofType(classOf[String]) + val deleteConfigOpt = parser.accepts("deleteConfig", "A topic configuration override to be removed for an existing topic") + .withRequiredArg + .describedAs("name") + .ofType(classOf[String]) + val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " + + "altered (WARNING: If partitions are increased for a topic that has a key, the partition logic or ordering of the messages will be affected") + .withRequiredArg + .describedAs("# of partitions") + .ofType(classOf[java.lang.Integer]) + val replicationFactorOpt = parser.accepts("replication-factor", "The replication factor for each partition in the topic being created.") + .withRequiredArg + .describedAs("replication factor") + .ofType(classOf[java.lang.Integer]) + val maxRackReplicationOpt = parser.accepts("max-rack-replication", "The maximum number of replicas assigned to a single rack for each partition in the topic being created.") + .withRequiredArg + .describedAs("max rack replication") + .ofType(classOf[java.lang.Integer]) + val replicaAssignmentOpt = parser.accepts("replica-assignment", "A list of manual partition-to-broker assignments for the topic being created.") + .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]) + val reportUnderReplicatedPartitionsOpt = parser.accepts("under-replicated-partitions", + "if set when describing topics, only show under replicated partitions") + val reportUnavailablePartitionsOpt = parser.accepts("unavailable-partitions", + "if set when describing topics, only show partitions whose leader is not available") + val topicsWithOverridesOpt = parser.accepts("topics-with-overrides", + "if set when listing topics, only show topics that have overridden configs") + + + val options = parser.parse(args : _*) + } + +} diff --git a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala index ad6a20d43f9bb..7dacb20237880 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala @@ -58,6 +58,15 @@ case class ControlledShutdownRequest(val versionId: Short, } override def toString(): String = { + describe(true) + } + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val errorResponse = ControlledShutdownResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]]), Set.empty[TopicAndPartition]) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } + + override def describe(details: Boolean = false): String = { val controlledShutdownRequest = new StringBuilder controlledShutdownRequest.append("Name: " + this.getClass.getSimpleName) controlledShutdownRequest.append("; Version: " + versionId) @@ -65,9 +74,4 @@ case class ControlledShutdownRequest(val versionId: Short, controlledShutdownRequest.append("; BrokerId: " + brokerId) controlledShutdownRequest.toString() } - - override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - val errorResponse = ControlledShutdownResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]]), Set.empty[TopicAndPartition]) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) - } -} +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala index b7c844892a975..a80aa4924cfe9 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala @@ -67,4 +67,7 @@ case class ControlledShutdownResponse(override val correlationId: Int, buffer.putInt(topicAndPartition.partition) } } + + override def describe(details: Boolean):String = { toString } + } \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index fb2a2306003ac..dea118a2e3792 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -132,7 +132,7 @@ case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentV }) } - def isFromFollower = replicaId != Request.OrdinaryConsumerId && replicaId != Request.DebuggingConsumerId + def isFromFollower = Request.isReplicaIdFromFollower(replicaId) def isFromOrdinaryConsumer = replicaId == Request.OrdinaryConsumerId @@ -141,16 +141,7 @@ case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentV def numPartitions = requestInfo.size override def toString(): String = { - val fetchRequest = new StringBuilder - fetchRequest.append("Name: " + this.getClass.getSimpleName) - fetchRequest.append("; Version: " + versionId) - fetchRequest.append("; CorrelationId: " + correlationId) - fetchRequest.append("; ClientId: " + clientId) - fetchRequest.append("; ReplicaId: " + replicaId) - fetchRequest.append("; MaxWait: " + maxWait + " ms") - fetchRequest.append("; MinBytes: " + minBytes + " bytes") - fetchRequest.append("; RequestInfo: " + requestInfo.mkString(",")) - fetchRequest.toString() + describe(true) } override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { @@ -161,8 +152,21 @@ case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentV val errorResponse = FetchResponse(correlationId, fetchResponsePartitionData) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(errorResponse))) } -} + override def describe(details: Boolean): String = { + val fetchRequest = new StringBuilder + fetchRequest.append("Name: " + this.getClass.getSimpleName) + fetchRequest.append("; Version: " + versionId) + fetchRequest.append("; CorrelationId: " + correlationId) + fetchRequest.append("; ClientId: " + clientId) + fetchRequest.append("; ReplicaId: " + replicaId) + fetchRequest.append("; MaxWait: " + maxWait + " ms") + fetchRequest.append("; MinBytes: " + minBytes + " bytes") + if(details) + fetchRequest.append("; RequestInfo: " + requestInfo.mkString(",")) + fetchRequest.toString() + } +} @nonthreadsafe class FetchRequestBuilder() { diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index e5287427438ea..d117f10f724b0 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -42,11 +42,7 @@ object FetchResponsePartitionData { } case class FetchResponsePartitionData(error: Short = ErrorMapping.NoError, hw: Long = -1L, messages: MessageSet) { - val sizeInBytes = FetchResponsePartitionData.headerSize + messages.sizeInBytes - - def this(messages: MessageSet) = this(ErrorMapping.NoError, -1L, messages) - } // SENDS diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index 981d2bbecf2fa..a984878fbd814 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -37,11 +37,7 @@ case class LeaderAndIsr(var leader: Int, var leaderEpoch: Int, var isr: List[Int def this(leader: Int, isr: List[Int]) = this(leader, LeaderAndIsr.initialLeaderEpoch, isr, LeaderAndIsr.initialZKVersion) override def toString(): String = { - val jsonDataMap = new collection.mutable.HashMap[String, String] - jsonDataMap.put("leader", leader.toString) - jsonDataMap.put("leaderEpoch", leaderEpoch.toString) - jsonDataMap.put("ISR", isr.mkString(",")) - Utils.mapToJson(jsonDataMap, valueInQuotes = true) + Json.encode(Map("leader" -> leader, "leader_epoch" -> leaderEpoch, "isr" -> isr)) } } @@ -177,6 +173,18 @@ case class LeaderAndIsrRequest (versionId: Short, } override def toString(): String = { + describe(true) + } + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val responseMap = partitionStateInfos.map { + case (topicAndPartition, partitionAndState) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) + } + val errorResponse = LeaderAndIsrResponse(correlationId, responseMap) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } + + override def describe(details: Boolean): String = { val leaderAndIsrRequest = new StringBuilder leaderAndIsrRequest.append("Name:" + this.getClass.getSimpleName) leaderAndIsrRequest.append(";Version:" + versionId) @@ -184,16 +192,9 @@ case class LeaderAndIsrRequest (versionId: Short, leaderAndIsrRequest.append(";ControllerEpoch:" + controllerEpoch) leaderAndIsrRequest.append(";CorrelationId:" + correlationId) leaderAndIsrRequest.append(";ClientId:" + clientId) - leaderAndIsrRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) leaderAndIsrRequest.append(";Leaders:" + leaders.mkString(",")) + if(details) + leaderAndIsrRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) leaderAndIsrRequest.toString() } - - override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - val responseMap = partitionStateInfos.map { - case (topicAndPartition, partitionAndState) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - } - val errorResponse = LeaderAndIsrResponse(correlationId, responseMap) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) - } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala index 378b2b371377a..f63644448bb5a 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala @@ -69,4 +69,7 @@ case class LeaderAndIsrResponse(override val correlationId: Int, buffer.putShort(value) } } + + override def describe(details: Boolean):String = { toString } + } \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala new file mode 100644 index 0000000000000..4d1fa5cbfde92 --- /dev/null +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.utils.Logging +import kafka.network.{RequestChannel, BoundedByteBufferSend} +import kafka.common.{ErrorMapping, TopicAndPartition, OffsetMetadataAndError} +import kafka.network.RequestChannel.Response +object OffsetCommitRequest extends Logging { + val CurrentVersion: Short = 0 + val DefaultClientId = "" + + def readFrom(buffer: ByteBuffer): OffsetCommitRequest = { + // Read values from the envelope + val versionId = buffer.getShort + val correlationId = buffer.getInt + val clientId = readShortString(buffer) + + // Read the OffsetRequest + val consumerGroupId = readShortString(buffer) + val topicCount = buffer.getInt + val pairs = (1 to topicCount).flatMap(_ => { + val topic = readShortString(buffer) + val partitionCount = buffer.getInt + (1 to partitionCount).map(_ => { + val partitionId = buffer.getInt + val offset = buffer.getLong + val metadata = readShortString(buffer) + (TopicAndPartition(topic, partitionId), OffsetMetadataAndError(offset, metadata)) + }) + }) + OffsetCommitRequest(consumerGroupId, Map(pairs:_*), versionId, correlationId, clientId) + } +} + +case class OffsetCommitRequest(groupId: String, + requestInfo: Map[TopicAndPartition, OffsetMetadataAndError], + versionId: Short = OffsetCommitRequest.CurrentVersion, + override val correlationId: Int = 0, + clientId: String = OffsetCommitRequest.DefaultClientId) + extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey), correlationId) { + + lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) + + def writeTo(buffer: ByteBuffer) { + // Write envelope + buffer.putShort(versionId) + buffer.putInt(correlationId) + writeShortString(buffer, clientId) + + // Write OffsetCommitRequest + writeShortString(buffer, groupId) // consumer group + buffer.putInt(requestInfoGroupedByTopic.size) // number of topics + requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError] + writeShortString(buffer, t1._1) // topic + buffer.putInt(t1._2.size) // number of partitions for this topic + t1._2.foreach( t2 => { + buffer.putInt(t2._1.partition) // partition + buffer.putLong(t2._2.offset) // offset + writeShortString(buffer, t2._2.metadata) // metadata + }) + }) + } + + override def sizeInBytes = + 2 + /* versionId */ + 4 + /* correlationId */ + shortStringLength(clientId) + + shortStringLength(groupId) + + 4 + /* topic count */ + requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { + val (topic, offsets) = topicAndOffsets + count + + shortStringLength(topic) + /* topic */ + 4 + /* number of partitions */ + offsets.foldLeft(0)((innerCount, offsetAndMetadata) => { + innerCount + + 4 /* partition */ + + 8 /* offset */ + + shortStringLength(offsetAndMetadata._2.metadata) + }) + }) + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val responseMap = requestInfo.map { + case (topicAndPartition, offset) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) + }.toMap + val errorResponse = OffsetCommitResponse(requestInfo=responseMap, correlationId=correlationId) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } + + override def describe(details: Boolean): String = { + val offsetCommitRequest = new StringBuilder + offsetCommitRequest.append("Name: " + this.getClass.getSimpleName) + offsetCommitRequest.append("; Version: " + versionId) + offsetCommitRequest.append("; CorrelationId: " + correlationId) + offsetCommitRequest.append("; ClientId: " + clientId) + offsetCommitRequest.append("; GroupId: " + groupId) + if(details) + offsetCommitRequest.append("; RequestInfo: " + requestInfo.mkString(",")) + offsetCommitRequest.toString() + } + + override def toString(): String = { + describe(true) + } +} diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala new file mode 100644 index 0000000000000..9e1795f9db15e --- /dev/null +++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.TopicAndPartition +import kafka.utils.Logging + +object OffsetCommitResponse extends Logging { + val CurrentVersion: Short = 0 + + def readFrom(buffer: ByteBuffer): OffsetCommitResponse = { + val correlationId = buffer.getInt + val topicCount = buffer.getInt + val pairs = (1 to topicCount).flatMap(_ => { + val topic = readShortString(buffer) + val partitionCount = buffer.getInt + (1 to partitionCount).map(_ => { + val partitionId = buffer.getInt + val error = buffer.getShort + (TopicAndPartition(topic, partitionId), error) + }) + }) + OffsetCommitResponse(Map(pairs:_*), correlationId) + } +} + +case class OffsetCommitResponse(requestInfo: Map[TopicAndPartition, Short], + override val correlationId: Int = 0) + extends RequestOrResponse(correlationId=correlationId) { + + lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) + + def writeTo(buffer: ByteBuffer) { + buffer.putInt(correlationId) + buffer.putInt(requestInfoGroupedByTopic.size) // number of topics + requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, Short] + writeShortString(buffer, t1._1) // topic + buffer.putInt(t1._2.size) // number of partitions for this topic + t1._2.foreach( t2 => { // TopicAndPartition -> Short + buffer.putInt(t2._1.partition) + buffer.putShort(t2._2) //error + }) + }) + } + + override def sizeInBytes = + 4 + /* correlationId */ + 4 + /* topic count */ + requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { + val (topic, offsets) = topicAndOffsets + count + + shortStringLength(topic) + /* topic */ + 4 + /* number of partitions */ + offsets.size * ( + 4 + /* partition */ + 2 /* error */ + ) + }) + + override def describe(details: Boolean):String = { toString } + +} + diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala new file mode 100644 index 0000000000000..7036532db1405 --- /dev/null +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.utils.Logging +import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.network.RequestChannel.Response +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition} +object OffsetFetchRequest extends Logging { + val CurrentVersion: Short = 0 + val DefaultClientId = "" + + def readFrom(buffer: ByteBuffer): OffsetFetchRequest = { + // Read values from the envelope + val versionId = buffer.getShort + val correlationId = buffer.getInt + val clientId = readShortString(buffer) + + // Read the OffsetFetchRequest + val consumerGroupId = readShortString(buffer) + val topicCount = buffer.getInt + val pairs = (1 to topicCount).flatMap(_ => { + val topic = readShortString(buffer) + val partitionCount = buffer.getInt + (1 to partitionCount).map(_ => { + val partitionId = buffer.getInt + TopicAndPartition(topic, partitionId) + }) + }) + OffsetFetchRequest(consumerGroupId, pairs, versionId, correlationId, clientId) + } +} + +case class OffsetFetchRequest(groupId: String, + requestInfo: Seq[TopicAndPartition], + versionId: Short = OffsetFetchRequest.CurrentVersion, + override val correlationId: Int = 0, + clientId: String = OffsetFetchRequest.DefaultClientId) + extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey), correlationId) { + + lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic) + + def writeTo(buffer: ByteBuffer) { + // Write envelope + buffer.putShort(versionId) + buffer.putInt(correlationId) + writeShortString(buffer, clientId) + + // Write OffsetFetchRequest + writeShortString(buffer, groupId) // consumer group + buffer.putInt(requestInfoGroupedByTopic.size) // number of topics + requestInfoGroupedByTopic.foreach( t1 => { // (topic, Seq[TopicAndPartition]) + writeShortString(buffer, t1._1) // topic + buffer.putInt(t1._2.size) // number of partitions for this topic + t1._2.foreach( t2 => { + buffer.putInt(t2.partition) + }) + }) + } + + override def sizeInBytes = + 2 + /* versionId */ + 4 + /* correlationId */ + shortStringLength(clientId) + + shortStringLength(groupId) + + 4 + /* topic count */ + requestInfoGroupedByTopic.foldLeft(0)((count, t) => { + count + shortStringLength(t._1) + /* topic */ + 4 + /* number of partitions */ + t._2.size * 4 /* partition */ + }) + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val responseMap = requestInfo.map { + case (topicAndPartition) => (topicAndPartition, OffsetMetadataAndError( + offset=OffsetMetadataAndError.InvalidOffset, + error=ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + )) + }.toMap + val errorResponse = OffsetFetchResponse(requestInfo=responseMap, correlationId=correlationId) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } + + override def describe(details: Boolean): String = { + val offsetFetchRequest = new StringBuilder + offsetFetchRequest.append("Name: " + this.getClass.getSimpleName) + offsetFetchRequest.append("; Version: " + versionId) + offsetFetchRequest.append("; CorrelationId: " + correlationId) + offsetFetchRequest.append("; ClientId: " + clientId) + offsetFetchRequest.append("; GroupId: " + groupId) + if(details) + offsetFetchRequest.append("; RequestInfo: " + requestInfo.mkString(",")) + offsetFetchRequest.toString() + } + + override def toString(): String = { + describe(true) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala new file mode 100644 index 0000000000000..c1222f422ddb6 --- /dev/null +++ b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.{TopicAndPartition, OffsetMetadataAndError} +import kafka.utils.Logging + +object OffsetFetchResponse extends Logging { + val CurrentVersion: Short = 0 + + def readFrom(buffer: ByteBuffer): OffsetFetchResponse = { + val correlationId = buffer.getInt + val topicCount = buffer.getInt + val pairs = (1 to topicCount).flatMap(_ => { + val topic = readShortString(buffer) + val partitionCount = buffer.getInt + (1 to partitionCount).map(_ => { + val partitionId = buffer.getInt + val offset = buffer.getLong + val metadata = readShortString(buffer) + val error = buffer.getShort + (TopicAndPartition(topic, partitionId), OffsetMetadataAndError(offset, metadata, error)) + }) + }) + OffsetFetchResponse(Map(pairs:_*), correlationId) + } +} + +case class OffsetFetchResponse(requestInfo: Map[TopicAndPartition, OffsetMetadataAndError], + override val correlationId: Int = 0) + extends RequestOrResponse(correlationId = correlationId) { + + lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) + + def writeTo(buffer: ByteBuffer) { + buffer.putInt(correlationId) + buffer.putInt(requestInfoGroupedByTopic.size) // number of topics + requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError] + writeShortString(buffer, t1._1) // topic + buffer.putInt(t1._2.size) // number of partitions for this topic + t1._2.foreach( t2 => { // TopicAndPartition -> OffsetMetadataAndError + buffer.putInt(t2._1.partition) + buffer.putLong(t2._2.offset) + writeShortString(buffer, t2._2.metadata) + buffer.putShort(t2._2.error) + }) + }) + } + + override def sizeInBytes = + 4 + /* correlationId */ + 4 + /* topic count */ + requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { + val (topic, offsets) = topicAndOffsets + count + + shortStringLength(topic) + /* topic */ + 4 + /* number of partitions */ + offsets.foldLeft(0)((innerCount, offsetsAndMetadata) => { + innerCount + + 4 /* partition */ + + 8 /* offset */ + + shortStringLength(offsetsAndMetadata._2.metadata) + + 2 /* error */ + }) + }) + + override def describe(details: Boolean):String = { toString } +} + diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala index 0a94a6c352581..7cbc26c6e3842 100644 --- a/core/src/main/scala/kafka/api/OffsetRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetRequest.scala @@ -108,14 +108,7 @@ case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequ def isFromDebuggingClient = replicaId == Request.DebuggingConsumerId override def toString(): String = { - val offsetRequest = new StringBuilder - offsetRequest.append("Name: " + this.getClass.getSimpleName) - offsetRequest.append("; Version: " + versionId) - offsetRequest.append("; CorrelationId: " + correlationId) - offsetRequest.append("; ClientId: " + clientId) - offsetRequest.append("; RequestInfo: " + requestInfo.mkString(",")) - offsetRequest.append("; ReplicaId: " + replicaId) - offsetRequest.toString() + describe(true) } override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { @@ -126,4 +119,16 @@ case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequ val errorResponse = OffsetResponse(correlationId, partitionOffsetResponseMap) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) } -} + + override def describe(details: Boolean): String = { + val offsetRequest = new StringBuilder + offsetRequest.append("Name: " + this.getClass.getSimpleName) + offsetRequest.append("; Version: " + versionId) + offsetRequest.append("; CorrelationId: " + correlationId) + offsetRequest.append("; ClientId: " + clientId) + offsetRequest.append("; ReplicaId: " + replicaId) + if(details) + offsetRequest.append("; RequestInfo: " + requestInfo.mkString(",")) + offsetRequest.toString() + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/OffsetResponse.scala b/core/src/main/scala/kafka/api/OffsetResponse.scala index 08dc3cd3d166e..0e1d6e362a1ce 100644 --- a/core/src/main/scala/kafka/api/OffsetResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetResponse.scala @@ -44,7 +44,11 @@ object OffsetResponse { } -case class PartitionOffsetsResponse(error: Short, offsets: Seq[Long]) +case class PartitionOffsetsResponse(error: Short, offsets: Seq[Long]) { + override def toString(): String = { + new String("error: " + ErrorMapping.exceptionFor(error).getClass.getName + " offsets: " + offsets.mkString) + } +} case class OffsetResponse(override val correlationId: Int, @@ -90,5 +94,6 @@ case class OffsetResponse(override val correlationId: Int, } } + override def describe(details: Boolean):String = { toString } } diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index c606351117364..0c295a2fe6712 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -123,15 +123,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, def numPartitions = data.size override def toString(): String = { - val producerRequest = new StringBuilder - producerRequest.append("Name: " + this.getClass.getSimpleName) - producerRequest.append("; Version: " + versionId) - producerRequest.append("; CorrelationId: " + correlationId) - producerRequest.append("; ClientId: " + clientId) - producerRequest.append("; RequiredAcks: " + requiredAcks) - producerRequest.append("; AckTimeoutMs: " + ackTimeoutMs + " ms") - producerRequest.append("; TopicAndPartition: " + topicPartitionMessageSizeMap.mkString(",")) - producerRequest.toString() + describe(true) } override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { @@ -148,6 +140,20 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, } } + override def describe(details: Boolean): String = { + val producerRequest = new StringBuilder + producerRequest.append("Name: " + this.getClass.getSimpleName) + producerRequest.append("; Version: " + versionId) + producerRequest.append("; CorrelationId: " + correlationId) + producerRequest.append("; ClientId: " + clientId) + producerRequest.append("; RequiredAcks: " + requiredAcks) + producerRequest.append("; AckTimeoutMs: " + ackTimeoutMs + " ms") + if(details) + producerRequest.append("; TopicAndPartition: " + topicPartitionMessageSizeMap.mkString(",")) + producerRequest.toString() + } + + def emptyData(){ data.clear() } diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index d59c5bb8b8d73..06261b9136399 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -87,5 +87,7 @@ case class ProducerResponse(override val correlationId: Int, } }) } + + override def describe(details: Boolean):String = { toString } } diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala index e2ce9bd9e6ade..c81214fec0bf7 100644 --- a/core/src/main/scala/kafka/api/RequestKeys.scala +++ b/core/src/main/scala/kafka/api/RequestKeys.scala @@ -29,6 +29,8 @@ object RequestKeys { val StopReplicaKey: Short = 5 val UpdateMetadataKey: Short = 6 val ControlledShutdownKey: Short = 7 + val OffsetCommitKey: Short = 8 + val OffsetFetchKey: Short = 9 val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]= Map(ProduceKey -> ("Produce", ProducerRequest.readFrom), @@ -38,7 +40,9 @@ object RequestKeys { LeaderAndIsrKey -> ("LeaderAndIsr", LeaderAndIsrRequest.readFrom), StopReplicaKey -> ("StopReplica", StopReplicaRequest.readFrom), UpdateMetadataKey -> ("UpdateMetadata", UpdateMetadataRequest.readFrom), - ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom)) + ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom), + OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom), + OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom)) def nameForKey(key: Short): String = { keyToNameAndDeserializerMap.get(key) match { diff --git a/core/src/main/scala/kafka/api/RequestOrResponse.scala b/core/src/main/scala/kafka/api/RequestOrResponse.scala index b62330be6241c..708e547a358c9 100644 --- a/core/src/main/scala/kafka/api/RequestOrResponse.scala +++ b/core/src/main/scala/kafka/api/RequestOrResponse.scala @@ -24,15 +24,24 @@ import kafka.utils.Logging object Request { val OrdinaryConsumerId: Int = -1 val DebuggingConsumerId: Int = -2 + + // Followers use broker id as the replica id, which are non-negative int. + def isReplicaIdFromFollower(replicaId: Int): Boolean = (replicaId >= 0) } -private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None, val correlationId: Int) extends Logging{ +private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None, val correlationId: Int) extends Logging { def sizeInBytes: Int def writeTo(buffer: ByteBuffer): Unit def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {} + + /* The purpose of this API is to return a string description of the Request mainly for the purpose of request logging. + * This API has no meaning for a Response object. + * @param details If this is false, omit the parts of the request description that are proportional to the number of + * topics or partitions. This is mainly to control the amount of request logging. */ + def describe(details: Boolean):String } diff --git a/core/src/main/scala/kafka/api/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala index efd7046970244..820f0f57b0084 100644 --- a/core/src/main/scala/kafka/api/StopReplicaRequest.scala +++ b/core/src/main/scala/kafka/api/StopReplicaRequest.scala @@ -98,6 +98,18 @@ case class StopReplicaRequest(versionId: Short, } override def toString(): String = { + describe(true) + } + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val responseMap = partitions.map { + case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) + }.toMap + val errorResponse = StopReplicaResponse(correlationId, responseMap) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } + + override def describe(details: Boolean): String = { val stopReplicaRequest = new StringBuilder stopReplicaRequest.append("Name: " + this.getClass.getSimpleName) stopReplicaRequest.append("; Version: " + versionId) @@ -106,15 +118,8 @@ case class StopReplicaRequest(versionId: Short, stopReplicaRequest.append("; DeletePartitions: " + deletePartitions) stopReplicaRequest.append("; ControllerId: " + controllerId) stopReplicaRequest.append("; ControllerEpoch: " + controllerEpoch) - stopReplicaRequest.append("; Partitions: " + partitions.mkString(",")) + if(details) + stopReplicaRequest.append("; Partitions: " + partitions.mkString(",")) stopReplicaRequest.toString() } - - override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - val responseMap = partitions.map { - case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - }.toMap - val errorResponse = StopReplicaResponse(correlationId, responseMap) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) - } -} +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/StopReplicaResponse.scala b/core/src/main/scala/kafka/api/StopReplicaResponse.scala index c82eadd6a9f7c..d7e36308263ae 100644 --- a/core/src/main/scala/kafka/api/StopReplicaResponse.scala +++ b/core/src/main/scala/kafka/api/StopReplicaResponse.scala @@ -70,4 +70,6 @@ case class StopReplicaResponse(override val correlationId: Int, buffer.putShort(value) } } + + override def describe(details: Boolean):String = { toString } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index c5221c4e31a0e..a319f2f438bfd 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -72,13 +72,7 @@ case class TopicMetadataRequest(val versionId: Short, } override def toString(): String = { - val topicMetadataRequest = new StringBuilder - topicMetadataRequest.append("Name: " + this.getClass.getSimpleName) - topicMetadataRequest.append("; Version: " + versionId) - topicMetadataRequest.append("; CorrelationId: " + correlationId) - topicMetadataRequest.append("; ClientId: " + clientId) - topicMetadataRequest.append("; Topics: " + topics.mkString(",")) - topicMetadataRequest.toString() + describe(true) } override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { @@ -88,4 +82,15 @@ case class TopicMetadataRequest(val versionId: Short, val errorResponse = TopicMetadataResponse(topicMetadata, correlationId) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) } -} + + override def describe(details: Boolean): String = { + val topicMetadataRequest = new StringBuilder + topicMetadataRequest.append("Name: " + this.getClass.getSimpleName) + topicMetadataRequest.append("; Version: " + versionId) + topicMetadataRequest.append("; CorrelationId: " + correlationId) + topicMetadataRequest.append("; ClientId: " + clientId) + if(details) + topicMetadataRequest.append("; Topics: " + topics.mkString(",")) + topicMetadataRequest.toString() + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala index 290f263dfdceb..f6b7429faeab3 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala @@ -57,4 +57,6 @@ case class TopicMetadataResponse(topicsMetadata: Seq[TopicMetadata], val brokers = (parts.flatMap(_.replicas)) ++ (parts.map(_.leader).collect{case Some(l) => l}) brokers.map(b => (b.id, b)).toMap } + + override def describe(details: Boolean):String = { toString } } diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala index 2ead3644eb865..54dd7bd4e195c 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -101,6 +101,15 @@ case class UpdateMetadataRequest (versionId: Short, } override def toString(): String = { + describe(true) + } + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val errorResponse = new UpdateMetadataResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]])) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } + + override def describe(details: Boolean): String = { val updateMetadataRequest = new StringBuilder updateMetadataRequest.append("Name:" + this.getClass.getSimpleName) updateMetadataRequest.append(";Version:" + versionId) @@ -108,13 +117,9 @@ case class UpdateMetadataRequest (versionId: Short, updateMetadataRequest.append(";ControllerEpoch:" + controllerEpoch) updateMetadataRequest.append(";CorrelationId:" + correlationId) updateMetadataRequest.append(";ClientId:" + clientId) - updateMetadataRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) updateMetadataRequest.append(";AliveBrokers:" + aliveBrokers.mkString(",")) + if(details) + updateMetadataRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) updateMetadataRequest.toString() } - - override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - val errorResponse = new UpdateMetadataResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]])) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) - } -} +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala index b1e42c39c8801..c583c1f00c89a 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala @@ -41,4 +41,6 @@ case class UpdateMetadataResponse(override val correlationId: Int, buffer.putInt(correlationId) buffer.putShort(errorCode) } + + override def describe(details: Boolean):String = { toString } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 1d2f81be4f980..22819133ea7d4 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -89,7 +89,7 @@ object ClientUtils extends Logging{ } /** - * Parse a list of broker urls in the form host1:port1, host2:port2, ... + * Parse a list of broker urls in the form host1:port1:rack1, host2:port2:rack2, ... */ def parseBrokerList(brokerListStr: String): Seq[Broker] = { val brokersStr = Utils.parseCsvList(brokerListStr) @@ -100,7 +100,8 @@ object ClientUtils extends Logging{ val brokerInfos = brokerStr.split(":") val hostName = brokerInfos(0) val port = brokerInfos(1).toInt - new Broker(brokerId, hostName, port) + val rack = if (brokerInfos.size > 2) brokerInfos(2).toInt else 0 + new Broker(brokerId, hostName, port, rack) }) } diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 9407ed21fbbd5..fbc77fc9d8fc3 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -37,7 +37,11 @@ private[kafka] object Broker { val brokerInfo = m.asInstanceOf[Map[String, Any]] val host = brokerInfo.get("host").get.asInstanceOf[String] val port = brokerInfo.get("port").get.asInstanceOf[Int] - new Broker(id, host, port) + val rack = brokerInfo.get("rack") match { + case Some(r) => r.asInstanceOf[Int] + case None => 0 + } + new Broker(id, host, port, rack) case None => throw new BrokerNotAvailableException("Broker id %d does not exist".format(id)) } @@ -50,32 +54,34 @@ private[kafka] object Broker { val id = buffer.getInt val host = readShortString(buffer) val port = buffer.getInt - new Broker(id, host, port) + val rack = buffer.getInt + new Broker(id, host, port, rack) } } -private[kafka] case class Broker(val id: Int, val host: String, val port: Int) { +private[kafka] case class Broker(val id: Int, val host: String, val port: Int, val rack: Int) { - override def toString(): String = new String("id:" + id + ",host:" + host + ",port:" + port) + override def toString(): String = new String("id:" + id + ",host:" + host + ",port:" + port + ",rack:" + rack) - def getConnectionString(): String = host + ":" + port + def getConnectionString(): String = host + ":" + port + ":" + rack def writeTo(buffer: ByteBuffer) { buffer.putInt(id) writeShortString(buffer, host) buffer.putInt(port) + buffer.putInt(rack) } - def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ + def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ + 4 /* rack id*/ override def equals(obj: Any): Boolean = { obj match { case null => false - case n: Broker => id == n.id && host == n.host && port == n.port + case n: Broker => id == n.id && host == n.host && port == n.port && rack == n.rack case _ => false } } - override def hashCode(): Int = hashcode(id, host, port) + override def hashCode(): Int = hashcode(id, host, port, rack) } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 60f3ed4e88b08..1087a2e91c86e 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -17,16 +17,19 @@ package kafka.cluster import scala.collection._ +import kafka.admin.AdminUtils import kafka.utils._ import java.lang.Object -import kafka.api.LeaderAndIsr +import kafka.api.{PartitionStateInfo, LeaderAndIsr} +import kafka.log.LogConfig import kafka.server.ReplicaManager import com.yammer.metrics.core.Gauge import kafka.metrics.KafkaMetricsGroup -import kafka.common.{NotLeaderForPartitionException, ErrorMapping} -import kafka.controller.{LeaderIsrAndControllerEpoch, KafkaController} +import kafka.controller.KafkaController import org.apache.log4j.Logger import kafka.message.ByteBufferMessageSet +import kafka.common.{NotAssignedReplicaException, TopicAndPartition, NotLeaderForPartitionException, ErrorMapping} +import java.io.IOException /** @@ -39,7 +42,6 @@ class Partition(val topic: String, val replicaManager: ReplicaManager) extends Logging with KafkaMetricsGroup { private val localBrokerId = replicaManager.config.brokerId private val logManager = replicaManager.logManager - private val replicaFetcherManager = replicaManager.replicaFetcherManager private val zkClient = replicaManager.zkClient var leaderReplicaIdOpt: Option[Int] = None var inSyncReplicas: Set[Replica] = Set.empty[Replica] @@ -71,7 +73,7 @@ class Partition(val topic: String, leaderIsrUpdateLock synchronized { leaderReplicaIfLocal() match { case Some(_) => - inSyncReplicas.size < replicationFactor + inSyncReplicas.size < assignedReplicas.size case None => false } @@ -84,12 +86,16 @@ class Partition(val topic: String, case Some(replica) => replica case None => if (isReplicaLocal(replicaId)) { - val log = logManager.getOrCreateLog(topic, partitionId) - val offset = replicaManager.highWatermarkCheckpoints(log.dir.getParent).read(topic, partitionId).min(log.logEndOffset) + val config = LogConfig.fromProps(logManager.defaultConfig.toProps, AdminUtils.fetchTopicConfig(zkClient, topic)) + val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) + val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParent) + val offsetMap = checkpoint.read + if (!offsetMap.contains(TopicAndPartition(topic, partitionId))) + warn("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId)) + val offset = offsetMap.getOrElse(TopicAndPartition(topic, partitionId), 0L).min(log.logEndOffset) val localReplica = new Replica(replicaId, this, time, offset, Some(log)) addReplicaIfNotExists(localReplica) - } - else { + } else { val remoteReplica = new Replica(replicaId, this, time) addReplicaIfNotExists(remoteReplica) } @@ -126,32 +132,50 @@ class Partition(val topic: String, assignedReplicaMap.values.toSet } + def removeReplica(replicaId: Int) { + assignedReplicaMap.remove(replicaId) + } + + def delete() { + // need to hold the lock to prevent appendMessagesToLeader() from hitting I/O exceptions due to log being deleted + leaderIsrUpdateLock synchronized { + assignedReplicaMap.clear() + inSyncReplicas = Set.empty[Replica] + leaderReplicaIdOpt = None + try { + logManager.deleteLog(TopicAndPartition(topic, partitionId)) + } catch { + case e: IOException => + fatal("Error deleting the log for partition [%s,%d]".format(topic, partitionId), e) + Runtime.getRuntime().halt(1) + } + } + } + + def getLeaderEpoch(): Int = { + leaderIsrUpdateLock synchronized { + return this.leaderEpoch + } + } /** - * If the leaderEpoch of the incoming request is higher than locally cached epoch, make the local replica the leader in the following steps. - * 1. stop the existing replica fetcher - * 2. create replicas in ISR if needed (the ISR expand/shrink logic needs replicas in ISR to be available) - * 3. reset LogEndOffset for remote replicas (there could be old LogEndOffset from the time when this broker was the leader last time) - * 4. set the new leader and ISR + * Make the local replica the leader by resetting LogEndOffset for remote replicas (there could be old LogEndOffset from the time when this broker was the leader last time) + * and setting the new leader and ISR */ - def makeLeader(controllerId: Int, topic: String, partitionId: Int, - leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, correlationId: Int): Boolean = { + def makeLeader(controllerId: Int, + partitionStateInfo: PartitionStateInfo, correlationId: Int): Boolean = { leaderIsrUpdateLock synchronized { + val allReplicas = partitionStateInfo.allReplicas + val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - if (leaderEpoch >= leaderAndIsr.leaderEpoch){ - stateChangeLogger.trace(("Broker %d discarded the become-leader request with correlation id %d from " + - "controller %d epoch %d for partition [%s,%d] since current leader epoch %d is >= the request's leader epoch %d") - .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, topic, - partitionId, leaderEpoch, leaderAndIsr.leaderEpoch)) - return false - } // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch - // stop replica fetcher thread, if any - replicaFetcherManager.removeFetcher(topic, partitionId) - + // add replicas that are new + allReplicas.foreach(replica => getOrCreateReplica(replica)) val newInSyncReplicas = leaderAndIsr.isr.map(r => getOrCreateReplica(r)).toSet + // remove assigned replicas that have been removed by the controller + (assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_)) // reset LogEndOffset for remote replicas assignedReplicas.foreach(r => if (r.brokerId != localBrokerId) r.logEndOffset = ReplicaManager.UnknownLogEndOffset) inSyncReplicas = newInSyncReplicas @@ -165,51 +189,30 @@ class Partition(val topic: String, } /** - * If the leaderEpoch of the incoming request is higher than locally cached epoch, make the local replica the follower in the following steps. - * 1. stop any existing fetcher on this partition from the local replica - * 2. make sure local replica exists and truncate the log to high watermark - * 3. set the leader and set ISR to empty - * 4. start a fetcher to the new leader + * Make the local replica the follower by setting the new leader and ISR to empty */ - def makeFollower(controllerId: Int, topic: String, partitionId: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, + def makeFollower(controllerId: Int, + partitionStateInfo: PartitionStateInfo, leaders: Set[Broker], correlationId: Int): Boolean = { leaderIsrUpdateLock synchronized { + val allReplicas = partitionStateInfo.allReplicas + val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - if (leaderEpoch >= leaderAndIsr.leaderEpoch) { - stateChangeLogger.trace(("Broker %d discarded the become-follower request with correlation id %d from " + - "controller %d epoch %d for partition [%s,%d] since current leader epoch %d is >= the request's leader epoch %d") - .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, topic, - partitionId, leaderEpoch, leaderAndIsr.leaderEpoch)) - return false - } + val newLeaderBrokerId: Int = leaderAndIsr.leader // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch - // make sure local replica exists. This reads the last check pointed high watermark from disk. On startup, it is - // important to ensure that this operation happens for every single partition in a leader and isr request, else - // some high watermark values could be overwritten with 0. This leads to replicas fetching from the earliest offset - // on the leader - val localReplica = getOrCreateReplica() - val newLeaderBrokerId: Int = leaderAndIsr.leader // 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) - localReplica.log.get.truncateTo(localReplica.highWatermark) + // add replicas that are new + allReplicas.foreach(r => getOrCreateReplica(r)) + // remove assigned replicas that have been removed by the controller + (assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_)) inSyncReplicas = Set.empty[Replica] leaderEpoch = leaderAndIsr.leaderEpoch zkVersion = leaderAndIsr.zkVersion leaderReplicaIdOpt = Some(newLeaderBrokerId) - if (!replicaManager.isShuttingDown.get()) { - // start fetcher thread to current leader if we are not shutting down - replicaFetcherManager.addFetcher(topic, partitionId, localReplica.logEndOffset, leaderBroker) - } - else { - stateChangeLogger.trace(("Broker %d ignored the become-follower state change with correlation id %d from " + - "controller %d epoch %d since it is shutting down") - .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch)) - } case None => // 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") @@ -223,7 +226,13 @@ class Partition(val topic: String, def updateLeaderHWAndMaybeExpandIsr(replicaId: Int, offset: Long) { leaderIsrUpdateLock synchronized { debug("Recording follower %d position %d for partition [%s,%d].".format(replicaId, offset, topic, partitionId)) - val replica = getOrCreateReplica(replicaId) + val replicaOpt = getReplica(replicaId) + if(!replicaOpt.isDefined) { + throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d for partition [%s,%d] since the replica %d" + + " is not recognized to be one of the assigned replicas %s for partition [%s,%d]").format(localBrokerId, replicaId, + offset, topic, partitionId, replicaId, assignedReplicas().map(_.brokerId).mkString(","), topic, partitionId)) + } + val replica = replicaOpt.get replica.logEndOffset = offset // check if this replica needs to be added to the ISR @@ -231,7 +240,11 @@ class Partition(val topic: String, case Some(leaderReplica) => val replica = getReplica(replicaId).get val leaderHW = leaderReplica.highWatermark - if (!inSyncReplicas.contains(replica) && replica.logEndOffset >= leaderHW) { + // For a replica to get added back to ISR, it has to satisfy 3 conditions- + // 1. It is not already in the ISR + // 2. It is part of the assigned replica list. See KAFKA-1097 + // 3. It's log end offset >= leader's highwatermark + if (!inSyncReplicas.contains(replica) && assignedReplicas.map(_.brokerId).contains(replicaId) && replica.logEndOffset >= leaderHW) { // expand ISR val newInSyncReplicas = inSyncReplicas + replica info("Expanding ISR for partition [%s,%d] from %s to %s" @@ -314,19 +327,15 @@ class Partition(val topic: String, def getOutOfSyncReplicas(leaderReplica: Replica, keepInSyncTimeMs: Long, keepInSyncMessages: Long): Set[Replica] = { /** * there are two cases that need to be handled here - - * 1. Stuck followers: If the leo of the replica is less than the leo of leader and the leo hasn't been updated - * for keepInSyncTimeMs ms, the follower is stuck and should be removed from the ISR + * 1. Stuck followers: If the leo of the replica hasn't been updated for keepInSyncTimeMs ms, + * the follower is stuck and should be removed from the ISR * 2. Slow followers: If the leo of the slowest follower is behind the leo of the leader by keepInSyncMessages, the * follower is not catching up and should be removed from the ISR **/ val leaderLogEndOffset = leaderReplica.logEndOffset val candidateReplicas = inSyncReplicas - leaderReplica // Case 1 above - val possiblyStuckReplicas = candidateReplicas.filter(r => r.logEndOffset < leaderLogEndOffset) - if(possiblyStuckReplicas.size > 0) - debug("Possibly stuck replicas for partition [%s,%d] are %s".format(topic, partitionId, - possiblyStuckReplicas.map(_.brokerId).mkString(","))) - val stuckReplicas = possiblyStuckReplicas.filter(r => r.logEndOffsetUpdateTimeMs < (time.milliseconds - keepInSyncTimeMs)) + val stuckReplicas = candidateReplicas.filter(r => (time.milliseconds - r.logEndOffsetUpdateTimeMs) > keepInSyncTimeMs) if(stuckReplicas.size > 0) debug("Stuck replicas for partition [%s,%d] are %s".format(topic, partitionId, stuckReplicas.map(_.brokerId).mkString(","))) // Case 2 above @@ -336,16 +345,16 @@ class Partition(val topic: String, stuckReplicas ++ slowReplicas } - def appendMessagesToLeader(messages: ByteBufferMessageSet): (Long, Long) = { + def appendMessagesToLeader(messages: ByteBufferMessageSet) = { leaderIsrUpdateLock synchronized { val leaderReplicaOpt = leaderReplicaIfLocal() leaderReplicaOpt match { case Some(leaderReplica) => val log = leaderReplica.log.get - val (start, end) = log.append(messages, assignOffsets = true) + val info = log.append(messages, assignOffsets = true) // we may need to increment high watermark since ISR could be down to 1 maybeIncrementLeaderHW(leaderReplica) - (start, end) + info case None => throw new NotLeaderForPartitionException("Leader not local for partition [%s,%d] on broker %d" .format(topic, partitionId, localBrokerId)) diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index c8769e0697581..b0b5dcedc4dad 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -41,6 +41,8 @@ object ErrorMapping { val ReplicaNotAvailableCode: Short = 9 val MessageSizeTooLargeCode: Short = 10 val StaleControllerEpochCode: Short = 11 + val OffsetMetadataTooLargeCode: Short = 12 + val StaleLeaderEpochCode: Short = 13 private val exceptionToCode = Map[Class[Throwable], Short]( @@ -54,7 +56,8 @@ object ErrorMapping { classOf[BrokerNotAvailableException].asInstanceOf[Class[Throwable]] -> BrokerNotAvailableCode, classOf[ReplicaNotAvailableException].asInstanceOf[Class[Throwable]] -> ReplicaNotAvailableCode, classOf[MessageSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSizeTooLargeCode, - classOf[ControllerMovedException].asInstanceOf[Class[Throwable]] -> StaleControllerEpochCode + classOf[ControllerMovedException].asInstanceOf[Class[Throwable]] -> StaleControllerEpochCode, + classOf[OffsetMetadataTooLargeException].asInstanceOf[Class[Throwable]] -> OffsetMetadataTooLargeCode ).withDefaultValue(UnknownCode) /* invert the mapping */ diff --git a/core/src/main/scala/kafka/common/KafkaZookeperClient.scala b/core/src/main/scala/kafka/common/KafkaZookeperClient.scala deleted file mode 100644 index bace1d228f7bf..0000000000000 --- a/core/src/main/scala/kafka/common/KafkaZookeperClient.scala +++ /dev/null @@ -1,35 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ - -package kafka.common - -import org.I0Itec.zkclient.ZkClient -import kafka.utils.{ZKStringSerializer, ZKConfig} -import java.util.concurrent.atomic.AtomicReference - -object KafkaZookeeperClient { - private val INSTANCE = new AtomicReference[ZkClient](null) - - def getZookeeperClient(config: ZKConfig): ZkClient = { - // TODO: This cannot be a singleton since unit tests break if we do that -// INSTANCE.compareAndSet(null, new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, -// ZKStringSerializer)) - INSTANCE.set(new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer)) - INSTANCE.get() - } -} diff --git a/core/src/main/scala/kafka/utils/Topic.scala b/core/src/main/scala/kafka/common/LogCleaningAbortedException.scala similarity index 82% rename from core/src/main/scala/kafka/utils/Topic.scala rename to core/src/main/scala/kafka/common/LogCleaningAbortedException.scala index 487329ad228e9..5ea66322a92c7 100644 --- a/core/src/main/scala/kafka/utils/Topic.scala +++ b/core/src/main/scala/kafka/common/LogCleaningAbortedException.scala @@ -5,7 +5,7 @@ * 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 @@ -13,4 +13,12 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - */ \ No newline at end of file + */ + +package kafka.common + +/** + * Thrown when a log cleaning task is requested to be aborted. + */ +class LogCleaningAbortedException() extends RuntimeException() { +} diff --git a/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala b/core/src/main/scala/kafka/common/NotAssignedReplicaException.scala similarity index 77% rename from core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala rename to core/src/main/scala/kafka/common/NotAssignedReplicaException.scala index 487329ad228e9..409d11275280c 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala +++ b/core/src/main/scala/kafka/common/NotAssignedReplicaException.scala @@ -5,7 +5,7 @@ * 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 @@ -13,4 +13,11 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - */ \ No newline at end of file + */ + +package kafka.common + +class NotAssignedReplicaException(message: String, cause: Throwable) extends RuntimeException(message, cause) { + def this(message: String) = this(message, null) + def this() = this(null, null) +} diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala new file mode 100644 index 0000000000000..59608a34202b4 --- /dev/null +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -0,0 +1,36 @@ +package kafka.common + +/** + * 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. + */ + +/** + * Convenience case class since (topic, partition) pairs are ubiquitous. + */ +case class OffsetMetadataAndError(offset: Long, metadata: String = OffsetMetadataAndError.NoMetadata, error: Short = ErrorMapping.NoError) { + + def this(tuple: (Long, String, Short)) = this(tuple._1, tuple._2, tuple._3) + + def asTuple = (offset, metadata, error) + + override def toString = "OffsetAndMetadata[%d,%s,%d]".format(offset, metadata, error) + +} + +object OffsetMetadataAndError { + val InvalidOffset: Long = -1L; + val NoMetadata: String = ""; +} diff --git a/core/src/main/scala/kafka/common/OffsetMetadataTooLargeException.scala b/core/src/main/scala/kafka/common/OffsetMetadataTooLargeException.scala new file mode 100644 index 0000000000000..50edb273b3a79 --- /dev/null +++ b/core/src/main/scala/kafka/common/OffsetMetadataTooLargeException.scala @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +/** + * Indicates the client has specified offset metadata that exceeds the configured + * maximum size in bytes + */ +class OffsetMetadataTooLargeException(message: String) extends RuntimeException(message) { + def this() = this(null) +} + diff --git a/core/src/main/scala/kafka/common/BrokerNotExistException.scala b/core/src/main/scala/kafka/common/ThreadShutdownException.scala similarity index 82% rename from core/src/main/scala/kafka/common/BrokerNotExistException.scala rename to core/src/main/scala/kafka/common/ThreadShutdownException.scala index 487329ad228e9..6554a5ee4debb 100644 --- a/core/src/main/scala/kafka/common/BrokerNotExistException.scala +++ b/core/src/main/scala/kafka/common/ThreadShutdownException.scala @@ -5,7 +5,7 @@ * 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 @@ -13,4 +13,12 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - */ \ No newline at end of file + */ + +package kafka.common + +/** + * An exception that indicates a thread is being shut down normally. + */ +class ThreadShutdownException() extends RuntimeException { +} diff --git a/core/src/main/scala/kafka/common/TopicAndPartition.scala b/core/src/main/scala/kafka/common/TopicAndPartition.scala index 63596b7b2260d..df3db912f5dae 100644 --- a/core/src/main/scala/kafka/common/TopicAndPartition.scala +++ b/core/src/main/scala/kafka/common/TopicAndPartition.scala @@ -1,5 +1,7 @@ package kafka.common +import kafka.cluster.{Replica, Partition} + /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -24,6 +26,10 @@ case class TopicAndPartition(topic: String, partition: Int) { def this(tuple: (String, Int)) = this(tuple._1, tuple._2) + def this(partition: Partition) = this(partition.topic, partition.partitionId) + + def this(replica: Replica) = this(replica.topic, replica.partitionId) + def asTuple = (topic, partition) override def toString = "[%s,%d]".format(topic, partition) diff --git a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala index 48fa7a3dbce29..dc066c23e9c22 100644 --- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala @@ -289,10 +289,10 @@ class DefaultMessageFormatter extends MessageFormatter { def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) { if(printKey) { - output.write(key) + output.write(if (key == null) "null".getBytes() else key) output.write(keySeparator) } - output.write(value) + output.write(if (value == null) "null".getBytes() else value) output.write(lineSeparator) } } diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index 8c033089691d3..e4451bb2a2179 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -18,12 +18,14 @@ package kafka.consumer import org.I0Itec.zkclient.ZkClient -import kafka.server.{AbstractFetcherThread, AbstractFetcherManager} +import kafka.server.{BrokerAndInitialOffset, AbstractFetcherThread, AbstractFetcherManager} import kafka.cluster.{Cluster, Broker} import scala.collection.immutable +import scala.collection.Map import collection.mutable.HashMap import scala.collection.mutable import java.util.concurrent.locks.ReentrantLock +import kafka.utils.Utils.inLock import kafka.utils.ZkUtils._ import kafka.utils.{ShutdownableThread, SystemTime} import kafka.common.TopicAndPartition @@ -54,7 +56,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker] lock.lock() try { - if (noLeaderPartitionSet.isEmpty) { + while (noLeaderPartitionSet.isEmpty) { trace("No partition for leader election.") cond.await() } @@ -89,23 +91,22 @@ class ConsumerFetcherManager(private val consumerIdString: String, lock.unlock() } - leaderForPartitionsMap.foreach { - case(topicAndPartition, leaderBroker) => - val pti = partitionMap(topicAndPartition) - try { - addFetcher(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset(), leaderBroker) - } catch { - case t: Throwable => { - 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() - } - } + try { + addFetcherForPartitions(leaderForPartitionsMap.map{ + case (topicAndPartition, broker) => + topicAndPartition -> BrokerAndInitialOffset(broker, partitionMap(topicAndPartition).getFetchOffset())} + ) + } catch { + case t: Throwable => { + if (!isRunning.get()) + throw t /* If this thread is stopped, propagate this exception to kill the thread. */ + else { + warn("Failed to add leader for partitions %s; will retry".format(leaderForPartitionsMap.keySet.mkString(",")), t) + lock.lock() + noLeaderPartitionSet ++= leaderForPartitionsMap.keySet + lock.unlock() } + } } shutdownIdleFetcherThreads() @@ -123,14 +124,11 @@ class ConsumerFetcherManager(private val consumerIdString: String, leaderFinderThread = new LeaderFinderThread(consumerIdString + "-leader-finder-thread") leaderFinderThread.start() - lock.lock() - try { + inLock(lock) { partitionMap = topicInfos.map(tpi => (TopicAndPartition(tpi.topic, tpi.partitionId), tpi)).toMap this.cluster = cluster noLeaderPartitionSet ++= topicInfos.map(tpi => TopicAndPartition(tpi.topic, tpi.partitionId)) cond.signalAll() - } finally { - lock.unlock() } } @@ -158,14 +156,11 @@ class ConsumerFetcherManager(private val consumerIdString: String, def addPartitionsWithError(partitionList: Iterable[TopicAndPartition]) { debug("adding partitions with error %s".format(partitionList)) - lock.lock() - try { + inLock(lock) { if (partitionMap != null) { noLeaderPartitionSet ++= partitionList cond.signalAll() } - } finally { - lock.unlock() } } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index dda0a8f041f24..f8c1b4e674f75 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -66,7 +66,7 @@ class ConsumerFetcherThread(name: String, // any logic for partitions whose leader has changed def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) { - partitions.foreach(tap => removePartition(tap.topic, tap.partition)) + removePartitions(partitions.toSet) consumerFetcherManager.addPartitionsWithError(partitions) } } diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index b80c0b0a89ee0..ac491b4da2583 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -101,10 +101,7 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk item.message.ensureValid() // validate checksum of message to ensure it is valid - val keyBuffer = item.message.key - val key = if(keyBuffer == null) null.asInstanceOf[K] else keyDecoder.fromBytes(Utils.readBytes(keyBuffer)) - val value = valueDecoder.fromBytes(Utils.readBytes(item.message.payload)) - new MessageAndMetadata(key, value, currentTopicInfo.topic, currentTopicInfo.partitionId, item.offset) + new MessageAndMetadata(currentTopicInfo.topic, currentTopicInfo.partitionId, item.message, item.offset, keyDecoder, valueDecoder) } def clearCurrentChunk() { diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index fac64aaa7f783..24f7fb5455ef2 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -124,6 +124,20 @@ class SimpleConsumer(val host: String, */ def getOffsetsBefore(request: OffsetRequest) = OffsetResponse.readFrom(sendRequest(request).buffer) + /** + * Commit offsets for a topic + * @param request a [[kafka.api.OffsetCommitRequest]] object. + * @return a [[kafka.api.OffsetCommitResponse]] object. + */ + def commitOffsets(request: OffsetCommitRequest) = OffsetCommitResponse.readFrom(sendRequest(request).buffer) + + /** + * Fetch offsets for a topic + * @param request a [[kafka.api.OffsetFetchRequest]] object. + * @return a [[kafka.api.OffsetFetchResponse]] object. + */ + def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).buffer) + private def getOrMakeConnection() { if(!isClosed && !blockingChannel.isConnected) { connect() diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala index a3eb53e826211..e33263378489f 100644 --- a/core/src/main/scala/kafka/consumer/TopicCount.scala +++ b/core/src/main/scala/kafka/consumer/TopicCount.scala @@ -25,7 +25,7 @@ import kafka.common.KafkaException private[kafka] trait TopicCount { def getConsumerThreadIdsPerTopic: Map[String, Set[String]] - def dbString: String + def getTopicCountMap: Map[String, Int] def pattern: String protected def makeConsumerThreadIdsPerTopic(consumerIdString: String, @@ -111,24 +111,7 @@ private[kafka] class StaticTopicCount(val consumerIdString: String, } } - /** - * return json of - * { "topic1" : 4, - * "topic2" : 4 } - */ - def dbString = { - val builder = new StringBuilder - builder.append("{ ") - var i = 0 - for ( (topic, nConsumers) <- topicCountMap) { - if (i > 0) - builder.append(",") - builder.append("\"" + topic + "\": " + nConsumers) - i += 1 - } - builder.append(" }") - builder.toString() - } + def getTopicCountMap = topicCountMap def pattern = TopicCount.staticPattern } @@ -142,7 +125,7 @@ private[kafka] class WildcardTopicCount(zkClient: ZkClient, makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*)) } - def dbString = "{ \"%s\" : %d }".format(topicFilter.regex, numStreams) + def getTopicCountMap = Map(topicFilter.regex -> numStreams) def pattern: String = { topicFilter match { diff --git a/core/src/main/scala/kafka/consumer/TopicFilter.scala b/core/src/main/scala/kafka/consumer/TopicFilter.scala index cf3853b223095..4f2082360cfbb 100644 --- a/core/src/main/scala/kafka/consumer/TopicFilter.scala +++ b/core/src/main/scala/kafka/consumer/TopicFilter.scala @@ -41,14 +41,10 @@ sealed abstract class TopicFilter(rawRegex: String) extends Logging { override def toString = regex - def requiresTopicEventWatcher: Boolean - def isTopicAllowed(topic: String): Boolean } case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) { - override def requiresTopicEventWatcher = !regex.matches("""[\p{Alnum}-|]+""") - override def isTopicAllowed(topic: String) = { val allowed = topic.matches(regex) @@ -62,8 +58,6 @@ case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) { } case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) { - override def requiresTopicEventWatcher = true - override def isTopicAllowed(topic: String) = { val allowed = !topic.matches(regex) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 36b167b28ac08..703b2e22605ca 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -30,6 +30,7 @@ import org.apache.zookeeper.Watcher.Event.KeeperState import java.util.UUID import kafka.serializer._ import kafka.utils.ZkUtils._ +import kafka.utils.Utils.inLock import kafka.common._ import com.yammer.metrics.core.Gauge import kafka.metrics._ @@ -86,11 +87,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, 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(1) + private val scheduler = new KafkaScheduler(threads = 1, threadNamePrefix = "kafka-consumer-scheduler-") private val messageStreamCreated = new AtomicBoolean(false) private var sessionExpirationListener: ZKSessionExpireListener = null - private var topicPartitionChangeListenner: ZKTopicPartitionChangeListener = null + private var topicPartitionChangeListener: ZKTopicPartitionChangeListener = null private var loadBalancerListener: ZKRebalancerListener = null private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null @@ -115,8 +116,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, if (config.autoCommitEnable) { scheduler.startup info("starting auto committer every " + config.autoCommitIntervalMs + " ms") - scheduler.scheduleWithRate(autoCommit, "Kafka-consumer-autocommit-", config.autoCommitIntervalMs, - config.autoCommitIntervalMs, false) + scheduler.schedule("kafka-consumer-autocommit", + autoCommit, + delay = config.autoCommitIntervalMs, + period = config.autoCommitIntervalMs, + unit = TimeUnit.MILLISECONDS) } KafkaMetricsReporter.startReporters(config.props) @@ -162,7 +166,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, wildcardTopicWatcher.shutdown() try { if (config.autoCommitEnable) - scheduler.shutdownNow() + scheduler.shutdown() fetcher match { case Some(f) => f.stopConnections case None => @@ -180,7 +184,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } info("ZKConsumerConnector shut down completed") } - } + } } def consume[K, V](topicCountMap: scala.collection.Map[String,Int], keyDecoder: Decoder[K], valueDecoder: Decoder[V]) @@ -216,11 +220,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, 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, "timestamp" -> timestamp), valueInQuotes = true)) + val consumerRegistrationInfo = Json.encode(Map("version" -> 1, "subscription" -> topicCount.getTopicCountMap, "pattern" -> topicCount.pattern, + "timestamp" -> timestamp)) - createEphemeralPathExpectConflictHandleZKBug(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, consumerRegistrationInfo, null, (consumerZKString, consumer) => true, config.zkSessionTimeoutMs) + createEphemeralPathExpectConflictHandleZKBug(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, consumerRegistrationInfo, null, + (consumerZKString, consumer) => true, config.zkSessionTimeoutMs) info("end registering consumer " + consumerIdString + " in ZK") } @@ -298,7 +302,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, registerConsumerInZK(dirs, consumerIdString, topicCount) // explicitly trigger load balancing for this consumer loadBalancerListener.syncedRebalance() - // There is no need to resubscribe to child and state changes. // The child change watchers will be set inside rebalance when we read the children list. } @@ -311,9 +314,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def handleDataChange(dataPath : String, data: Object) { try { info("Topic info for path " + dataPath + " changed to " + data.toString + ", triggering rebalance") - // explicitly trigger load balancing for this consumer - loadBalancerListener.syncedRebalance() - + // queue up the rebalance event + loadBalancerListener.rebalanceEventTriggered() // There is no need to re-subscribe the watcher since it will be automatically // re-registered upon firing of this event by zkClient } catch { @@ -331,7 +333,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, class ZKRebalancerListener(val group: String, val consumerIdString: String, val kafkaMessageAndMetadataStreams: mutable.Map[String,List[KafkaStream[_,_]]]) extends IZkChildListener { - private val correlationId = new AtomicInteger(0) private var isWatcherTriggered = false private val lock = new ReentrantLock private val cond = lock.newCondition() @@ -363,12 +364,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, @throws(classOf[Exception]) def handleChildChange(parentPath : String, curChilds : java.util.List[String]) { - lock.lock() - try { + rebalanceEventTriggered() + } + + def rebalanceEventTriggered() { + inLock(lock) { isWatcherTriggered = true cond.signalAll() - } finally { - lock.unlock() } } @@ -654,8 +656,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, dirs, consumerIdString, topicCount, loadBalancerListener) // create listener for topic partition change event if not exist yet - if (topicPartitionChangeListenner == null) - topicPartitionChangeListenner = new ZKTopicPartitionChangeListener(loadBalancerListener) + if (topicPartitionChangeListener == null) + topicPartitionChangeListener = new ZKTopicPartitionChangeListener(loadBalancerListener) val topicStreamsMap = loadBalancerListener.kafkaMessageAndMetadataStreams @@ -713,7 +715,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, topicStreamsMap.foreach { topicAndStreams => // register on broker partition path changes val topicPath = BrokerTopicsPath + "/" + topicAndStreams._1 - zkClient.subscribeDataChanges(topicPath, topicPartitionChangeListenner) + zkClient.subscribeDataChanges(topicPath, topicPartitionChangeListener) } // explicitly trigger load balancing for this consumer @@ -753,19 +755,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount) reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams) - if (!topicFilter.requiresTopicEventWatcher) { - info("Not creating event watcher for trivial whitelist " + topicFilter) - } - else { - info("Creating topic event watcher for whitelist " + topicFilter) - wildcardTopicWatcher = new ZookeeperTopicEventWatcher(config, this) - - /* - * Topic events will trigger subsequent synced rebalances. Also, the - * consumer will get registered only after an allowed topic becomes - * available. - */ - } + /* + * Topic events will trigger subsequent synced rebalances. + */ + info("Creating topic event watcher for topics " + topicFilter) + wildcardTopicWatcher = new ZookeeperTopicEventWatcher(zkClient, this) def handleTopicEvent(allTopics: Seq[String]) { debug("Handling topic event") diff --git a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala index a67c193df9f7c..38f4ec0bd1b38 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala @@ -22,14 +22,11 @@ import kafka.utils.{ZkUtils, ZKStringSerializer, Logging} import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} import org.apache.zookeeper.Watcher.Event.KeeperState -class ZookeeperTopicEventWatcher(val config:ConsumerConfig, +class ZookeeperTopicEventWatcher(val zkClient: ZkClient, val eventHandler: TopicEventHandler[String]) extends Logging { val lock = new Object() - private var zkClient: ZkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, ZKStringSerializer) - startWatchingTopicEvents() private def startWatchingTopicEvents() { @@ -53,11 +50,10 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig, info("Shutting down topic event watcher.") if (zkClient != null) { stopWatchingTopicEvents() - zkClient.close() - zkClient = null } - else - warn("Cannot shutdown already shutdown topic event watcher.") + else { + warn("Cannot shutdown since the embedded zookeeper client has already closed.") + } } } @@ -70,7 +66,6 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig, if (zkClient != null) { val latestTopics = zkClient.getChildren(ZkUtils.BrokerTopicsPath).toList debug("all topics: %s".format(latestTopics)) - eventHandler.handleTopicEvent(latestTopics) } } @@ -93,10 +88,8 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig, def handleNewSession() { lock.synchronized { if (zkClient != null) { - info( - "ZK expired: resubscribing topic event listener to topic registry") - zkClient.subscribeChildChanges( - ZkUtils.BrokerTopicsPath, topicEventListener) + info("ZK expired: resubscribing topic event listener to topic registry") + zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicEventListener) } } } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index beca460dfe0f4..ea8485b479155 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -17,7 +17,7 @@ package kafka.controller import kafka.network.{Receive, BlockingChannel} -import kafka.utils.{Logging, ShutdownableThread} +import kafka.utils.{Utils, Logging, ShutdownableThread} import collection.mutable.HashMap import kafka.cluster.Broker import java.util.concurrent.{LinkedBlockingQueue, BlockingQueue} @@ -81,8 +81,7 @@ class ControllerChannelManager (private val controllerContext: ControllerContext BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) - channel.connect() - val requestThread = new RequestSendThread(config.brokerId, controllerContext, broker.id, messageQueue, channel) + val requestThread = new RequestSendThread(config.brokerId, controllerContext, broker, messageQueue, channel) requestThread.setDaemon(false) brokerStateInfo.put(broker.id, new ControllerBrokerStateInfo(channel, broker, messageQueue, requestThread)) } @@ -90,6 +89,7 @@ class ControllerChannelManager (private val controllerContext: ControllerContext private def removeExistingBroker(brokerId: Int) { try { brokerStateInfo(brokerId).channel.disconnect() + brokerStateInfo(brokerId).messageQueue.clear() brokerStateInfo(brokerId).requestSendThread.shutdown() brokerStateInfo.remove(brokerId) }catch { @@ -106,12 +106,13 @@ class ControllerChannelManager (private val controllerContext: ControllerContext class RequestSendThread(val controllerId: Int, val controllerContext: ControllerContext, - val toBrokerId: Int, + val toBroker: Broker, val queue: BlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)], val channel: BlockingChannel) - extends ShutdownableThread("Controller-%d-to-broker-%d-send-thread".format(controllerId, toBrokerId)) { + extends ShutdownableThread("Controller-%d-to-broker-%d-send-thread".format(controllerId, toBroker.id)) { private val lock = new Object() private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) + connectToBroker(toBroker, channel) override def doWork(): Unit = { val queueItem = queue.take() @@ -122,8 +123,25 @@ class RequestSendThread(val controllerId: Int, try{ lock synchronized { - channel.connect() // establish a socket connection if needed - channel.send(request) + var isSendSuccessful = false + while(isRunning.get() && !isSendSuccessful) { + // if a broker goes down for a long time, then at some point the controller's zookeeper listener will trigger a + // removeBroker which will invoke shutdown() on this thread. At that point, we will stop retrying. + try { + channel.send(request) + isSendSuccessful = true + } catch { + case e => // if the send was not successful, reconnect to broker and resend the message + error(("Controller %d epoch %d failed to send %s request with correlation id %s to broker %s. " + + "Reconnecting to broker.").format(controllerId, controllerContext.epoch, + RequestKeys.nameForKey(request.requestId.get), request.correlationId, toBroker.toString()), e) + channel.disconnect() + connectToBroker(toBroker, channel) + isSendSuccessful = false + // backoff before retrying the connection and send + Utils.swallow(Thread.sleep(300)) + } + } receive = channel.receive() var response: RequestOrResponse = null request.requestId.get match { @@ -134,8 +152,8 @@ class RequestSendThread(val controllerId: Int, case RequestKeys.UpdateMetadataKey => response = UpdateMetadataResponse.readFrom(receive.buffer) } - stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %d" - .format(controllerId, controllerContext.epoch, response.correlationId, toBrokerId)) + stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %s" + .format(controllerId, controllerContext.epoch, response.correlationId, toBroker.toString())) if(callback != null){ callback(response) @@ -143,11 +161,23 @@ class RequestSendThread(val controllerId: Int, } } catch { case e: Throwable => - warn("Controller %d fails to send a request to broker %d".format(controllerId, toBrokerId), e) + warn("Controller %d fails to send a request to broker %s".format(controllerId, toBroker.toString()), e) // If there is any socket error (eg, socket timeout), the channel is no longer usable and needs to be recreated. channel.disconnect() } } + + private def connectToBroker(broker: Broker, channel: BlockingChannel) { + try { + channel.connect() + info("Controller %d connected to %s for sending state change requests".format(controllerId, broker.toString())) + } catch { + case e => { + channel.disconnect() + error("Controller %d's connection to broker %s was unsuccessful".format(controllerId, broker.toString()), e) + } + } + } } class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendRequest: (Int, RequestOrResponse, (RequestOrResponse) => Unit) => Unit, @@ -173,10 +203,6 @@ class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendReq if(stopAndDeleteReplicaRequestMap.size > 0) throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + "new one. Some StopReplica with delete state changes %s might be lost ".format(stopAndDeleteReplicaRequestMap.toString())) - leaderAndIsrRequestMap.clear() - stopReplicaRequestMap.clear() - updateMetadataRequestMap.clear() - stopAndDeleteReplicaRequestMap.clear() } def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, @@ -238,8 +264,9 @@ class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendReq 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 " + - "for partition [%s,%d]").format(controllerId, controllerEpoch, typeOfRequest, correlationId, broker, + stateChangeLogger.trace(("Controller %d epoch %d sending %s LeaderAndIsr request %s with correlationId %d to broker %d " + + "for partition [%s,%d]").format(controllerId, controllerEpoch, typeOfRequest, + p._2.leaderIsrAndControllerEpoch, correlationId, broker, p._1._1, p._1._2)) } sendRequest(broker, leaderAndIsrRequest, null) @@ -250,8 +277,9 @@ class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendReq val partitionStateInfos = m._2.toMap val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, correlationId, clientId, partitionStateInfos, controllerContext.liveOrShuttingDownBrokers) - partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request with " + - "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, correlationId, broker, p._1))) + partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + + "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, + correlationId, broker, p._1))) sendRequest(broker, updateMetadataRequest, null) } updateMetadataRequestMap.clear() diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 88d130f55997b..a2efb98e93ae7 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -21,20 +21,21 @@ import collection.immutable.Set import com.yammer.metrics.core.Gauge import java.lang.{IllegalStateException, Object} import java.util.concurrent.TimeUnit -import kafka.admin.PreferredReplicaLeaderElectionCommand +import kafka.admin.{AdminOperationException, PreferredReplicaLeaderElectionCommand} import kafka.api._ import kafka.cluster.Broker import kafka.common._ import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} import kafka.server.{ZookeeperLeaderElector, KafkaConfig} import kafka.utils.ZkUtils._ -import kafka.utils.{Json, Utils, ZkUtils, Logging} +import kafka.utils.{Json, Utils, ZkUtils, Logging, KafkaScheduler} import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} import java.util.concurrent.atomic.AtomicInteger import scala.Some import kafka.common.TopicAndPartition +import org.apache.log4j.Logger class ControllerContext(val zkClient: ZkClient, val zkSessionTimeout: Int, @@ -51,7 +52,8 @@ class ControllerContext(val zkClient: ZkClient, var partitionsBeingReassigned: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap, var partitionsUndergoingPreferredReplicaElection: mutable.Set[TopicAndPartition] = - new mutable.HashSet) { + new mutable.HashSet, + var maxRackReplicaAssignment: mutable.Map[String, Int] = mutable.Map.empty) { private var liveBrokersUnderlying: Set[Broker] = Set.empty private var liveBrokerIdsUnderlying: Set[Int] = Set.empty @@ -68,6 +70,26 @@ class ControllerContext(val zkClient: ZkClient, def liveOrShuttingDownBrokerIds = liveBrokerIdsUnderlying def liveOrShuttingDownBrokers = liveBrokersUnderlying + + def partitionsOnBroker(brokerId: Int): Set[TopicAndPartition] = { + partitionReplicaAssignment + .filter { case(topicAndPartition, replicas) => replicas.contains(brokerId) } + .map { case(topicAndPartition, replicas) => topicAndPartition } + .toSet + } + + def replicasOnBrokers(brokerIds: Set[Int]): Set[PartitionAndReplica] = { + brokerIds.map { brokerId => + partitionReplicaAssignment + .filter { case(topicAndPartition, replicas) => replicas.contains(brokerId) } + .map { case(topicAndPartition, replicas) => + new PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, brokerId) } + }.flatten.toSet + } + + def allLiveReplicas(): Set[PartitionAndReplica] = { + replicasOnBrokers(liveBrokerIds) + } } trait KafkaControllerMBean { @@ -105,11 +127,15 @@ object KafkaController extends Logging { class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logging with KafkaMetricsGroup with KafkaControllerMBean { this.logIdent = "[Controller " + config.brokerId + "]: " private var isRunning = true + private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) 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, - config.brokerId) + onControllerResignation, config.brokerId) + // have a separate scheduler for the controller to be able to start and stop independently of the + // kafka server + private val autoRebalanceScheduler = new KafkaScheduler(1) val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext) private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) @@ -132,7 +158,23 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg if (!isActive()) 0 else - controllerContext.partitionLeadershipInfo.count(p => !controllerContext.liveBrokerIds.contains(p._2.leaderAndIsr.leader)) + controllerContext.partitionLeadershipInfo.count(p => !controllerContext.liveOrShuttingDownBrokerIds.contains(p._2.leaderAndIsr.leader)) + } + } + } + ) + + newGauge( + "PreferredReplicaImbalanceCount", + new Gauge[Int] { + def value(): Int = { + controllerContext.controllerLock synchronized { + if (!isActive()) + 0 + else + controllerContext.partitionReplicaAssignment.count { + case (topicPartition, replicas) => controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader != replicas.head + } } } } @@ -169,13 +211,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg debug("Live brokers: " + controllerContext.liveBrokerIds.mkString(",")) } - val allPartitionsAndReplicationFactorOnBroker = controllerContext.controllerLock synchronized { - getPartitionsAssignedToBroker(zkClient, controllerContext.allTopics.toSeq, id).map { - case(topic, partition) => - val topicAndPartition = TopicAndPartition(topic, partition) - (topicAndPartition, controllerContext.partitionReplicaAssignment(topicAndPartition).size) + val allPartitionsAndReplicationFactorOnBroker: Set[(TopicAndPartition, Int)] = + controllerContext.controllerLock synchronized { + controllerContext.partitionsOnBroker(id) + .map(topicAndPartition => (topicAndPartition, controllerContext.partitionReplicaAssignment(topicAndPartition).size)) } - } allPartitionsAndReplicationFactorOnBroker.foreach { case(topicAndPartition, replicationFactor) => @@ -248,11 +288,33 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg initializeAndMaybeTriggerPreferredReplicaElection() /* send partition leadership info to all live brokers */ sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) + if (config.autoLeaderRebalanceEnable) { + info("starting the partition rebalance scheduler") + autoRebalanceScheduler.startup() + autoRebalanceScheduler.schedule("partition-rebalance-thread", checkAndTriggerPartitionRebalance, + 5, config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS) + } } else info("Controller has been shut down, aborting startup/failover") } + /** + * This callback is invoked by the zookeeper leader elector when the current broker resigns as the controller. This is + * required to clean up internal controller data structures + */ + def onControllerResignation() { + controllerContext.controllerLock synchronized { + Utils.unregisterMBean(KafkaController.MBeanName) + partitionStateMachine.shutdown() + replicaStateMachine.shutdown() + if(controllerContext.controllerChannelManager != null) { + controllerContext.controllerChannelManager.shutdown() + controllerContext.controllerChannelManager = null + } + } + } + /** * Returns true if this broker is the current controller. */ @@ -285,7 +347,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg sendUpdateMetadataRequest(newBrokers) // the very first thing to do when a new broker comes up is send it the entire list of partitions that it is // supposed to host. Based on that the broker starts the high watermark threads for the input list of partitions - replicaStateMachine.handleStateChanges(getAllReplicasOnBroker(zkClient, controllerContext.allTopics.toSeq, newBrokers), OnlineReplica) + replicaStateMachine.handleStateChanges(controllerContext.replicasOnBrokers(newBrokersSet), OnlineReplica) // when a new broker comes up, the controller needs to trigger leader election for all new and offline partitions // to see if these brokers can become leaders for some/all of those partitionStateMachine.triggerOnlinePartitionStateChange() @@ -323,12 +385,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // trigger OnlinePartition state changes for offline or new partitions partitionStateMachine.triggerOnlinePartitionStateChange() // handle dead replicas - replicaStateMachine.handleStateChanges(getAllReplicasOnBroker(zkClient, controllerContext.allTopics.toSeq, deadBrokers), OfflineReplica) + replicaStateMachine.handleStateChanges(controllerContext.replicasOnBrokers(deadBrokersSet), OfflineReplica) } /** - * This callback is invoked by the partition state machine's topic change listener with the list of failed brokers - * as input. It does the following - + * This callback is invoked by the partition state machine's topic change listener with the list of new topics + * and partitions as input. It does the following - * 1. Registers partition change listener. This is not required until KAFKA-347 * 2. Invokes the new partition callback */ @@ -340,7 +402,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } /** - * This callback is invoked by the topic change callback with the list of failed brokers as input. + * This callback is invoked by the partition state machine's partition change listener with the list of new partitions. * It does the following - * 1. Move the newly created partitions to the NewPartition state * 2. Move the newly created partitions from NewPartition->OnlinePartition state @@ -356,44 +418,86 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg /** * This callback is invoked by the reassigned partitions listener. When an admin command initiates a partition * reassignment, it creates the /admin/reassign_partitions path that triggers the zookeeper listener. - * Reassigning replicas for a partition goes through a few stages - + * Reassigning replicas for a partition goes through a few steps listed in the code. * RAR = Reassigned replicas - * AR = Original list of replicas for partition - * 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 + * OAR = Original list of replicas for partition + * AR = current assigned replicas + * + * 1. Update AR in ZK with OAR + RAR. + * 2. Send LeaderAndIsr request to every replica in OAR + RAR (with AR as OAR + RAR). We do this by forcing an update + * of the leader epoch in zookeeper. + * 3. Start new replicas RAR - OAR by moving replicas in RAR - OAR to NewReplica state. + * 4. Wait until all replicas in RAR are in sync with the leader. + * 5 Move all replicas in RAR to OnlineReplica state. + * 6. Set AR to RAR in memory. + * 7. If the leader is not in RAR, elect a new leader from RAR. If new leader needs to be elected from RAR, a LeaderAndIsr + * will be sent. If not, then leader epoch will be incremented in zookeeper and a LeaderAndIsr request will be sent. + * In any case, the LeaderAndIsr request will have AR = RAR. This will prevent the leader from adding any replica in + * RAR - OAR back in the isr. + * 8. Move all replicas in OAR - RAR to OfflineReplica state. As part of OfflineReplica state change, we shrink the + * isr to remove OAR - RAR in zookeeper and sent a LeaderAndIsr ONLY to the Leader to notify it of the shrunk isr. + * After that, we send a StopReplica (delete = false) to the replicas in OAR - RAR. + * 9. Move all replicas in OAR - RAR to NonExistentReplica state. This will send a StopReplica (delete = false) to + * the replicas in OAR - RAR to physically delete the replicas on disk. + * 10. Update AR in ZK with RAR. + * 11. Update the /admin/reassign_partitions path in ZK to remove this partition. + * 12. After electing leader, the replicas and isr information changes. So resend the update metadata request to every broker. + * + * For example, if OAR = {1, 2, 3} and RAR = {4,5,6}, the values in the assigned replica (AR) and leader/isr path in ZK + * may go through the following transition. + * AR leader/isr + * {1,2,3} 1/{1,2,3} (initial state) + * {1,2,3,4,5,6} 1/{1,2,3} (step 2) + * {1,2,3,4,5,6} 1/{1,2,3,4,5,6} (step 4) + * {1,2,3,4,5,6} 4/{1,2,3,4,5,6} (step 7) + * {1,2,3,4,5,6} 4/{4,5,6} (step 8) + * {4,5,6} 4/{4,5,6} (step 10) + * + * Note that we have to update AR in ZK with RAR last since it's the only place where we store OAR persistently. + * This way, if the controller crashes before that step, we can still recover. */ def onPartitionReassignment(topicAndPartition: TopicAndPartition, reassignedPartitionContext: ReassignedPartitionsContext) { val reassignedReplicas = reassignedPartitionContext.newReplicas areReplicasInIsr(topicAndPartition.topic, topicAndPartition.partition, reassignedReplicas) match { + case false => + info("New replicas %s for partition %s being ".format(reassignedReplicas.mkString(","), topicAndPartition) + + "reassigned not yet caught up with the leader") + val newReplicasNotInOldReplicaList = reassignedReplicas.toSet -- controllerContext.partitionReplicaAssignment(topicAndPartition).toSet + val newAndOldReplicas = (reassignedPartitionContext.newReplicas ++ controllerContext.partitionReplicaAssignment(topicAndPartition)).toSet + val existingMaxReplicaAssignment = controllerContext.maxRackReplicaAssignment(topicAndPartition.topic) + //1. Update AR in ZK with OAR + RAR. + updateAssignedReplicasForPartition(topicAndPartition, newAndOldReplicas.toSeq, existingMaxReplicaAssignment) + //2. Send LeaderAndIsr request to every replica in OAR + RAR (with AR as OAR + RAR). + updateLeaderEpochAndSendRequest(topicAndPartition, controllerContext.partitionReplicaAssignment(topicAndPartition), + newAndOldReplicas.toSeq) + //3. replicas in RAR - OAR -> NewReplica + startNewReplicasForReassignedPartition(topicAndPartition, reassignedPartitionContext, newReplicasNotInOldReplicaList) + info("Waiting for new replicas %s for partition %s being ".format(reassignedReplicas.mkString(","), topicAndPartition) + + "reassigned to catch up with the leader") case true => - // mark the new replicas as online + //4. Wait until all replicas in RAR are in sync with the leader. + val oldReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition).toSet -- reassignedReplicas.toSet + //5. replicas in RAR -> OnlineReplica reassignedReplicas.foreach { replica => replicaStateMachine.handleStateChanges(Set(new PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, replica)), OnlineReplica) } - // check if current leader is in the new replicas list. If not, controller needs to trigger leader election + //6. Set AR to RAR in memory. + //7. Send LeaderAndIsr request with a potential new leader (if current leader not in RAR) and + // a new AR (using RAR) and same isr to every broker in RAR moveReassignedPartitionLeaderIfRequired(topicAndPartition, reassignedPartitionContext) - // stop older replicas - stopOldReplicasOfReassignedPartition(topicAndPartition, reassignedPartitionContext) - // write the new list of replicas for this partition in zookeeper - updateAssignedReplicasForPartition(topicAndPartition, reassignedPartitionContext) - // update the /admin/reassign_partitions path to remove this partition + //8. replicas in OAR - RAR -> Offline (force those replicas out of isr) + //9. replicas in OAR - RAR -> NonExistentReplica (force those replicas to be deleted) + stopOldReplicasOfReassignedPartition(topicAndPartition, reassignedPartitionContext, oldReplicas) + //10. Update AR in ZK with RAR. + val existingMaxReplicaAssignment = controllerContext.maxRackReplicaAssignment(topicAndPartition.topic) + updateAssignedReplicasForPartition(topicAndPartition, reassignedReplicas, existingMaxReplicaAssignment) + //11. Update the /admin/reassign_partitions path in ZK to remove this partition. removePartitionFromReassignedPartitions(topicAndPartition) info("Removed partition %s from the list of reassigned partitions in zookeeper".format(topicAndPartition)) controllerContext.partitionsBeingReassigned.remove(topicAndPartition) - // after electing leader, the replicas and isr information changes, so resend the update metadata request + //12. After electing leader, the replicas and isr information changes. So resend the update metadata request to every broker. sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicAndPartition)) - case false => - info("New replicas %s for partition %s being ".format(reassignedReplicas.mkString(","), topicAndPartition) + - "reassigned not yet caught up with the leader") - // start new replicas - startNewReplicasForReassignedPartition(topicAndPartition, reassignedPartitionContext) - info("Waiting for new replicas %s for partition %s being ".format(reassignedReplicas.mkString(","), topicAndPartition) + - "reassigned to catch up with the leader") } } @@ -482,6 +586,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg isRunning = false partitionStateMachine.shutdown() replicaStateMachine.shutdown() + if (config.autoLeaderRebalanceEnable) + autoRebalanceScheduler.shutdown() if(controllerContext.controllerChannelManager != null) { controllerContext.controllerChannelManager.shutdown() controllerContext.controllerChannelManager = null @@ -534,6 +640,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg controllerContext.allTopics = ZkUtils.getAllTopics(zkClient).toSet controllerContext.partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, controllerContext.allTopics.toSeq) controllerContext.partitionLeadershipInfo = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch] + controllerContext.maxRackReplicaAssignment = ZkUtils.getMaxRackReplicationForTopics(zkClient, controllerContext.allTopics.toSeq) controllerContext.shuttingDownBrokerIds = mutable.Set.empty[Int] // update the leader and isr cache for all existing partitions from Zookeeper updateLeaderAndIsrCache() @@ -602,6 +709,10 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg reassignedPartitionContext: ReassignedPartitionsContext) { val reassignedReplicas = reassignedPartitionContext.newReplicas val currentLeader = controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader + // change the assigned replica list to just the reassigned replicas in the cache so it gets sent out on the LeaderAndIsr + // request to the current or new leader. This will prevent it from adding the old replicas to the ISR + val oldAndNewReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) + controllerContext.partitionReplicaAssignment.put(topicAndPartition, reassignedReplicas) if(!reassignedPartitionContext.newReplicas.contains(currentLeader)) { info("Leader %s for partition %s being reassigned, ".format(currentLeader, topicAndPartition) + "is not in the new list of replicas %s. Re-electing leader".format(reassignedReplicas.mkString(","))) @@ -613,6 +724,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg case true => info("Leader %s for partition %s being reassigned, ".format(currentLeader, topicAndPartition) + "is already in the new list of replicas %s and is alive".format(reassignedReplicas.mkString(","))) + // shrink replication factor and update the leader epoch in zookeeper to use on the next LeaderAndIsrRequest + updateLeaderEpochAndSendRequest(topicAndPartition, oldAndNewReplicas, reassignedReplicas) case false => info("Leader %s for partition %s being reassigned, ".format(currentLeader, topicAndPartition) + "is already in the new list of replicas %s but is dead".format(reassignedReplicas.mkString(","))) @@ -622,12 +735,10 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } private def stopOldReplicasOfReassignedPartition(topicAndPartition: TopicAndPartition, - reassignedPartitionContext: ReassignedPartitionsContext) { - val reassignedReplicas = reassignedPartitionContext.newReplicas + reassignedPartitionContext: ReassignedPartitionsContext, + oldReplicas: Set[Int]) { val topic = topicAndPartition.topic val partition = topicAndPartition.partition - // send stop replica state change request to the old replicas - val oldReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition).toSet -- reassignedReplicas.toSet // first move the replica to offline state (the controller removes it from the ISR) oldReplicas.foreach { replica => replicaStateMachine.handleStateChanges(Set(new PartitionAndReplica(topic, partition, replica)), OfflineReplica) @@ -639,31 +750,43 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } private def updateAssignedReplicasForPartition(topicAndPartition: TopicAndPartition, - reassignedPartitionContext: ReassignedPartitionsContext) { - val reassignedReplicas = reassignedPartitionContext.newReplicas + replicas: Seq[Int], + maxRackReplication: Int) { val partitionsAndReplicasForThisTopic = controllerContext.partitionReplicaAssignment.filter(_._1.topic.equals(topicAndPartition.topic)) - partitionsAndReplicasForThisTopic.put(topicAndPartition, reassignedReplicas) - updateAssignedReplicasForPartition(topicAndPartition, partitionsAndReplicasForThisTopic) - info("Updated assigned replicas for partition %s being reassigned to %s ".format(topicAndPartition, reassignedReplicas.mkString(","))) + partitionsAndReplicasForThisTopic.put(topicAndPartition, replicas) + updateAssignedReplicasForPartition(topicAndPartition, partitionsAndReplicasForThisTopic, maxRackReplication) + info("Updated assigned replicas for partition %s being reassigned to %s ".format(topicAndPartition, replicas.mkString(","))) // update the assigned replica list after a successful zookeeper write - controllerContext.partitionReplicaAssignment.put(topicAndPartition, reassignedReplicas) - // stop watching the ISR changes for this partition - zkClient.unsubscribeDataChanges(ZkUtils.getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition), - controllerContext.partitionsBeingReassigned(topicAndPartition).isrChangeListener) + controllerContext.partitionReplicaAssignment.put(topicAndPartition, replicas) } private def startNewReplicasForReassignedPartition(topicAndPartition: TopicAndPartition, - reassignedPartitionContext: ReassignedPartitionsContext) { + reassignedPartitionContext: ReassignedPartitionsContext, + newReplicas: Set[Int]) { // send the start replica request to the brokers in the reassigned replicas list that are not in the assigned // replicas list - val assignedReplicaSet = Set.empty[Int] ++ controllerContext.partitionReplicaAssignment(topicAndPartition) - val reassignedReplicaSet = Set.empty[Int] ++ reassignedPartitionContext.newReplicas - val newReplicas: Seq[Int] = (reassignedReplicaSet -- assignedReplicaSet).toSeq newReplicas.foreach { replica => replicaStateMachine.handleStateChanges(Set(new PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, replica)), NewReplica) } } + private def updateLeaderEpochAndSendRequest(topicAndPartition: TopicAndPartition, replicasToReceiveRequest: Seq[Int], newAssignedReplicas: Seq[Int]) { + brokerRequestBatch.newBatch() + updateLeaderEpoch(topicAndPartition.topic, topicAndPartition.partition) match { + case Some(updatedLeaderIsrAndControllerEpoch) => + brokerRequestBatch.addLeaderAndIsrRequestForBrokers(replicasToReceiveRequest, topicAndPartition.topic, + topicAndPartition.partition, updatedLeaderIsrAndControllerEpoch, newAssignedReplicas) + brokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch, controllerContext.correlationId.getAndIncrement) + stateChangeLogger.trace(("Controller %d epoch %d sent LeaderAndIsr request %s with new assigned replica list %s " + + "to leader %d for partition being reassigned %s").format(config.brokerId, controllerContext.epoch, updatedLeaderIsrAndControllerEpoch, + newAssignedReplicas.mkString(","), updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader, topicAndPartition)) + case None => // fail the reassignment + stateChangeLogger.error(("Controller %d epoch %d failed to send LeaderAndIsr request with new assigned replica list %s " + + "to leader for partition being reassigned %s").format(config.brokerId, controllerContext.epoch, + newAssignedReplicas.mkString(","), topicAndPartition)) + } + } + private def registerReassignedPartitionsListener() = { zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, new PartitionsReassignedListener(this)) } @@ -677,21 +800,27 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } def removePartitionFromReassignedPartitions(topicAndPartition: TopicAndPartition) { + if(controllerContext.partitionsBeingReassigned.get(topicAndPartition).isDefined) { + // stop watching the ISR changes for this partition + zkClient.unsubscribeDataChanges(ZkUtils.getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition), + controllerContext.partitionsBeingReassigned(topicAndPartition).isrChangeListener) + } // read the current list of reassigned partitions from zookeeper val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient) // remove this partition from that list val updatedPartitionsBeingReassigned = partitionsBeingReassigned - topicAndPartition // write the new list to zookeeper ZkUtils.updatePartitionReassignmentData(zkClient, updatedPartitionsBeingReassigned.mapValues(_.newReplicas)) - // update the cache + // update the cache. NO-OP if the partition's reassignment was never started controllerContext.partitionsBeingReassigned.remove(topicAndPartition) } def updateAssignedReplicasForPartition(topicAndPartition: TopicAndPartition, - newReplicaAssignmentForTopic: Map[TopicAndPartition, Seq[Int]]) { + newReplicaAssignmentForTopic: Map[TopicAndPartition, Seq[Int]], + maxRackReplication: Int) { try { val zkPath = ZkUtils.getTopicPath(topicAndPartition.topic) - val jsonPartitionMap = ZkUtils.replicaAssignmentZkdata(newReplicaAssignmentForTopic.map(e => (e._1.partition.toString -> e._2))) + val jsonPartitionMap = ZkUtils.replicaAssignmentZkData(newReplicaAssignmentForTopic.map(e => (e._1.partition.toString -> e._2)), maxRackReplication) ZkUtils.updatePersistentPath(zkClient, zkPath, jsonPartitionMap) debug("Updated path %s with %s for replica assignment".format(zkPath, jsonPartitionMap)) } catch { @@ -779,7 +908,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("New leader and ISR for partition %s is %s".format(topicAndPartition, newLeaderAndIsr.toString())) updateSucceeded } else { - warn("Cannot remove replica %d from ISR of %s. Leader = %d ; ISR = %s" + warn("Cannot remove replica %d from ISR of partition %s since it is not in the ISR. Leader = %d ; ISR = %s" .format(replicaId, topicAndPartition, leaderAndIsr.leader, leaderAndIsr.isr)) finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(leaderAndIsr, epoch)) controllerContext.partitionLeadershipInfo.put(topicAndPartition, finalLeaderIsrAndControllerEpoch.get) @@ -793,6 +922,52 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg finalLeaderIsrAndControllerEpoch } + /** + * Does not change leader or isr, but just increments the leader epoch + * @param topic topic + * @param partition partition + * @return the new leaderAndIsr with an incremented leader epoch, or None if leaderAndIsr is empty. + */ + private def updateLeaderEpoch(topic: String, partition: Int): Option[LeaderIsrAndControllerEpoch] = { + val topicAndPartition = TopicAndPartition(topic, partition) + debug("Updating leader epoch for partition %s.".format(topicAndPartition)) + var finalLeaderIsrAndControllerEpoch: Option[LeaderIsrAndControllerEpoch] = None + var zkWriteCompleteOrUnnecessary = false + while (!zkWriteCompleteOrUnnecessary) { + // refresh leader and isr from zookeeper again + val leaderIsrAndEpochOpt = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) + zkWriteCompleteOrUnnecessary = leaderIsrAndEpochOpt match { + case Some(leaderIsrAndEpoch) => + val leaderAndIsr = leaderIsrAndEpoch.leaderAndIsr + val controllerEpoch = leaderIsrAndEpoch.controllerEpoch + if(controllerEpoch > epoch) + throw new StateChangeFailedException("Leader and isr path written by another controller. This probably" + + "means the current controller with epoch %d went through a soft failure and another ".format(epoch) + + "controller was elected with epoch %d. Aborting state change by this controller".format(controllerEpoch)) + // increment the leader epoch even if there are no leader or isr changes to allow the leader to cache the expanded + // assigned replica list + val newLeaderAndIsr = new LeaderAndIsr(leaderAndIsr.leader, leaderAndIsr.leaderEpoch + 1, + leaderAndIsr.isr, leaderAndIsr.zkVersion + 1) + // update the new leadership decision in zookeeper or retry + val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath( + zkClient, + ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition), + ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, epoch), + leaderAndIsr.zkVersion) + newLeaderAndIsr.zkVersion = newVersion + finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newLeaderAndIsr, epoch)) + if (updateSucceeded) + info("Updated leader epoch for partition %s to %d".format(topicAndPartition, newLeaderAndIsr.leaderEpoch)) + updateSucceeded + case None => + throw new IllegalStateException(("Cannot update leader epoch for partition %s as leaderAndIsr path is empty. " + + "This could mean we somehow tried to reassign a partition that doesn't exist").format(topicAndPartition)) + true + } + } + finalLeaderIsrAndControllerEpoch + } + class SessionExpirationListener() extends IZkStateListener with Logging { this.logIdent = "[SessionExpirationListener on " + config.brokerId + "], " @throws(classOf[Exception]) @@ -809,18 +984,74 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg */ @throws(classOf[Exception]) def handleNewSession() { + info("ZK expired; shut down all controller components and try to re-elect") controllerContext.controllerLock synchronized { - Utils.unregisterMBean(KafkaController.MBeanName) - partitionStateMachine.shutdown() - replicaStateMachine.shutdown() - if(controllerContext.controllerChannelManager != null) { - controllerContext.controllerChannelManager.shutdown() - controllerContext.controllerChannelManager = null - } + onControllerResignation() controllerElector.elect } } } + + private def checkAndTriggerPartitionRebalance(): Unit = { + if (isActive()) { + trace("checking need to trigger partition rebalance") + // get all the active brokers + var preferredReplicasForTopicsByBrokers: Map[Int, Map[TopicAndPartition, Seq[Int]]] = null + controllerContext.controllerLock synchronized { + preferredReplicasForTopicsByBrokers = controllerContext.partitionReplicaAssignment.groupBy { + case(topicAndPartition, assignedReplicas) => assignedReplicas.head + } + } + debug("preferred replicas by broker " + preferredReplicasForTopicsByBrokers) + // for each broker, check if a preferred replica election needs to be triggered + preferredReplicasForTopicsByBrokers.foreach { + case(leaderBroker, topicAndPartitionsForBroker) => { + var imbalanceRatio: Double = 0 + var topicsNotInPreferredReplica: Map[TopicAndPartition, Seq[Int]] = null + controllerContext.controllerLock synchronized { + topicsNotInPreferredReplica = + topicAndPartitionsForBroker.filter { + case(topicPartition, replicas) => { + controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader != leaderBroker + } + } + debug("topics not in preferred replica " + topicsNotInPreferredReplica) + val totalTopicPartitionsForBroker = topicAndPartitionsForBroker.size + val totalTopicPartitionsNotLedByBroker = topicsNotInPreferredReplica.size + imbalanceRatio = totalTopicPartitionsNotLedByBroker.toDouble / totalTopicPartitionsForBroker + trace("leader imbalance ratio for broker %d is %f".format(leaderBroker, imbalanceRatio)) + } + // check ratio and if greater than desired ratio, trigger a rebalance for the topic partitions + // that need to be on this broker + if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) { + controllerContext.controllerLock synchronized { + // do this check only if the broker is live and there are no partitions being reassigned currently + // and preferred replica election is not in progress + if (controllerContext.liveBrokerIds.contains(leaderBroker) && + controllerContext.partitionsBeingReassigned.size == 0 && + controllerContext.partitionsUndergoingPreferredReplicaElection.size == 0) { + val zkPath = ZkUtils.PreferredReplicaLeaderElectionPath + val partitionsList = topicsNotInPreferredReplica.keys.map(e => Map("topic" -> e.topic, "partition" -> e.partition)) + val jsonData = Json.encode(Map("version" -> 1, "partitions" -> partitionsList)) + try { + ZkUtils.createPersistentPath(zkClient, zkPath, jsonData) + info("Created preferred replica election path with %s".format(jsonData)) + } catch { + case e2: ZkNodeExistsException => + val partitionsUndergoingPreferredReplicaElection = + PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(ZkUtils.readData(zkClient, zkPath)._1) + error("Preferred replica leader election currently in progress for " + + "%s. Aborting operation".format(partitionsUndergoingPreferredReplicaElection)); + case e3: Throwable => + error("Error while trying to auto rebalance topics %s".format(topicsNotInPreferredReplica.keys)) + } + } + } + } + } + } + } + } } /** diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index a47b142940fd8..fd9200f3bf941 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -26,19 +26,20 @@ trait PartitionLeaderSelector { * @param topicAndPartition The topic and partition whose leader needs to be elected * @param currentLeaderAndIsr The current leader and isr of input partition read from zookeeper * @throws NoReplicaOnlineException If no replica in the assigned replicas list is alive - * @return The leader and isr request, with the newly selected leader info, to send to the brokers - * Also, returns the list of replicas the returned leader and isr request should be sent to - * This API selects a new leader for the input partition + * @return The leader and isr request, with the newly selected leader and isr, and the set of replicas to receive + * the LeaderAndIsrRequest. */ def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) } /** - * This API selects a new leader for the input partition - - * 1. If at least one broker from the isr is alive, it picks a broker from the isr as the new leader - * 2. Else, it picks some alive broker from the assigned replica list as the new leader + * Select the new leader, new isr and receiving replicas (for the LeaderAndIsrRequest): + * 1. If at least one broker from the isr is alive, it picks a broker from the live isr as the new leader and the live + * isr as the new isr. + * 2. Else, it picks some alive broker from the assigned replica list as the new leader and the new isr. * 3. If no broker in the assigned replica list is alive, it throws NoReplicaOnlineException + * Replicas to receive LeaderAndIsr request = live assigned replicas * Once the leader is successfully registered in zookeeper, it updates the allLeaders cache */ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector with Logging { @@ -82,7 +83,9 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten } /** - * Picks one of the alive in-sync reassigned replicas as the new leader. + * New leader = a live in-sync reassigned replica + * New isr = current isr + * Replicas to receive LeaderAndIsr request = reassigned replicas */ class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector with Logging { this.logIdent = "[ReassignedPartitionLeaderSelector]: " @@ -94,7 +97,8 @@ class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) ex val reassignedInSyncReplicas = controllerContext.partitionsBeingReassigned(topicAndPartition).newReplicas val currentLeaderEpoch = currentLeaderAndIsr.leaderEpoch val currentLeaderIsrZkPathVersion = currentLeaderAndIsr.zkVersion - val aliveReassignedInSyncReplicas = reassignedInSyncReplicas.filter(r => controllerContext.liveBrokerIds.contains(r)) + val aliveReassignedInSyncReplicas = reassignedInSyncReplicas.filter(r => controllerContext.liveBrokerIds.contains(r) && + currentLeaderAndIsr.isr.contains(r)) val newLeaderOpt = aliveReassignedInSyncReplicas.headOption newLeaderOpt match { case Some(newLeader) => (new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, currentLeaderAndIsr.isr, @@ -106,16 +110,16 @@ class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) ex " %s is empty. Current leader and ISR: [%s]".format(topicAndPartition, currentLeaderAndIsr)) case _ => throw new StateChangeFailedException("None of the reassigned replicas for partition " + - "%s are alive. Current leader and ISR: [%s]".format(topicAndPartition, currentLeaderAndIsr)) + "%s are in-sync with the leader. Current leader and ISR: [%s]".format(topicAndPartition, currentLeaderAndIsr)) } } } } /** - * Picks the preferred replica as the new leader if - - * 1. It is already not the current leader - * 2. It is alive + * New leader = preferred (first assigned) replica (if in isr and alive); + * New isr = current isr; + * Replicas to receive LeaderAndIsr request = assigned replicas */ class PreferredReplicaPartitionLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector with Logging { @@ -145,8 +149,9 @@ with Logging { } /** - * Picks one of the alive replicas (other than the current leader) in ISR as - * new leader, fails if there are no other replicas in ISR. + * New leader = replica in isr that's not being shutdown; + * New isr = current isr - shutdown replica; + * Replicas to receive LeaderAndIsr request = live assigned replicas */ class ControlledShutdownLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector @@ -164,8 +169,7 @@ class ControlledShutdownLeaderSelector(controllerContext: ControllerContext) val liveOrShuttingDownBrokerIds = controllerContext.liveOrShuttingDownBrokerIds val liveAssignedReplicas = assignedReplicas.filter(r => liveOrShuttingDownBrokerIds.contains(r)) - val newIsr = currentLeaderAndIsr.isr.filter(brokerId => brokerId != currentLeader && - !controllerContext.shuttingDownBrokerIds.contains(brokerId)) + val newIsr = currentLeaderAndIsr.isr.filter(brokerId => !controllerContext.shuttingDownBrokerIds.contains(brokerId)) val newLeaderOpt = newIsr.headOption newLeaderOpt match { case Some(newLeader) => @@ -174,8 +178,8 @@ class ControlledShutdownLeaderSelector(controllerContext: ControllerContext) (LeaderAndIsr(newLeader, currentLeaderEpoch + 1, newIsr, currentLeaderIsrZkPathVersion + 1), liveAssignedReplicas) case None => - throw new StateChangeFailedException(("No other replicas in ISR %s for %s besides current leader %d and" + - " shutting down brokers %s").format(currentLeaderAndIsr.isr.mkString(","), topicAndPartition, currentLeader, controllerContext.shuttingDownBrokerIds.mkString(","))) + throw new StateChangeFailedException(("No other replicas in ISR %s for %s besides" + + " shutting down brokers %s").format(currentLeaderAndIsr.isr.mkString(","), topicAndPartition, controllerContext.shuttingDownBrokerIds.mkString(","))) } } } diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index 829163afab997..90478434b5408 100644 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -119,7 +119,23 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { /** * This API exercises the partition's state machine. It ensures that every state transition happens from a legal - * previous state to the target state. + * previous state to the target state. Valid state transitions are: + * NonExistentPartition -> NewPartition: + * --load assigned replicas from ZK to controller cache + * + * NewPartition -> OnlinePartition + * --assign first live replica as the leader and all live replicas as the isr; write leader and isr to ZK for this partition + * --send LeaderAndIsr request to every live replica and UpdateMetadata request to every live broker + * + * OnlinePartition,OfflinePartition -> OnlinePartition + * --select new leader and isr for this partition and a set of replicas to receive the LeaderAndIsr request, and write leader and isr to ZK + * --for this partition, send LeaderAndIsr request to every receiving replica and UpdateMetadata request to every live broker + * + * NewPartition,OnlinePartition -> OfflinePartition + * --nothing other than marking partition state as Offline + * + * OfflinePartition -> NonExistentPartition + * --nothing other than marking the partition state as NonExistentPartition * @param topic The topic of the partition for which the state transition is invoked * @param partition The partition for which the state transition is invoked * @param targetState The end state that the partition should be moved to @@ -273,8 +289,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } /** - * Invoked on the OfflinePartition->OnlinePartition state change. It invokes the leader election API to elect a leader - * for the input offline partition + * Invoked on the OfflinePartition,OnlinePartition->OnlinePartition state change. + * It invokes the leader election API to elect a leader for the input offline partition * @param topic The topic of the offline partition * @param partition The offline partition * @param leaderSelector Specific leader selector (e.g., offline/reassigned/etc.) @@ -371,9 +387,11 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { controllerContext.allTopics = currentChildren val addedPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, newTopics.toSeq) + val maxRackReplicaAssignment = ZkUtils.getMaxRackReplicationForTopics(zkClient, newTopics.toSeq) controllerContext.partitionReplicaAssignment = controllerContext.partitionReplicaAssignment.filter(p => !deletedTopics.contains(p._1.topic)) controllerContext.partitionReplicaAssignment.++=(addedPartitionReplicaAssignment) + controllerContext.maxRackReplicaAssignment.++=(maxRackReplicaAssignment) info("New topics: [%s], deleted topics: [%s], new partition replica assignment [%s]".format(newTopics, deletedTopics, addedPartitionReplicaAssignment)) if(newTopics.size > 0) @@ -401,7 +419,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { val partitionsRemainingToBeAdded = partitionReplicaAssignment.filter(p => !controllerContext.partitionReplicaAssignment.contains(p._1)) info("New partitions to be added [%s]".format(partitionsRemainingToBeAdded)) - controller.onNewPartitionCreation(partitionsRemainingToBeAdded.keySet.toSet) + if (partitionsRemainingToBeAdded.size > 0) + controller.onNewPartitionCreation(partitionsRemainingToBeAdded.keySet.toSet) } catch { case e: Throwable => error("Error while handling add partitions for data path " + dataPath, e ) } diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 212c05d65dcdc..483559aa64726 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -58,8 +58,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { initializeReplicaState() hasStarted.set(true) // move all Online replicas to Online - handleStateChanges(getAllReplicasOnBroker(controllerContext.allTopics.toSeq, - controllerContext.liveBrokerIds.toSeq), OnlineReplica) + handleStateChanges(controllerContext.allLiveReplicas(), OnlineReplica) info("Started replica state machine with initial state -> " + replicaState.toString()) } @@ -95,7 +94,23 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { /** * This API exercises the replica's state machine. It ensures that every state transition happens from a legal - * previous state to the target state. + * previous state to the target state. Valid state transitions are: + * NonExistentReplica --> NewReplica + * --send LeaderAndIsr request with current leader and isr to the new replica and UpdateMetadata request for the partition to every live broker + * + * NewReplica -> OnlineReplica + * --add the new replica to the assigned replica list if needed + * + * OnlineReplica,OfflineReplica -> OnlineReplica + * --send LeaderAndIsr request with current leader and isr to the new replica and UpdateMetadata request for the partition to every live broker + * + * NewReplica,OnlineReplica -> OfflineReplica + * --send StopReplicaRequest to the replica (w/o deletion) + * --remove this replica from the isr and send LeaderAndIsr request (with new isr) to the leader replica and UpdateMetadata request for the partition to every live broker. + * + * OfflineReplica -> NonExistentReplica + * --send StopReplicaRequest to the replica (with deletion) + * * @param topic The topic of the replica for which the state transition is invoked * @param partition The partition of the replica for which the state transition is invoked * @param replicaId The replica for which the state transition is invoked @@ -165,28 +180,23 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { replicaState.put((topic, partition, replicaId), OnlineReplica) case OfflineReplica => assertValidPreviousStates(topic, partition, replicaId, List(NewReplica, OnlineReplica), targetState) + // send stop replica command to the replica so that it stops fetching from the leader + brokerRequestBatch.addStopReplicaRequestForBrokers(List(replicaId), topic, partition, deletePartition = false) // As an optimization, the controller removes dead replicas from the ISR val leaderAndIsrIsEmpty: Boolean = controllerContext.partitionLeadershipInfo.get(topicAndPartition) match { case Some(currLeaderIsrAndControllerEpoch) => - if (currLeaderIsrAndControllerEpoch.leaderAndIsr.isr.contains(replicaId)) - controller.removeReplicaFromIsr(topic, partition, replicaId) match { - case Some(updatedLeaderIsrAndControllerEpoch) => - // send the shrunk ISR state change request only to the leader - brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader), - topic, partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment) - replicaState.put((topic, partition, replicaId), OfflineReplica) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OfflineReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) - false - case None => - true - } - 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 + controller.removeReplicaFromIsr(topic, partition, replicaId) match { + case Some(updatedLeaderIsrAndControllerEpoch) => + // send the shrunk ISR state change request only to the leader + brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader), + topic, partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment) + replicaState.put((topic, partition, replicaId), OfflineReplica) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OfflineReplica" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + false + case None => + true } case None => true @@ -233,20 +243,6 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { } } - private def getAllReplicasOnBroker(topics: Seq[String], brokerIds: Seq[Int]): Set[PartitionAndReplica] = { - brokerIds.map { brokerId => - val partitionsAssignedToThisBroker = - controllerContext.partitionReplicaAssignment.filter(p => topics.contains(p._1.topic) && p._2.contains(brokerId)) - if(partitionsAssignedToThisBroker.size == 0) - info("No state transitions triggered since no partitions are assigned to brokers %s".format(brokerIds.mkString(","))) - partitionsAssignedToThisBroker.map(p => new PartitionAndReplica(p._1.topic, p._1.partition, brokerId)) - }.flatten.toSet - } - - def getPartitionsAssignedToBroker(topics: Seq[String], brokerId: Int):Seq[TopicAndPartition] = { - controllerContext.partitionReplicaAssignment.filter(_._2.contains(brokerId)).keySet.toSeq - } - /** * This is the zookeeper listener that triggers all the state transitions for a replica */ diff --git a/core/src/main/scala/kafka/javaapi/FetchRequest.scala b/core/src/main/scala/kafka/javaapi/FetchRequest.scala index 6abdc17d14b80..4060077469939 100644 --- a/core/src/main/scala/kafka/javaapi/FetchRequest.scala +++ b/core/src/main/scala/kafka/javaapi/FetchRequest.scala @@ -28,10 +28,11 @@ class FetchRequest(correlationId: Int, minBytes: Int, requestInfo: java.util.Map[TopicAndPartition, PartitionFetchInfo]) { - import scala.collection.JavaConversions._ - val underlying = { - val scalaMap = (requestInfo: mutable.Map[TopicAndPartition, PartitionFetchInfo]).toMap + val scalaMap: Map[TopicAndPartition, PartitionFetchInfo] = { + import scala.collection.JavaConversions._ + (requestInfo: mutable.Map[TopicAndPartition, PartitionFetchInfo]).toMap + } kafka.api.FetchRequest( correlationId = correlationId, clientId = clientId, @@ -42,10 +43,6 @@ class FetchRequest(correlationId: Int, ) } - def writeTo(buffer: ByteBuffer) { underlying.writeTo(buffer) } - - def sizeInBytes = underlying.sizeInBytes - override def toString = underlying.toString override def equals(other: Any) = canEqual(other) && { diff --git a/core/src/main/scala/kafka/javaapi/Implicits.scala b/core/src/main/scala/kafka/javaapi/Implicits.scala index 0af3a678a13e4..8baf4d468027a 100644 --- a/core/src/main/scala/kafka/javaapi/Implicits.scala +++ b/core/src/main/scala/kafka/javaapi/Implicits.scala @@ -34,6 +34,12 @@ private[javaapi] object Implicits extends Logging { implicit def toJavaOffsetResponse(response: kafka.api.OffsetResponse): kafka.javaapi.OffsetResponse = new kafka.javaapi.OffsetResponse(response) + implicit def toJavaOffsetFetchResponse(response: kafka.api.OffsetFetchResponse): kafka.javaapi.OffsetFetchResponse = + new kafka.javaapi.OffsetFetchResponse(response) + + implicit def toJavaOffsetCommitResponse(response: kafka.api.OffsetCommitResponse): kafka.javaapi.OffsetCommitResponse = + new kafka.javaapi.OffsetCommitResponse(response) + implicit def optionToJavaRef[T](opt: Option[T]): T = { opt match { case Some(obj) => obj diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala new file mode 100644 index 0000000000000..57b9d2a9c1974 --- /dev/null +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala @@ -0,0 +1,58 @@ +/** + * 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.javaapi + +import kafka.common.{TopicAndPartition, OffsetMetadataAndError} +import collection.JavaConversions +import java.nio.ByteBuffer + +class OffsetCommitRequest(groupId: String, + requestInfo: java.util.Map[TopicAndPartition, OffsetMetadataAndError], + versionId: Short, + correlationId: Int, + clientId: String) { + val underlying = { + val scalaMap: Map[TopicAndPartition, OffsetMetadataAndError] = { + import JavaConversions._ + requestInfo.toMap + } + kafka.api.OffsetCommitRequest( + groupId = groupId, + requestInfo = scalaMap, + versionId = versionId, + correlationId = correlationId, + clientId = clientId + ) + } + + + override def toString = underlying.toString + + + override def equals(other: Any) = canEqual(other) && { + val otherOffsetRequest = other.asInstanceOf[kafka.javaapi.OffsetCommitRequest] + this.underlying.equals(otherOffsetRequest.underlying) + } + + + def canEqual(other: Any) = other.isInstanceOf[kafka.javaapi.OffsetCommitRequest] + + + override def hashCode = underlying.hashCode + +} diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala new file mode 100644 index 0000000000000..570bf3173f485 --- /dev/null +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala @@ -0,0 +1,31 @@ +/** + * 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.javaapi + +import kafka.common.TopicAndPartition +import collection.JavaConversions + +class OffsetCommitResponse(private val underlying: kafka.api.OffsetCommitResponse) { + + def errors: java.util.Map[TopicAndPartition, Short] = { + import JavaConversions._ + underlying.requestInfo + } + +} + diff --git a/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala new file mode 100644 index 0000000000000..5b4f4bbcdd606 --- /dev/null +++ b/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala @@ -0,0 +1,62 @@ +/** + * 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.javaapi + +import kafka.common.TopicAndPartition +import scala.collection.mutable +import collection.JavaConversions +import java.nio.ByteBuffer + +class OffsetFetchRequest(groupId: String, + requestInfo: java.util.List[TopicAndPartition], + versionId: Short, + correlationId: Int, + clientId: String) { + + val underlying = { + val scalaSeq = { + import JavaConversions._ + requestInfo: mutable.Buffer[TopicAndPartition] + } + kafka.api.OffsetFetchRequest( + groupId = groupId, + requestInfo = scalaSeq, + versionId = versionId, + correlationId = correlationId, + clientId = clientId + ) + } + + + override def toString = underlying.toString + + + override def equals(other: Any) = canEqual(other) && { + val otherOffsetRequest = other.asInstanceOf[kafka.javaapi.OffsetFetchRequest] + this.underlying.equals(otherOffsetRequest.underlying) + } + + + def canEqual(other: Any) = other.isInstanceOf[kafka.javaapi.OffsetFetchRequest] + + + override def hashCode = underlying.hashCode + +} + + diff --git a/core/src/main/scala/kafka/utils/MockTime.scala b/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala similarity index 69% rename from core/src/main/scala/kafka/utils/MockTime.scala rename to core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala index 5296abad87b1c..60924d2fe543d 100644 --- a/core/src/main/scala/kafka/utils/MockTime.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala @@ -5,7 +5,7 @@ * 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 @@ -15,20 +15,17 @@ * limitations under the License. */ -package kafka.utils +package kafka.javaapi -import java.util.concurrent._ +import kafka.common.{TopicAndPartition, OffsetMetadataAndError} +import collection.JavaConversions -class MockTime(@volatile var currentMs: Long) extends Time { - - def this() = this(System.currentTimeMillis) - - def milliseconds: Long = currentMs +class OffsetFetchResponse(private val underlying: kafka.api.OffsetFetchResponse) { - def nanoseconds: Long = - TimeUnit.NANOSECONDS.convert(currentMs, TimeUnit.MILLISECONDS) + def offsets: java.util.Map[TopicAndPartition, OffsetMetadataAndError] = { + import JavaConversions._ + underlying.requestInfo + } - def sleep(ms: Long): Unit = - currentMs += ms - } + diff --git a/core/src/main/scala/kafka/javaapi/OffsetRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetRequest.scala index d88c7e4fd749e..c8a0ded327815 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetRequest.scala @@ -28,8 +28,10 @@ class OffsetRequest(requestInfo: java.util.Map[TopicAndPartition, PartitionOffse clientId: String) { val underlying = { - import collection.JavaConversions._ - val scalaMap = (requestInfo: mutable.Map[TopicAndPartition, PartitionOffsetRequestInfo]).toMap + val scalaMap = { + import collection.JavaConversions._ + (requestInfo: mutable.Map[TopicAndPartition, PartitionOffsetRequestInfo]).toMap + } kafka.api.OffsetRequest( requestInfo = scalaMap, versionId = versionId, @@ -39,11 +41,6 @@ class OffsetRequest(requestInfo: java.util.Map[TopicAndPartition, PartitionOffse } - def writeTo(buffer: ByteBuffer) { underlying.writeTo(buffer) } - - - def sizeInBytes = underlying.sizeInBytes - override def toString = underlying.toString diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index 05757a13ff149..7e6da164a26b1 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -19,6 +19,9 @@ package kafka.javaapi import kafka.api._ import java.nio.ByteBuffer import scala.collection.mutable +import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.common.ErrorMapping +import kafka.network.RequestChannel.Response class TopicMetadataRequest(val versionId: Short, override val correlationId: Int, @@ -41,4 +44,26 @@ class TopicMetadataRequest(val versionId: Short, def sizeInBytes: Int = underlying.sizeInBytes() + override def toString(): String = { + describe(true) + } + + override def describe(details: Boolean): String = { + val topicMetadataRequest = new StringBuilder + topicMetadataRequest.append("Name: " + this.getClass.getSimpleName) + topicMetadataRequest.append("; Version: " + versionId) + topicMetadataRequest.append("; CorrelationId: " + correlationId) + topicMetadataRequest.append("; ClientId: " + clientId) + if(details) { + topicMetadataRequest.append("; Topics: ") + val topicIterator = topics.iterator() + while (topicIterator.hasNext) { + val topic = topicIterator.next() + topicMetadataRequest.append("%s".format(topic)) + if(topicIterator.hasNext) + topicMetadataRequest.append(",") + } + } + topicMetadataRequest.toString() + } } diff --git a/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala index 58c708114ea2c..0ab0195dc9f66 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala @@ -21,7 +21,6 @@ import kafka.utils.threadsafe import kafka.javaapi.FetchResponse import kafka.javaapi.OffsetRequest - /** * A consumer of kafka messages */ @@ -80,6 +79,26 @@ class SimpleConsumer(val host: String, underlying.getOffsetsBefore(request.underlying) } + /** + * Commit offsets for a topic + * @param request a [[kafka.javaapi.OffsetCommitRequest]] object. + * @return a [[kafka.javaapi.OffsetCommitResponse]] object. + */ + def commitOffsets(request: kafka.javaapi.OffsetCommitRequest): kafka.javaapi.OffsetCommitResponse = { + import kafka.javaapi.Implicits._ + underlying.commitOffsets(request.underlying) + } + + /** + * Fetch offsets for a topic + * @param request a [[kafka.javaapi.OffsetFetchRequest]] object. + * @return a [[kafka.javaapi.OffsetFetchResponse]] object. + */ + def fetchOffsets(request: kafka.javaapi.OffsetFetchRequest): kafka.javaapi.OffsetFetchResponse = { + import kafka.javaapi.Implicits._ + underlying.fetchOffsets(request.underlying) + } + def close() { underlying.close } diff --git a/core/src/main/scala/kafka/log/CleanerConfig.scala b/core/src/main/scala/kafka/log/CleanerConfig.scala new file mode 100644 index 0000000000000..fa946ada92602 --- /dev/null +++ b/core/src/main/scala/kafka/log/CleanerConfig.scala @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +/** + * Configuration parameters for the log cleaner + * + * @param numThreads The number of cleaner threads to run + * @param dedupeBufferSize The total memory used for log deduplication + * @param dedupeBufferLoadFactor The maximum percent full for the deduplication buffer + * @param maxMessageSize The maximum size of a message that can appear in the log + * @param maxIoBytesPerSecond The maximum read and write I/O that all cleaner threads are allowed to do + * @param backOffMs The amount of time to wait before rechecking if no logs are eligible for cleaning + * @param enableCleaner Allows completely disabling the log cleaner + * @param hashAlgorithm The hash algorithm to use in key comparison. + */ +case class CleanerConfig(val numThreads: Int = 1, + val dedupeBufferSize: Long = 4*1024*1024L, + val dedupeBufferLoadFactor: Double = 0.9d, + val ioBufferSize: Int = 1024*1024, + val maxMessageSize: Int = 32*1024*1024, + val maxIoBytesPerSecond: Double = Double.MaxValue, + val backOffMs: Long = 60 * 1000, + val enableCleaner: Boolean = true, + val hashAlgorithm: String = "MD5") { +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index c4397b71fb70b..e1f8b979c3e6f 100644 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -29,55 +29,89 @@ import java.util.concurrent.TimeUnit import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} /** - * An on-disk message set. The set can be opened either mutably or immutably. Mutation attempts - * will fail on an immutable message set. An optional limit and start position can be applied to the message set - * which will control the position in the file at which the set begins. + * An on-disk message set. An optional start and end position can be applied to the message set + * which will allow slicing a subset of the file. + * @param file The file name for the underlying log data + * @param channel the underlying file channel used + * @param start A lower bound on the absolute position in the file from which the message set begins + * @param end The upper bound on the absolute position in the file at which the message set ends + * @param isSlice Should the start and end parameters be used for slicing? */ @nonthreadsafe -class FileMessageSet private[kafka](val file: File, +class FileMessageSet private[kafka](@volatile var file: File, private[log] val channel: FileChannel, - private[log] val start: Int = 0, - private[log] val limit: Int = Int.MaxValue, - initChannelPositionToEnd: Boolean = true) extends MessageSet with Logging { + private[log] val start: Int, + private[log] val end: Int, + isSlice: Boolean) extends MessageSet with Logging { /* the size of the message set in bytes */ - private val _size = new AtomicInteger(scala.math.min(channel.size().toInt, limit) - start) + private val _size = + if(isSlice) + new AtomicInteger(end - start) // don't check the file size if this is just a slice view + else + new AtomicInteger(math.min(channel.size().toInt, end) - start) - if (initChannelPositionToEnd) { - debug("Creating or reloading log segment %s".format(file.getAbsolutePath)) + /* if this is not a slice, update the file pointer to the end of the file */ + if (!isSlice) /* set the file position to the last byte in the file */ channel.position(channel.size) - } /** - * Create a file message set with no limit or offset + * Create a file message set with no slicing. + */ + def this(file: File, channel: FileChannel) = + this(file, channel, start = 0, end = Int.MaxValue, isSlice = false) + + /** + * Create a file message set with no slicing + */ + def this(file: File) = + this(file, Utils.openChannel(file, mutable = true)) + + /** + * Create a file message set with mutable option */ - def this(file: File, channel: FileChannel) = this(file, channel, 0, Int.MaxValue) + def this(file: File, mutable: Boolean) = this(file, Utils.openChannel(file, mutable)) /** - * Create a file message set with no limit or offset + * Create a slice view of the file message set that begins and ends at the given byte offsets */ - def this(file: File) = this(file, Utils.openChannel(file, mutable = true)) + def this(file: File, channel: FileChannel, start: Int, end: Int) = + this(file, channel, start, end, isSlice = true) /** * Return a message set which is a view into this set starting from the given position and with the given size limit. + * + * If the size is beyond the end of the file, the end will be based on the size of the file at the time of the read. + * + * If this message set is already sliced, the position will be taken relative to that slicing. + * + * @param position The start position to begin the read from + * @param size The number of bytes after the start position to include + * + * @return A sliced wrapper on this message set limited based on the given position and size */ def read(position: Int, size: Int): FileMessageSet = { + if(position < 0) + throw new IllegalArgumentException("Invalid position: " + position) + if(size < 0) + throw new IllegalArgumentException("Invalid size: " + size) new FileMessageSet(file, channel, - this.start + position, - scala.math.min(this.start + position + size, sizeInBytes()), - false) + start = this.start + position, + end = math.min(this.start + position + size, sizeInBytes())) } /** * Search forward for the file position of the last offset that is greater than or equal to the target offset * and return its physical position. If no such offsets are found, return null. + * @param targetOffset The offset to search for. + * @param startingPosition The starting position in the file to begin searching from. */ - private[log] def searchFor(targetOffset: Long, startingPosition: Int): OffsetPosition = { + def searchFor(targetOffset: Long, startingPosition: Int): OffsetPosition = { var position = startingPosition val buffer = ByteBuffer.allocate(MessageSet.LogOverhead) - val size = _size.get() + val size = sizeInBytes() while(position + MessageSet.LogOverhead < size) { buffer.rewind() channel.read(buffer, position) @@ -89,37 +123,55 @@ class FileMessageSet private[kafka](val file: File, if(offset >= targetOffset) return OffsetPosition(offset, position) val messageSize = buffer.getInt() + if(messageSize < Message.MessageOverhead) + throw new IllegalStateException("Invalid message size: " + messageSize) position += MessageSet.LogOverhead + messageSize } null } /** - * Write some of this set to the given channel, return the amount written + * Write some of this set to the given channel. + * @param destChannel The channel to write to. + * @param writePosition The position in the message set to begin writing from. + * @param size The maximum number of bytes to write + * @return The number of bytes actually written. */ def writeTo(destChannel: GatheringByteChannel, writePosition: Long, size: Int): Int = { // Ensure that the underlying size has not changed. - val newSize = scala.math.min(channel.size().toInt, limit) - start + val newSize = math.min(channel.size().toInt, end) - start if (newSize < _size.get()) { throw new KafkaException("Size of FileMessageSet %s has been truncated during write: old size %d, new size %d" .format(file.getAbsolutePath, _size.get(), newSize)) } - val bytesTransferred = channel.transferTo(start + writePosition, scala.math.min(size, sizeInBytes), destChannel).toInt + val bytesTransferred = channel.transferTo(start + writePosition, math.min(size, sizeInBytes), destChannel).toInt trace("FileMessageSet " + file.getAbsolutePath + " : bytes transferred : " + bytesTransferred - + " bytes requested for transfer : " + scala.math.min(size, sizeInBytes)) + + " bytes requested for transfer : " + math.min(size, sizeInBytes)) bytesTransferred } + /** + * Get a shallow iterator over the messages in the set. + */ + override def iterator() = iterator(Int.MaxValue) + /** * Get an iterator over the messages in the set. We only do shallow iteration here. + * @param maxMessageSize A limit on allowable message size to avoid allocating unbounded memory. + * If we encounter a message larger than this we throw an InvalidMessageException. + * @return The iterator. */ - override def iterator: Iterator[MessageAndOffset] = { + def iterator(maxMessageSize: Int): Iterator[MessageAndOffset] = { new IteratorTemplate[MessageAndOffset] { var location = start + val sizeOffsetBuffer = ByteBuffer.allocate(12) override def makeNext(): MessageAndOffset = { + if(location >= end) + return allDone() + // read the size of the item - val sizeOffsetBuffer = ByteBuffer.allocate(12) + sizeOffsetBuffer.rewind() channel.read(sizeOffsetBuffer, location) if(sizeOffsetBuffer.hasRemaining) return allDone() @@ -127,8 +179,10 @@ class FileMessageSet private[kafka](val file: File, sizeOffsetBuffer.rewind() val offset = sizeOffsetBuffer.getLong() val size = sizeOffsetBuffer.getInt() - if (size < Message.MinHeaderSize) + if(size < Message.MinHeaderSize) return allDone() + if(size > maxMessageSize) + throw new InvalidMessageException("Message size exceeds the largest allowable message size (%d).".format(maxMessageSize)) // read the item itself val buffer = ByteBuffer.allocate(size) @@ -150,7 +204,7 @@ class FileMessageSet private[kafka](val file: File, def sizeInBytes(): Int = _size.get() /** - * Append this message to the message set + * Append these messages to the message set */ def append(messages: ByteBufferMessageSet) { val written = messages.writeTo(channel, 0, messages.sizeInBytes) @@ -161,9 +215,7 @@ class FileMessageSet private[kafka](val file: File, * Commit all written data to the physical disk */ def flush() = { - LogFlushStats.logFlushTimer.time { - channel.force(true) - } + channel.force(true) } /** @@ -176,6 +228,7 @@ class FileMessageSet private[kafka](val file: File, /** * Delete this message set from the filesystem + * @return True iff this message set was deleted. */ def delete(): Boolean = { Utils.swallow(channel.close()) @@ -183,16 +236,39 @@ class FileMessageSet private[kafka](val file: File, } /** - * Truncate this file message set to the given size. Note that this API does no checking that the - * given size falls on a valid byte offset. + * Truncate this file message set to the given size in bytes. Note that this API does no checking that the + * given size falls on a valid message boundary. + * @param targetSize The size to truncate to. + * @return The number of bytes truncated off */ - def truncateTo(targetSize: Int) = { - if(targetSize > sizeInBytes()) - throw new KafkaException("Attempt to truncate log segment to %d bytes failed since the current ".format(targetSize) + - " size of this log segment is only %d bytes".format(sizeInBytes())) + def truncateTo(targetSize: Int): Int = { + val originalSize = sizeInBytes + if(targetSize > originalSize || targetSize < 0) + throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " + + " size of this log segment is " + originalSize + " bytes.") channel.truncate(targetSize) channel.position(targetSize) _size.set(targetSize) + originalSize - targetSize + } + + /** + * Read from the underlying file into the buffer starting at the given position + */ + def readInto(buffer: ByteBuffer, position: Int): ByteBuffer = { + channel.read(buffer, position) + buffer.flip() + buffer + } + + /** + * Rename the file that backs this message set + * @return true iff the rename was successful + */ + def renameTo(f: File): Boolean = { + val success = this.file.renameTo(f) + this.file = f + success } } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index f6348969ea382..b3ab5220a66a2 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -17,76 +17,17 @@ package kafka.log -import kafka.api.OffsetRequest import java.io.{IOException, File} -import java.util.{Comparator, Collections, ArrayList} +import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap} import java.util.concurrent.atomic._ import kafka.utils._ -import scala.math._ +import scala.collection.JavaConversions import java.text.NumberFormat -import kafka.server.BrokerTopicStats import kafka.message._ import kafka.common._ import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge -object Log { - val LogFileSuffix = ".log" - val IndexFileSuffix = ".index" - - /** - * Search for the greatest range with start <= the target value. - */ - def findRange[T <: Range](ranges: Array[T], value: Long, arraySize: Int): Option[T] = { - if(ranges.size < 1) - return None - - // check out of bounds - if(value < ranges(0).start) - return None - - var low = 0 - var high = arraySize - 1 - while(low < high) { - val mid = ceil((high + low) / 2.0).toInt - val found = ranges(mid) - if(found.start == value) - return Some(found) - else if (value < found.start) - high = mid - 1 - else - low = mid - } - Some(ranges(low)) - } - - def findRange[T <: Range](ranges: Array[T], value: Long): Option[T] = - findRange(ranges, value, ranges.length) - - /** - * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros - * so that ls sorts the files numerically - */ - def filenamePrefixFromOffset(offset: Long): String = { - val nf = NumberFormat.getInstance() - nf.setMinimumIntegerDigits(20) - nf.setMaximumFractionDigits(0) - nf.setGroupingUsed(false) - nf.format(offset) - } - - def logFilename(dir: File, offset: Long) = - new File(dir, filenamePrefixFromOffset(offset) + LogFileSuffix) - - def indexFilename(dir: File, offset: Long) = - new File(dir, filenamePrefixFromOffset(offset) + IndexFileSuffix) - - def getEmptyOffsets(timestamp: Long): Seq[Long] = - if (timestamp == OffsetRequest.LatestTime || timestamp == OffsetRequest.EarliestTime) - Seq(0L) - else Nil -} - /** * An append-only log for storing messages. @@ -96,37 +37,37 @@ object Log { * New log segments are created according to a configurable policy that controls the size in bytes or time interval * for a given segment. * + * @param dir The directory in which log segments are created. + * @param config The log configuration settings + * @param recoveryPoint The offset at which to begin recovery--i.e. the first offset which has not been flushed to disk + * @param scheduler The thread pool scheduler used for background actions + * @param time The time instance used for checking the clock + * */ @threadsafe -private[kafka] class Log(val dir: File, - val maxLogFileSize: Int, - val maxMessageSize: Int, - val flushInterval: Int = Int.MaxValue, - val rollIntervalMs: Long = Long.MaxValue, - val needsRecovery: Boolean, - val maxIndexSize: Int = (10*1024*1024), - val indexIntervalBytes: Int = 4096, - time: Time = SystemTime, - brokerId: Int = 0) extends Logging with KafkaMetricsGroup { - this.logIdent = "[Kafka Log on Broker " + brokerId + "], " +class Log(val dir: File, + @volatile var config: LogConfig, + @volatile var recoveryPoint: Long = 0L, + val scheduler: Scheduler, + time: Time = SystemTime) extends Logging with KafkaMetricsGroup { import kafka.log.Log._ /* A lock that guards all modifications to the log */ private val lock = new Object - /* The current number of unflushed messages appended to the write */ - private val unflushed = new AtomicInteger(0) - /* last time it was flushed */ - private val lastflushedTime = new AtomicLong(System.currentTimeMillis) + private val lastflushedTime = new AtomicLong(time.milliseconds) /* the actual segments of the log */ - private[log] val segments: SegmentList[LogSegment] = loadSegments() - - /* Calculate the offset of the next message */ - private var nextOffset: AtomicLong = new AtomicLong(segments.view.last.nextOffset()) + private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment] + loadSegments() + /* Calculate the offset of the next message */ + private val nextOffset: AtomicLong = new AtomicLong(activeSegment.nextOffset()) + + val topicAndPartition: TopicAndPartition = Log.parseTopicPartitionName(name) + info("Completed load of log %s with log end offset %d".format(name, logEndOffset)) newGauge(name + "-" + "NumLogSegments", @@ -135,136 +76,137 @@ private[kafka] class Log(val dir: File, newGauge(name + "-" + "LogEndOffset", new Gauge[Long] { def value = logEndOffset }) - /* The name of this log */ + /** The name of this log */ def name = dir.getName() /* Load the log segments from the log files on disk */ - private def loadSegments(): SegmentList[LogSegment] = { - // open all the segments read-only - val logSegments = new ArrayList[LogSegment] - val ls = dir.listFiles() - - if(ls != null) { - for(file <- ls if file.isFile) { - val filename = file.getName() - if(!file.canRead) { - throw new IOException("Could not read file " + file) - } else if(filename.endsWith(IndexFileSuffix)) { - // ensure that we have a corresponding log file for this index file - val log = new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix)) - if(!log.exists) { - warn("Found an orphaned index file, %s, with no corresponding log file.".format(file.getAbsolutePath)) - file.delete() - } - } else if(filename.endsWith(LogFileSuffix)) { - val offset = filename.substring(0, filename.length - LogFileSuffix.length).toLong - // TODO: we should ideally rebuild any missing index files, instead of erroring out - if(!Log.indexFilename(dir, offset).exists) - throw new IllegalStateException("Found log file with no corresponding index file.") - logSegments.add(new LogSegment(dir = dir, - startOffset = offset, - indexIntervalBytes = indexIntervalBytes, - maxIndexSize = maxIndexSize)) + private def loadSegments() { + // create the log directory if it doesn't exist + dir.mkdirs() + + // first do a pass through the files in the log directory and remove any temporary files + // and complete any interrupted swap operations + for(file <- dir.listFiles if file.isFile) { + if(!file.canRead) + throw new IOException("Could not read file " + file) + val filename = file.getName + if(filename.endsWith(DeletedFileSuffix) || filename.endsWith(CleanedFileSuffix)) { + // if the file ends in .deleted or .cleaned, delete it + file.delete() + } else if(filename.endsWith(SwapFileSuffix)) { + // we crashed in the middle of a swap operation, to recover: + // if a log, swap it in and delete the .index file + // if an index just delete it, it will be rebuilt + val baseName = new File(Utils.replaceSuffix(file.getPath, SwapFileSuffix, "")) + if(baseName.getPath.endsWith(IndexFileSuffix)) { + file.delete() + } else if(baseName.getPath.endsWith(LogFileSuffix)){ + // delete the index + val index = new File(Utils.replaceSuffix(baseName.getPath, LogFileSuffix, IndexFileSuffix)) + index.delete() + // complete the swap operation + val renamed = file.renameTo(baseName) + if(renamed) + info("Found log file %s from interrupted swap operation, repairing.".format(file.getPath)) + else + throw new KafkaException("Failed to rename file %s.".format(file.getPath)) } } } + // now do a second pass and load all the .log and .index files + for(file <- dir.listFiles if file.isFile) { + val filename = file.getName + if(filename.endsWith(IndexFileSuffix)) { + // if it is an index file, make sure it has a corresponding .log file + val logFile = new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix)) + if(!logFile.exists) { + warn("Found an orphaned index file, %s, with no corresponding log file.".format(file.getAbsolutePath)) + file.delete() + } + } else if(filename.endsWith(LogFileSuffix)) { + // if its a log file, load the corresponding log segment + val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong + val hasIndex = Log.indexFilename(dir, start).exists + val segment = new LogSegment(dir = dir, + startOffset = start, + indexIntervalBytes = config.indexInterval, + maxIndexSize = config.maxIndexSize, + time = time) + if(!hasIndex) { + error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath)) + segment.recover(config.maxMessageSize) + } + segments.put(start, segment) + } + } + if(logSegments.size == 0) { - // no existing segments, create a new mutable segment - logSegments.add(new LogSegment(dir = dir, + // no existing segments, create a new mutable segment beginning at offset 0 + segments.put(0, new LogSegment(dir = dir, startOffset = 0, - indexIntervalBytes = indexIntervalBytes, - maxIndexSize = maxIndexSize)) + indexIntervalBytes = config.indexInterval, + maxIndexSize = config.maxIndexSize, + time = time)) } else { - // there is at least one existing segment, validate and recover them/it - // sort segments into ascending order for fast searching - Collections.sort(logSegments, new Comparator[LogSegment] { - def compare(s1: LogSegment, s2: LogSegment): Int = { - if(s1.start == s2.start) 0 - else if(s1.start < s2.start) -1 - else 1 - } - }) + recoverLog() + // reset the index size of the currently active log segment to allow more entries + activeSegment.index.resize(config.maxIndexSize) + } - // reset the index size of the last (current active) log segment to its maximum value - logSegments.get(logSegments.size() - 1).index.resize(maxIndexSize) + // sanity check the index file of every segment to ensure we don't proceed with a corrupt segment + for (s <- logSegments) + s.index.sanityCheck() + } + + private def recoverLog() { + // if we have the clean shutdown marker, skip recovery + if(hasCleanShutdownFile) { + this.recoveryPoint = activeSegment.nextOffset + return + } - // run recovery on the last segment if necessary - if(needsRecovery) { - var activeSegment = logSegments.get(logSegments.size - 1) + // okay we need to actually recovery this log + val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator + while(unflushed.hasNext) { + val curr = unflushed.next + info("Recovering unflushed segment %d in log %s.".format(curr.baseOffset, name)) + val truncatedBytes = try { - recoverSegment(activeSegment) + curr.recover(config.maxMessageSize) } catch { - case e: InvalidOffsetException => - val startOffset = activeSegment.start - warn("Found invalid offset during recovery of the active segment for topic partition " + dir.getName +". Deleting the segment and " + + case e: InvalidOffsetException => + val startOffset = curr.baseOffset + warn("Found invalid offset during recovery for log " + dir.getName +". Deleting the corrupt segment and " + "creating an empty one with starting offset " + startOffset) - // truncate the active segment to its starting offset - activeSegment.truncateTo(startOffset) + curr.truncateTo(startOffset) } + if(truncatedBytes > 0) { + // we had an invalid message, delete all remaining log + warn("Corruption found in segment %d of log %s, truncating to offset %d.".format(curr.baseOffset, name, curr.nextOffset)) + unflushed.foreach(deleteSegment) } } - - val segmentList = logSegments.toArray(new Array[LogSegment](logSegments.size)) - // Check for the index file of every segment, if it's empty or its last offset is greater than its base offset. - for (s <- segmentList) { - require(s.index.entries == 0 || s.index.lastOffset > s.index.baseOffset, - "Corrupt index found, index file (%s) has non-zero size but the last offset is %d and the base offset is %d" - .format(s.index.file.getAbsolutePath, s.index.lastOffset, s.index.baseOffset)) - } - - new SegmentList(segmentList) } /** - * Run recovery on the given segment. This will rebuild the index from the log file and lop off any invalid bytes from the end of the log. + * Check if we have the "clean shutdown" file */ - private def recoverSegment(segment: LogSegment) { - info("Recovering log segment %s".format(segment.messageSet.file.getAbsolutePath)) - segment.index.truncate() - var validBytes = 0 - var lastIndexEntry = 0 - val iter = segment.messageSet.iterator - try { - while(iter.hasNext) { - val entry = iter.next - entry.message.ensureValid() - if(validBytes - lastIndexEntry > indexIntervalBytes) { - // we need to decompress the message, if required, to get the offset of the first uncompressed message - val startOffset = - entry.message.compressionCodec match { - case NoCompressionCodec => - entry.offset - case _ => - ByteBufferMessageSet.decompress(entry.message).head.offset - } - segment.index.append(startOffset, validBytes) - lastIndexEntry = validBytes - } - validBytes += MessageSet.entrySize(entry.message) - } - } catch { - case e: InvalidMessageException => - logger.warn("Found invalid messages in log " + name) - } - val truncated = segment.messageSet.sizeInBytes - validBytes - if(truncated > 0) - warn("Truncated " + truncated + " invalid bytes from the log " + name + ".") - segment.messageSet.truncateTo(validBytes) - } + private def hasCleanShutdownFile() = new File(dir.getParentFile, CleanShutdownFile).exists() /** - * The number of segments in the log + * The number of segments in the log. + * Take care! this is an O(n) operation. */ - def numberOfSegments: Int = segments.view.length - + def numberOfSegments: Int = segments.size + /** * Close this log */ def close() { debug("Closing log " + name) lock synchronized { - for(seg <- segments.view) + for(seg <- logSegments) seg.close() } } @@ -275,97 +217,97 @@ private[kafka] class Log(val dir: File, * This method will generally be responsible for assigning offsets to the messages, * however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid. * - * Returns a tuple containing (first_offset, last_offset) for the newly appended of the message set, - * or (-1,-1) if the message set is empty + * @param messages The message set to append + * @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given + * + * @throws KafkaStorageException If the append fails due to an I/O error. + * + * @return Information about the appended messages including the first and last offset. */ - def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): (Long, Long) = { - val messageSetInfo = analyzeAndValidateMessageSet(messages) - + def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): LogAppendInfo = { + val appendInfo = analyzeAndValidateMessageSet(messages) + // if we have any valid messages, append them to the log - if(messageSetInfo.count == 0) { - (-1L, -1L) - } else { - // trim any invalid bytes or partial messages before appending it to the on-disk log - var validMessages = trimInvalidBytes(messages) + if(appendInfo.shallowCount == 0) + return appendInfo + + // trim any invalid bytes or partial messages before appending it to the on-disk log + var validMessages = trimInvalidBytes(messages) + + try { + // they are valid, insert them in the log + lock synchronized { + appendInfo.firstOffset = nextOffset.get - try { - // they are valid, insert them in the log - val offsets = lock synchronized { - val firstOffset = nextOffset.get + // maybe roll the log if this segment is full + val segment = maybeRoll() - // maybe roll the log if this segment is full - val segment = maybeRoll(segments.view.last) - + if(assignOffsets) { // assign offsets to the messageset - val lastOffset = - if(assignOffsets) { - val offsetCounter = new AtomicLong(nextOffset.get) - try { - validMessages = validMessages.assignOffsets(offsetCounter, messageSetInfo.codec) - } catch { - case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e) - } - val assignedLastOffset = offsetCounter.get - 1 - val numMessages = assignedLastOffset - firstOffset + 1 - BrokerTopicStats.getBrokerTopicStats(topicName).messagesInRate.mark(numMessages) - BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numMessages) - assignedLastOffset - } else { - require(messageSetInfo.offsetsMonotonic, "Out of order offsets found in " + messages) - require(messageSetInfo.firstOffset >= nextOffset.get, - "Attempt to append a message set beginning with offset %d to a log with log end offset %d." - .format(messageSetInfo.firstOffset, nextOffset.get)) - messageSetInfo.lastOffset - } - - // Check if the message sizes are valid. This check is done after assigning offsets to ensure the comparison - // happens with the new message size (after re-compression, if any) - for(messageAndOffset <- validMessages.shallowIterator) { - if(MessageSet.entrySize(messageAndOffset.message) > maxMessageSize) - throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d." - .format(MessageSet.entrySize(messageAndOffset.message), maxMessageSize)) + val offset = new AtomicLong(nextOffset.get) + try { + validMessages = validMessages.assignOffsets(offset, appendInfo.codec) + } catch { + case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e) } + appendInfo.lastOffset = offset.get - 1 + } else { + // we are taking the offsets we are given + if(!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffset.get) + throw new IllegalArgumentException("Out of order offsets found in " + messages) + } - // now append to the log - segment.append(firstOffset, validMessages) - - // advance the log end offset - nextOffset.set(lastOffset + 1) + // Check if the message sizes are valid. This check is done after assigning offsets to ensure the comparison + // happens with the new message size (after re-compression, if any) + for(messageAndOffset <- validMessages.shallowIterator) { + if(MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) + throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d." + .format(MessageSet.entrySize(messageAndOffset.message), config.maxMessageSize)) + } - trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s" - .format(this.name, firstOffset, nextOffset.get(), validMessages)) + // now append to the log + segment.append(appendInfo.firstOffset, validMessages) - // return the offset at which the messages were appended - (firstOffset, lastOffset) - } - - // maybe flush the log and index - val numAppendedMessages = (offsets._2 - offsets._1 + 1).toInt - maybeFlush(numAppendedMessages) - - // return the first and last offset - offsets - } catch { - case e: IOException => throw new KafkaStorageException("I/O exception in append to log '%s'".format(name), e) + // increment the log end offset + nextOffset.set(appendInfo.lastOffset + 1) + + trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s" + .format(this.name, appendInfo.firstOffset, nextOffset.get(), validMessages)) + + if(unflushedMessages >= config.flushInterval) + flush() + + appendInfo } + } catch { + case e: IOException => throw new KafkaStorageException("I/O exception in append to log '%s'".format(name), e) } } - /* struct to hold various quantities we compute about each message set before appending to the log */ - case class MessageSetAppendInfo(firstOffset: Long, lastOffset: Long, codec: CompressionCodec, count: Int, offsetsMonotonic: Boolean) + /** Struct to hold various quantities we compute about each message set before appending to the log + * @param firstOffset The first offset in the message set + * @param lastOffset The last offset in the message set + * @param codec The codec used in the message set + * @param offsetsMonotonic Are the offsets in this message set monotonically increasing + */ + case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, offsetsMonotonic: Boolean) /** * Validate the following: - * 1. each message matches its CRC + *
    + *
  1. each message matches its CRC + *
* * Also compute the following quantities: - * 1. First offset in the message set - * 2. Last offset in the message set - * 3. Number of messages - * 4. Whether the offsets are monotonically increasing - * 5. Whether any compression codec is used (if many are used, then the last one is given) + *
    + *
  1. First offset in the message set + *
  2. Last offset in the message set + *
  3. Number of messages + *
  4. Whether the offsets are monotonically increasing + *
  5. Whether any compression codec is used (if many are used, then the last one is given) + *
*/ - private def analyzeAndValidateMessageSet(messages: ByteBufferMessageSet): MessageSetAppendInfo = { + private def analyzeAndValidateMessageSet(messages: ByteBufferMessageSet): LogAppendInfo = { var messageCount = 0 var firstOffset, lastOffset = -1L var codec: CompressionCodec = NoCompressionCodec @@ -383,18 +325,19 @@ private[kafka] class Log(val dir: File, // check the validity of the message by checking CRC val m = messageAndOffset.message m.ensureValid() - messageCount += 1; val messageCodec = m.compressionCodec if(messageCodec != NoCompressionCodec) codec = messageCodec } - MessageSetAppendInfo(firstOffset, lastOffset, codec, messageCount, monotonic) + LogAppendInfo(firstOffset, lastOffset, codec, messageCount, monotonic) } /** * Trim any invalid bytes from the end of this message set (if there are any) + * @param messages The message set to trim + * @return A trimmed message set. This may be the same as what was passed in or it may not. */ private def trimInvalidBytes(messages: ByteBufferMessageSet): ByteBufferMessageSet = { val messageSetValidBytes = messages.validBytes @@ -411,277 +354,387 @@ private[kafka] class Log(val dir: File, } /** - * Read a message set from the log. - * startOffset - The logical offset to begin reading at - * maxLength - The maximum number of bytes to read - * maxOffset - The first offset not included in the read + * Read messages from the log + * @param startOffset The offset to begin reading at + * @param maxLength The maximum number of bytes to read + * @param maxOffset -The offset to read up to, exclusive. (i.e. the first offset NOT included in the resulting message set). + * + * @throws OffsetOutOfRangeException If startOffset is beyond the log end offset or before the base offset of the first segment. + * @return The messages read */ def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None): MessageSet = { trace("Reading %d bytes from offset %d in log %s of length %d bytes".format(maxLength, startOffset, name, size)) - val view = segments.view - + // check if the offset is valid and in range - val first = view.head.start val next = nextOffset.get if(startOffset == next) return MessageSet.Empty - else if(startOffset > next || startOffset < first) - throw new OffsetOutOfRangeException("Request for offset %d but we only have log segments in the range %d to %d.".format(startOffset, first, next)) - // Do the read on the segment with a base offset less than the target offset - // TODO: to handle sparse offsets, we need to skip to the next segment if this read doesn't find anything - Log.findRange(view, startOffset, view.length) match { - case None => throw new OffsetOutOfRangeException("Offset is earlier than the earliest offset") - case Some(segment) => segment.read(startOffset, maxLength, maxOffset) + var entry = segments.floorEntry(startOffset) + + // attempt to read beyond the log end offset is an error + if(startOffset > next || entry == null) + throw new OffsetOutOfRangeException("Request for offset %d but we only have log segments in the range %d to %d.".format(startOffset, segments.firstKey, next)) + + // do the read on the segment with a base offset less than the target offset + // but if that segment doesn't contain any messages with an offset greater than that + // continue to read from successive segments until we get some messages or we reach the end of the log + while(entry != null) { + val messages = entry.getValue.read(startOffset, maxOffset, maxLength) + if(messages == null) + entry = segments.higherEntry(entry.getKey) + else + return messages } + + // okay we are beyond the end of the last segment but less than the log end offset + MessageSet.Empty } /** - * Delete any log segments matching the given predicate function + * Delete any log segments matching the given predicate function, + * starting with the oldest segment and moving forward until a segment doesn't match. + * @param predicate A function that takes in a single log segment and returns true iff it is deletable + * @return The number of segments deleted */ - def markDeletedWhile(predicate: LogSegment => Boolean): Seq[LogSegment] = { - lock synchronized { - debug("Garbage collecting log..") - debug("Segments of log %s : %s ".format(this.name, segments.view.mkString(","))) - debug("Index files for log %s: %s".format(this.name, segments.view.map(_.index.file.exists()).mkString(","))) - debug("Data files for log %s: %s".format(this.name, segments.view.map(_.messageSet.file.exists()).mkString(","))) - val view = segments.view - val deletable = view.takeWhile(predicate) - for(seg <- deletable) - seg.deleted = true - var numToDelete = deletable.size - // if we are deleting everything, create a new empty segment - if(numToDelete == view.size) { - if (view(numToDelete - 1).size > 0) + def deleteOldSegments(predicate: LogSegment => Boolean): Int = { + // find any segments that match the user-supplied predicate UNLESS it is the final segment + // and it is empty (since we would just end up re-creating it + val lastSegment = activeSegment + var deletable = logSegments.takeWhile(s => predicate(s) && (s.baseOffset != lastSegment.baseOffset || s.size > 0)) + val numToDelete = deletable.size + if(numToDelete > 0) { + lock synchronized { + // we must always have at least one segment, so if we are going to delete all the segments, create a new one first + if(segments.size == numToDelete) roll() - else { - // If the last segment to be deleted is empty and we roll the log, the new segment will have the same - // file name. So simply reuse the last segment and reset the modified time. - view(numToDelete - 1).messageSet.file.setLastModified(time.milliseconds) - numToDelete -=1 - } + // remove the segments for lookups + deletable.foreach(deleteSegment(_)) } - segments.trunc(numToDelete) } + numToDelete } /** - * Get the size of the log in bytes + * The size of the log in bytes */ - def size: Long = segments.view.foldLeft(0L)(_ + _.size) + def size: Long = logSegments.map(_.size).sum /** - * Get the offset of the next message that will be appended + * The offset of the next message that will be appended to the log */ def logEndOffset: Long = nextOffset.get /** - * Roll the log over if necessary + * Roll the log over to a new empty log segment if necessary + * @return The currently active segment after (perhaps) rolling to a new segment */ - private def maybeRoll(segment: LogSegment): LogSegment = { - if(segment.messageSet.sizeInBytes > maxLogFileSize) { - info("Rolling %s due to full data log".format(name)) + private def maybeRoll(): LogSegment = { + val segment = activeSegment + if (segment.size > config.segmentSize || + segment.size > 0 && time.milliseconds - segment.created > config.segmentMs || + segment.index.isFull) { + debug("Rolling new log segment in %s (log_size = %d/%d, index_size = %d/%d, age_ms = %d/%d)." + .format(name, + segment.size, + config.segmentSize, + segment.index.entries, + segment.index.maxEntries, + time.milliseconds - segment.created, + config.segmentMs)) roll() - } else if((segment.firstAppendTime.isDefined) && (time.milliseconds - segment.firstAppendTime.get > rollIntervalMs)) { - info("Rolling %s due to time based rolling".format(name)) - roll() - } else if(segment.index.isFull) { - info("Rolling %s due to full index maxIndexSize = %d, entries = %d, maxEntries = %d" - .format(name, segment.index.maxIndexSize, segment.index.entries(), segment.index.maxEntries)) - roll() - } else + } else { segment + } } - + /** - * Create a new segment and make it active, and return it + * Roll the log over to a new active segment starting with the current logEndOffset. + * This will trim the index to the exact size of the number of entries it currently contains. + * @return The newly rolled segment */ def roll(): LogSegment = { + val start = time.nanoseconds lock synchronized { - flush() - rollToOffset(logEndOffset) + val newOffset = logEndOffset + val logFile = logFilename(dir, newOffset) + val indexFile = indexFilename(dir, newOffset) + for(file <- List(logFile, indexFile); if file.exists) { + warn("Newly rolled segment file " + file.getName + " already exists; deleting it first") + file.delete() + } + + segments.lastEntry() match { + case null => + case entry => entry.getValue.index.trimToValidSize() + } + val segment = new LogSegment(dir, + startOffset = newOffset, + indexIntervalBytes = config.indexInterval, + maxIndexSize = config.maxIndexSize, + time = time) + val prev = addSegment(segment) + if(prev != null) + throw new KafkaException("Trying to roll a new log segment for topic partition %s with start offset %d while it already exists.".format(name, newOffset)) + + // schedule an asynchronous flush of the old segment + scheduler.schedule("flush-log", () => flush(newOffset), delay = 0L) + + info("Rolled new log segment for '" + name + "' in %.0f ms.".format((System.nanoTime - start) / (1000.0*1000.0))) + + segment } } /** - * Roll the log over to the given new offset value + * The number of messages appended to the log since the last flush */ - private def rollToOffset(newOffset: Long): LogSegment = { - val logFile = logFilename(dir, newOffset) - val indexFile = indexFilename(dir, newOffset) - for(file <- List(logFile, indexFile); if file.exists) { - warn("Newly rolled segment file " + file.getAbsolutePath + " already exists; deleting it first") - file.delete() - } - info("Rolling log '" + name + "' to " + logFile.getAbsolutePath + " and " + indexFile.getAbsolutePath) - segments.view.lastOption match { - case Some(segment) => segment.index.trimToValidSize() - case None => - } - - val segmentsView = segments.view - if(segmentsView.size > 0 && segmentsView.last.start == newOffset) - throw new KafkaException("Trying to roll a new log segment for topic partition %s with start offset %d while it already exists".format(dir.getName, newOffset)) - - val segment = new LogSegment(dir, - startOffset = newOffset, - indexIntervalBytes = indexIntervalBytes, - maxIndexSize = maxIndexSize) - segments.append(segment) - segment - } - + def unflushedMessages() = this.logEndOffset - this.recoveryPoint + /** - * Flush the log if necessary + * Flush all log segments */ - private def maybeFlush(numberOfMessages : Int) { - if(unflushed.addAndGet(numberOfMessages) >= flushInterval) - flush() - } + def flush(): Unit = flush(this.logEndOffset) /** - * Flush this log file to the physical disk + * Flush log segments for all offsets up to offset-1 + * @param offset The offset to flush up to (non-inclusive); the new recovery point */ - def flush() : Unit = { - if (unflushed.get == 0) + def flush(offset: Long) : Unit = { + if (offset <= this.recoveryPoint) return - + debug("Flushing log '" + name + " up to offset " + offset + ", last flushed: " + lastFlushTime + " current time: " + + time.milliseconds + " unflushed = " + unflushedMessages) + for(segment <- logSegments(this.recoveryPoint, offset)) + segment.flush() lock synchronized { - debug("Flushing log '" + name + "' last flushed: " + getLastFlushedTime + " current time: " + - time.milliseconds) - segments.view.last.flush() - unflushed.set(0) - lastflushedTime.set(time.milliseconds) - } - } - - def getOffsetsBefore(timestamp: Long, maxNumOffsets: Int): Seq[Long] = { - val segsArray = segments.view - var offsetTimeArray: Array[(Long, Long)] = null - if(segsArray.last.size > 0) - offsetTimeArray = new Array[(Long, Long)](segsArray.length + 1) - else - offsetTimeArray = new Array[(Long, Long)](segsArray.length) - - for(i <- 0 until segsArray.length) - offsetTimeArray(i) = (segsArray(i).start, segsArray(i).messageSet.file.lastModified) - if(segsArray.last.size > 0) - offsetTimeArray(segsArray.length) = (logEndOffset, time.milliseconds) - - var startIndex = -1 - timestamp match { - case OffsetRequest.LatestTime => - startIndex = offsetTimeArray.length - 1 - case OffsetRequest.EarliestTime => - startIndex = 0 - case _ => - var isFound = false - debug("Offset time array = " + offsetTimeArray.foreach(o => "%d, %d".format(o._1, o._2))) - startIndex = offsetTimeArray.length - 1 - while (startIndex >= 0 && !isFound) { - if (offsetTimeArray(startIndex)._2 <= timestamp) - isFound = true - else - startIndex -=1 - } - } - - val retSize = maxNumOffsets.min(startIndex + 1) - val ret = new Array[Long](retSize) - for(j <- 0 until retSize) { - ret(j) = offsetTimeArray(startIndex)._1 - startIndex -= 1 + if(offset > this.recoveryPoint) { + this.recoveryPoint = offset + lastflushedTime.set(time.milliseconds) + } } - // ensure that the returned seq is in descending order of offsets - ret.toSeq.sortBy(- _) } - def delete(): Unit = { - deleteSegments(segments.contents.get()) - Utils.rm(dir) - } - - - /* Attempts to delete all provided segments from a log and returns how many it was able to */ - def deleteSegments(segments: Seq[LogSegment]): Int = { - var total = 0 - for(segment <- segments) { - info("Deleting log segment " + segment.start + " from " + name) - val deletedLog = segment.messageSet.delete() - val deletedIndex = segment.index.delete() - if(!deletedIndex || !deletedLog) { - throw new KafkaStorageException("Deleting log segment " + segment.start + " failed.") - } else { - total += 1 - } - if(segment.messageSet.file.exists()) - error("Data log file %s still exists".format(segment.messageSet.file.getAbsolutePath)) - if(segment.index.file.exists()) - error("Index file %s still exists".format(segment.index.file.getAbsolutePath)) + /** + * Completely delete this log directory and all contents from the file system with no delay + */ + private[log] def delete() { + lock synchronized { + logSegments.foreach(_.delete()) + segments.clear() + Utils.rm(dir) } - total } - - def truncateTo(targetOffset: Long) { + + /** + * Truncate this log so that it ends with the greatest offset < targetOffset. + * @param targetOffset The offset to truncate to, an upper bound on all offsets in the log after truncation is complete. + */ + private[log] def truncateTo(targetOffset: Long) { + info("Truncating log %s to offset %d.".format(name, targetOffset)) if(targetOffset < 0) throw new IllegalArgumentException("Cannot truncate to a negative offset (%d).".format(targetOffset)) + if(targetOffset > logEndOffset) { + info("Truncating %s to %d has no effect as the largest offset in the log is %d.".format(name, targetOffset, logEndOffset-1)) + return + } lock synchronized { - val segmentsToBeDeleted = segments.view.filter(segment => segment.start > targetOffset) - val viewSize = segments.view.size - val numSegmentsDeleted = deleteSegments(segmentsToBeDeleted) - /* We should not hit this error because segments.view is locked in markedDeletedWhile() */ - if(numSegmentsDeleted != segmentsToBeDeleted.size) - error("Failed to delete some segments when attempting to truncate to offset " + targetOffset +")") - if (numSegmentsDeleted == viewSize) { - segments.trunc(segments.view.size) - rollToOffset(targetOffset) - this.nextOffset.set(targetOffset) + if(segments.firstEntry.getValue.baseOffset > targetOffset) { + truncateFullyAndStartAt(targetOffset) } else { - if(targetOffset > logEndOffset) { - error("Target offset %d cannot be greater than the last message offset %d in the log %s". - format(targetOffset, logEndOffset, segments.view.last.messageSet.file.getAbsolutePath)) - } else { - // find the log segment that has this hw - val segmentToBeTruncated = findRange(segments.view, targetOffset) - segmentToBeTruncated match { - case Some(segment) => - val truncatedSegmentIndex = segments.view.indexOf(segment) - segments.truncLast(truncatedSegmentIndex) - segment.truncateTo(targetOffset) - this.nextOffset.set(targetOffset) - info("Truncated log segment %s to target offset %d".format(segments.view.last.messageSet.file.getAbsolutePath, targetOffset)) - case None => // nothing to do - } - } + val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset) + deletable.foreach(deleteSegment(_)) + activeSegment.truncateTo(targetOffset) + this.nextOffset.set(targetOffset) + this.recoveryPoint = math.min(targetOffset, this.recoveryPoint) } } } - /** - * Truncate all segments in the log and start a new segment on a new offset + /** + * Delete all data in the log and start at the new offset + * @param newOffset The new offset to start the log with */ - def truncateAndStartWithNewOffset(newOffset: Long) { + private[log] def truncateFullyAndStartAt(newOffset: Long) { + debug("Truncate and start log '" + name + "' to " + newOffset) lock synchronized { - val deletedSegments = segments.trunc(segments.view.size) - info("Truncate and start log '" + name + "' to " + newOffset) - deleteSegments(deletedSegments) - segments.append(new LogSegment(dir, - newOffset, - indexIntervalBytes = indexIntervalBytes, - maxIndexSize = maxIndexSize)) + val segmentsToDelete = logSegments.toList + segmentsToDelete.foreach(deleteSegment(_)) + addSegment(new LogSegment(dir, + newOffset, + indexIntervalBytes = config.indexInterval, + maxIndexSize = config.maxIndexSize, + time = time)) this.nextOffset.set(newOffset) + this.recoveryPoint = math.min(newOffset, this.recoveryPoint) } } - def topicName():String = { - name.substring(0, name.lastIndexOf("-")) + /** + * The time this log is last known to have been fully flushed to disk + */ + def lastFlushTime(): Long = lastflushedTime.get + + /** + * The active segment that is currently taking appends + */ + def activeSegment = segments.lastEntry.getValue + + /** + * All the log segments in this log ordered from oldest to newest + */ + def logSegments: Iterable[LogSegment] = { + import JavaConversions._ + segments.values + } + + /** + * Get all segments beginning with the segment that includes "from" and ending with the segment + * that includes up to "to-1" or the end of the log (if to > logEndOffset) + */ + def logSegments(from: Long, to: Long): Iterable[LogSegment] = { + import JavaConversions._ + lock synchronized { + val floor = segments.floorKey(from) + if(floor eq null) + segments.headMap(to).values + else + segments.subMap(floor, true, to, false).values + } + } + + override def toString() = "Log(" + dir + ")" + + /** + * This method performs an asynchronous log segment delete by doing the following: + *
    + *
  1. It removes the segment from the segment map so that it will no longer be used for reads. + *
  2. It renames the index and log files by appending .deleted to the respective file name + *
  3. It schedules an asynchronous delete operation to occur in the future + *
+ * This allows reads to happen concurrently without synchronization and without the possibility of physically + * deleting a file while it is being read from. + * + * @param segment The log segment to schedule for deletion + */ + private def deleteSegment(segment: LogSegment) { + info("Scheduling log segment %d for log %s for deletion.".format(segment.baseOffset, name)) + lock synchronized { + segments.remove(segment.baseOffset) + asyncDeleteSegment(segment) + } + } + + /** + * Perform an asynchronous delete on the given file if it exists (otherwise do nothing) + * @throws KafkaStorageException if the file can't be renamed and still exists + */ + private def asyncDeleteSegment(segment: LogSegment) { + segment.changeFileSuffixes("", Log.DeletedFileSuffix) + def deleteSeg() { + info("Deleting segment %d from log %s.".format(segment.baseOffset, name)) + segment.delete() + } + scheduler.schedule("delete-file", deleteSeg, delay = config.fileDeleteDelayMs) } + + /** + * Swap a new segment in place and delete one or more existing segments in a crash-safe manner. The old segments will + * be asynchronously deleted. + * + * @param newSegment The new log segment to add to the log + * @param oldSegments The old log segments to delete from the log + */ + private[log] def replaceSegments(newSegment: LogSegment, oldSegments: Seq[LogSegment]) { + lock synchronized { + // need to do this in two phases to be crash safe AND do the delete asynchronously + // if we crash in the middle of this we complete the swap in loadSegments() + newSegment.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix) + addSegment(newSegment) + + // delete the old files + for(seg <- oldSegments) { + // remove the index entry + if(seg.baseOffset != newSegment.baseOffset) + segments.remove(seg.baseOffset) + // delete segment + asyncDeleteSegment(seg) + } + // okay we are safe now, remove the swap suffix + newSegment.changeFileSuffixes(Log.SwapFileSuffix, "") + } + } + + /** + * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it. + * @param segment The segment to add + */ + def addSegment(segment: LogSegment) = this.segments.put(segment.baseOffset, segment) + +} + +/** + * Helper functions for logs + */ +object Log { + + /** a log file */ + val LogFileSuffix = ".log" + + /** an index file */ + val IndexFileSuffix = ".index" + + /** a file that is scheduled to be deleted */ + val DeletedFileSuffix = ".deleted" + + /** A temporary file that is being used for log cleaning */ + val CleanedFileSuffix = ".cleaned" + + /** A temporary file used when swapping files into the log */ + val SwapFileSuffix = ".swap" - def getLastFlushedTime():Long = { - return lastflushedTime.get + /** Clean shutdown file that indicates the broker was cleanly shutdown in 0.8. This is required to maintain backwards compatibility + * with 0.8 and avoid unnecessary log recovery when upgrading from 0.8 to 0.8.1 */ + /** TODO: Get rid of CleanShutdownFile in 0.8.2 */ + val CleanShutdownFile = ".kafka_cleanshutdown" + + /** + * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros + * so that ls sorts the files numerically. + * @param offset The offset to use in the file name + * @return The filename + */ + def filenamePrefixFromOffset(offset: Long): String = { + val nf = NumberFormat.getInstance() + nf.setMinimumIntegerDigits(20) + nf.setMaximumFractionDigits(0) + nf.setGroupingUsed(false) + nf.format(offset) } - override def toString() = "Log(" + this.dir + ")" + /** + * Construct a log file name in the given dir with the given base offset + * @param dir The directory in which the log will reside + * @param offset The base offset of the log file + */ + def logFilename(dir: File, offset: Long) = + new File(dir, filenamePrefixFromOffset(offset) + LogFileSuffix) + /** + * Construct an index file name in the given dir using the given base offset + * @param dir The directory in which the log will reside + * @param offset The base offset of the log file + */ + def indexFilename(dir: File, offset: Long) = + new File(dir, filenamePrefixFromOffset(offset) + IndexFileSuffix) + + + /** + * Parse the topic and partition out of the directory name of a log + */ + def parseTopicPartitionName(name: String): TopicAndPartition = { + val index = name.lastIndexOf('-') + TopicAndPartition(name.substring(0,index), name.substring(index+1).toInt) + } } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala new file mode 100644 index 0000000000000..64046472a1f88 --- /dev/null +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -0,0 +1,553 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +import scala.collection._ +import scala.math +import java.nio._ +import java.util.Date +import java.io.File +import kafka.common._ +import kafka.message._ +import kafka.utils._ +import java.lang.IllegalStateException + +/** + * The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy. + * A message with key K and offset O is obsolete if there exists a message with key K and offset O' such that O < O'. + * + * Each log can be thought of being split into two sections of segments: a "clean" section which has previously been cleaned followed by a + * "dirty" section that has not yet been cleaned. The active log segment is always excluded from cleaning. + * + * The cleaning is carried out by a pool of background threads. Each thread chooses the dirtiest log that has the "dedupe" retention policy + * and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log. + * + * To clean a log the cleaner first builds a mapping of key=>last_offset for the dirty section of the log. See kafka.log.OffsetMap for details of + * the implementation of the mapping. + * + * Once the key=>offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a + * higher offset than what is found in the segment (i.e. messages with a key that appears in the dirty section of the log). + * + * To avoid segments shrinking to very small sizes with repeated cleanings we implement a rule by which if we will merge successive segments when + * doing a cleaning if their log and index size are less than the maximum log and index size prior to the clean beginning. + * + * Cleaned segments are swapped into the log as they become available. + * + * One nuance that the cleaner must handle is log truncation. If a log is truncated while it is being cleaned the cleaning of that log is aborted. + * + * Messages with null payload are treated as deletes for the purpose of log compaction. This means that they receive special treatment by the cleaner. + * The cleaner will only retain delete records for a period of time to avoid accumulating space indefinitely. This period of time is configurable on a per-topic + * basis and is measured from the time the segment enters the clean portion of the log (at which point any prior message with that key has been removed). + * Delete markers in the clean section of the log that are older than this time will not be retained when log segments are being recopied as part of cleaning. + * + * @param config Configuration parameters for the cleaner + * @param logDirs The directories where offset checkpoints reside + * @param logs The pool of logs + * @param time A way to control the passage of time + */ +class LogCleaner(val config: CleanerConfig, + val logDirs: Array[File], + val logs: Pool[TopicAndPartition, Log], + time: Time = SystemTime) extends Logging { + /* for managing the state of partitions being cleaned. */ + private val cleanerManager = new LogCleanerManager(logDirs, logs); + + /* a throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate */ + private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond, + checkIntervalMs = 300, + throttleDown = true, + time = time) + + /* the threads */ + private val cleaners = (0 until config.numThreads).map(new CleanerThread(_)) + + /** + * Start the background cleaning + */ + def startup() { + info("Starting the log cleaner") + cleaners.foreach(_.start()) + } + + /** + * Stop the background cleaning + */ + def shutdown() { + info("Shutting down the log cleaner.") + cleaners.foreach(_.shutdown()) + } + + /** + * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of + * the partition is aborted. + */ + def abortCleaning(topicAndPartition: TopicAndPartition) { + cleanerManager.abortCleaning(topicAndPartition) + } + + /** + * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition. + * This call blocks until the cleaning of the partition is aborted and paused. + */ + def abortAndPauseCleaning(topicAndPartition: TopicAndPartition) { + cleanerManager.abortAndPauseCleaning(topicAndPartition) + } + + /** + * Resume the cleaning of a paused partition. This call blocks until the cleaning of a partition is resumed. + */ + def resumeCleaning(topicAndPartition: TopicAndPartition) { + cleanerManager.resumeCleaning(topicAndPartition) + } + + /** + * TODO: + * For testing, a way to know when work has completed. This method blocks until the + * cleaner has processed up to the given offset on the specified topic/partition + */ + def awaitCleaned(topic: String, part: Int, offset: Long, timeout: Long = 30000L): Unit = { + while(!cleanerManager.allCleanerCheckpoints.contains(TopicAndPartition(topic, part))) + Thread.sleep(10) + } + + /** + * The cleaner threads do the actual log cleaning. Each thread processes does its cleaning repeatedly by + * choosing the dirtiest log, cleaning it, and then swapping in the cleaned segments. + */ + private class CleanerThread(threadId: Int) + extends ShutdownableThread(name = "kafka-log-cleaner-thread-" + threadId, isInterruptible = false) { + if(config.dedupeBufferSize / config.numThreads > Int.MaxValue) + warn("Cannot use more than 2G of cleaner buffer space per cleaner thread, ignoring excess buffer space...") + + val cleaner = new Cleaner(id = threadId, + offsetMap = new SkimpyOffsetMap(memory = math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt, + hashAlgorithm = config.hashAlgorithm), + ioBufferSize = config.ioBufferSize / config.numThreads / 2, + maxIoBufferSize = config.maxMessageSize, + dupBufferLoadFactor = config.dedupeBufferLoadFactor, + throttler = throttler, + time = time, + checkDone = checkDone) + + private def checkDone(topicAndPartition: TopicAndPartition) { + if (!isRunning.get()) + throw new ThreadShutdownException + cleanerManager.checkCleaningAborted(topicAndPartition) + } + + /** + * The main loop for the cleaner thread + */ + override def doWork() { + cleanOrSleep() + } + + /** + * Clean a log if there is a dirty log available, otherwise sleep for a bit + */ + private def cleanOrSleep() { + cleanerManager.grabFilthiestLog() match { + case None => + // there are no cleanable logs, sleep a while + time.sleep(config.backOffMs) + case Some(cleanable) => + // there's a log, clean it + var endOffset = cleanable.firstDirtyOffset + try { + endOffset = cleaner.clean(cleanable) + logStats(cleaner.id, cleanable.log.name, cleanable.firstDirtyOffset, endOffset, cleaner.stats) + } catch { + case pe: LogCleaningAbortedException => // task can be aborted, let it go. + } finally { + cleanerManager.doneCleaning(cleanable.topicPartition, cleanable.log.dir.getParentFile, endOffset) + } + } + } + + /** + * Log out statistics on a single run of the cleaner. + */ + def logStats(id: Int, name: String, from: Long, to: Long, stats: CleanerStats) { + def mb(bytes: Double) = bytes / (1024*1024) + val message = + "%n\tLog cleaner %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) + + "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead), + stats.elapsedSecs, + mb(stats.bytesRead/stats.elapsedSecs)) + + "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead), + stats.elapsedIndexSecs, + mb(stats.mapBytesRead)/stats.elapsedIndexSecs, + 100 * stats.elapsedIndexSecs.toDouble/stats.elapsedSecs) + + "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead), + stats.elapsedSecs - stats.elapsedIndexSecs, + mb(stats.bytesRead)/(stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs).toDouble/stats.elapsedSecs) + + "\tStart size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesRead), stats.messagesRead) + + "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten), stats.messagesWritten) + + "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead), + 100.0 * (1.0 - stats.messagesWritten.toDouble/stats.messagesRead)) + info(message) + } + + } +} + +/** + * This class holds the actual logic for cleaning a log + * @param id An identifier used for logging + * @param offsetMap The map used for deduplication + * @param bufferSize The size of the buffers to use. Memory usage will be 2x this number as there is a read and write buffer. + * @param throttler The throttler instance to use for limiting I/O rate. + * @param time The time instance + */ +private[log] class Cleaner(val id: Int, + offsetMap: OffsetMap, + ioBufferSize: Int, + maxIoBufferSize: Int, + dupBufferLoadFactor: Double, + throttler: Throttler, + time: Time, + checkDone: (TopicAndPartition) => Unit) extends Logging { + + this.logIdent = "Cleaner " + id + ": " + + /* stats on this cleaning */ + val stats = new CleanerStats(time) + + /* buffer used for read i/o */ + private var readBuffer = ByteBuffer.allocate(ioBufferSize) + + /* buffer used for write i/o */ + private var writeBuffer = ByteBuffer.allocate(ioBufferSize) + + /** + * Clean the given log + * + * @param cleanable The log to be cleaned + * + * @return The first offset not cleaned + */ + private[log] def clean(cleanable: LogToClean): Long = { + stats.clear() + info("Beginning cleaning of log %s.".format(cleanable.log.name)) + val log = cleanable.log + + // build the offset map + info("Building offset map for %s...".format(cleanable.log.name)) + val upperBoundOffset = log.activeSegment.baseOffset + val endOffset = buildOffsetMap(log, cleanable.firstDirtyOffset, upperBoundOffset, offsetMap) + 1 + stats.indexDone() + + // figure out the timestamp below which it is safe to remove delete tombstones + // this position is defined to be a configurable time beneath the last modified time of the last clean segment + val deleteHorizonMs = + log.logSegments(0, cleanable.firstDirtyOffset).lastOption match { + case None => 0L + case Some(seg) => seg.lastModified - log.config.deleteRetentionMs + } + + // group the segments and clean the groups + info("Cleaning log %s (discarding tombstones prior to %s)...".format(log.name, new Date(deleteHorizonMs))) + for (group <- groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, log.config.maxIndexSize)) + cleanSegments(log, group, offsetMap, deleteHorizonMs) + + stats.allDone() + endOffset + } + + /** + * Clean a group of segments into a single replacement segment + * + * @param log The log being cleaned + * @param segments The group of segments being cleaned + * @param map The offset map to use for cleaning segments + * @param expectedTruncateCount A count used to check if the log is being truncated and rewritten under our feet + * @param deleteHorizonMs The time to retain delete tombstones + */ + private[log] def cleanSegments(log: Log, + segments: Seq[LogSegment], + map: OffsetMap, + deleteHorizonMs: Long) { + // create a new segment with the suffix .cleaned appended to both the log and index name + val logFile = new File(segments.head.log.file.getPath + Log.CleanedFileSuffix) + logFile.delete() + val indexFile = new File(segments.head.index.file.getPath + Log.CleanedFileSuffix) + indexFile.delete() + val messages = new FileMessageSet(logFile) + val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize) + val cleaned = new LogSegment(messages, index, segments.head.baseOffset, segments.head.indexIntervalBytes, time) + + try { + // clean segments into the new destination segment + for (old <- segments) { + val retainDeletes = old.lastModified > deleteHorizonMs + info("Cleaning segment %s in log %s (last modified %s) into %s, %s deletes." + .format(old.baseOffset, log.name, new Date(old.lastModified), cleaned.baseOffset, if(retainDeletes) "retaining" else "discarding")) + cleanInto(log.topicAndPartition, old, cleaned, map, retainDeletes) + } + + // trim excess index + index.trimToValidSize() + + // flush new segment to disk before swap + cleaned.flush() + + // update the modification date to retain the last modified date of the original files + val modified = segments.last.lastModified + cleaned.lastModified = modified + + // swap in new segment + info("Swapping in cleaned segment %d for segment(s) %s in log %s.".format(cleaned.baseOffset, segments.map(_.baseOffset).mkString(","), log.name)) + log.replaceSegments(cleaned, segments) + } catch { + case e: LogCleaningAbortedException => + cleaned.delete() + throw e + } + } + + /** + * Clean the given source log segment into the destination segment using the key=>offset mapping + * provided + * + * @param source The dirty log segment + * @param dest The cleaned log segment + * @param map The key=>offset mapping + * @param retainDeletes Should delete tombstones be retained while cleaning this segment + * + * TODO: Implement proper compression support + */ + private[log] def cleanInto(topicAndPartition: TopicAndPartition, source: LogSegment, + dest: LogSegment, map: OffsetMap, retainDeletes: Boolean) { + var position = 0 + while (position < source.log.sizeInBytes) { + checkDone(topicAndPartition) + // read a chunk of messages and copy any that are to be retained to the write buffer to be written out + readBuffer.clear() + writeBuffer.clear() + val messages = new ByteBufferMessageSet(source.log.readInto(readBuffer, position)) + throttler.maybeThrottle(messages.sizeInBytes) + // check each message to see if it is to be retained + var messagesRead = 0 + for (entry <- messages) { + messagesRead += 1 + val size = MessageSet.entrySize(entry.message) + position += size + stats.readMessage(size) + val key = entry.message.key + require(key != null, "Found null key in log segment %s which is marked as dedupe.".format(source.log.file.getAbsolutePath)) + val foundOffset = map.get(key) + /* two cases in which we can get rid of a message: + * 1) if there exists a message with the same key but higher offset + * 2) if the message is a delete "tombstone" marker and enough time has passed + */ + val redundant = foundOffset >= 0 && entry.offset < foundOffset + val obsoleteDelete = !retainDeletes && entry.message.isNull + if (!redundant && !obsoleteDelete) { + ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset) + stats.recopyMessage(size) + } + } + // if any messages are to be retained, write them out + if(writeBuffer.position > 0) { + writeBuffer.flip() + val retained = new ByteBufferMessageSet(writeBuffer) + dest.append(retained.head.offset, retained) + throttler.maybeThrottle(writeBuffer.limit) + } + + // if we read bytes but didn't get even one complete message, our I/O buffer is too small, grow it and try again + if(readBuffer.limit > 0 && messagesRead == 0) + growBuffers() + } + restoreBuffers() + } + + /** + * Double the I/O buffer capacity + */ + def growBuffers() { + if(readBuffer.capacity >= maxIoBufferSize || writeBuffer.capacity >= maxIoBufferSize) + throw new IllegalStateException("This log contains a message larger than maximum allowable size of %s.".format(maxIoBufferSize)) + val newSize = math.min(this.readBuffer.capacity * 2, maxIoBufferSize) + info("Growing cleaner I/O buffers from " + readBuffer.capacity + "bytes to " + newSize + " bytes.") + this.readBuffer = ByteBuffer.allocate(newSize) + this.writeBuffer = ByteBuffer.allocate(newSize) + } + + /** + * Restore the I/O buffer capacity to its original size + */ + def restoreBuffers() { + if(this.readBuffer.capacity > this.ioBufferSize) + this.readBuffer = ByteBuffer.allocate(this.ioBufferSize) + if(this.writeBuffer.capacity > this.ioBufferSize) + this.writeBuffer = ByteBuffer.allocate(this.ioBufferSize) + } + + /** + * Group the segments in a log into groups totaling less than a given size. the size is enforced separately for the log data and the index data. + * We collect a group of such segments together into a single + * destination segment. This prevents segment sizes from shrinking too much. + * + * @param segments The log segments to group + * @param maxSize the maximum size in bytes for the total of all log data in a group + * @param maxIndexSize the maximum size in bytes for the total of all index data in a group + * + * @return A list of grouped segments + */ + private[log] def groupSegmentsBySize(segments: Iterable[LogSegment], maxSize: Int, maxIndexSize: Int): List[Seq[LogSegment]] = { + var grouped = List[List[LogSegment]]() + var segs = segments.toList + while(!segs.isEmpty) { + var group = List(segs.head) + var logSize = segs.head.size + var indexSize = segs.head.index.sizeInBytes + segs = segs.tail + while(!segs.isEmpty && + logSize + segs.head.size < maxSize && + indexSize + segs.head.index.sizeInBytes < maxIndexSize) { + group = segs.head :: group + logSize += segs.head.size + indexSize += segs.head.index.sizeInBytes + segs = segs.tail + } + grouped ::= group.reverse + } + grouped.reverse + } + + /** + * Build a map of key_hash => offset for the keys in the dirty portion of the log to use in cleaning. + * @param log The log to use + * @param start The offset at which dirty messages begin + * @param end The ending offset for the map that is being built + * @param map The map in which to store the mappings + * + * @return The final offset the map covers + */ + private[log] def buildOffsetMap(log: Log, start: Long, end: Long, map: OffsetMap): Long = { + map.clear() + val dirty = log.logSegments(start, end).toSeq + info("Building offset map for log %s for %d segments in offset range [%d, %d).".format(log.name, dirty.size, start, end)) + + // Add all the dirty segments. We must take at least map.slots * load_factor, + // but we may be able to fit more (if there is lots of duplication in the dirty section of the log) + var offset = dirty.head.baseOffset + require(offset == start, "Last clean offset is %d but segment base offset is %d for log %s.".format(start, offset, log.name)) + val minStopOffset = (start + map.slots * this.dupBufferLoadFactor).toLong + for (segment <- dirty) { + checkDone(log.topicAndPartition) + if(segment.baseOffset <= minStopOffset || map.utilization < this.dupBufferLoadFactor) + offset = buildOffsetMapForSegment(log.topicAndPartition, segment, map) + } + info("Offset map for log %s complete.".format(log.name)) + offset + } + + /** + * Add the messages in the given segment to the offset map + * + * @param segment The segment to index + * @param map The map in which to store the key=>offset mapping + * + * @return The final offset covered by the map + */ + private def buildOffsetMapForSegment(topicAndPartition: TopicAndPartition, segment: LogSegment, map: OffsetMap): Long = { + var position = 0 + var offset = segment.baseOffset + while (position < segment.log.sizeInBytes) { + checkDone(topicAndPartition) + readBuffer.clear() + val messages = new ByteBufferMessageSet(segment.log.readInto(readBuffer, position)) + throttler.maybeThrottle(messages.sizeInBytes) + val startPosition = position + for (entry <- messages) { + val message = entry.message + require(message.hasKey) + val size = MessageSet.entrySize(message) + position += size + map.put(message.key, entry.offset) + offset = entry.offset + stats.indexMessage(size) + } + // if we didn't read even one complete message, our read buffer may be too small + if(position == startPosition) + growBuffers() + } + restoreBuffers() + offset + } +} + +/** + * A simple struct for collecting stats about log cleaning + */ +private case class CleanerStats(time: Time = SystemTime) { + var startTime, mapCompleteTime, endTime, bytesRead, bytesWritten, mapBytesRead, mapMessagesRead, messagesRead, messagesWritten = 0L + clear() + + def readMessage(size: Int) { + messagesRead += 1 + bytesRead += size + } + + def recopyMessage(size: Int) { + messagesWritten += 1 + bytesWritten += size + } + + def indexMessage(size: Int) { + mapMessagesRead += 1 + mapBytesRead += size + } + + def indexDone() { + mapCompleteTime = time.milliseconds + } + + def allDone() { + endTime = time.milliseconds + } + + def elapsedSecs = (endTime - startTime)/1000.0 + + def elapsedIndexSecs = (mapCompleteTime - startTime)/1000.0 + + def clear() { + startTime = time.milliseconds + mapCompleteTime = -1L + endTime = -1L + bytesRead = 0L + bytesWritten = 0L + mapBytesRead = 0L + mapMessagesRead = 0L + messagesRead = 0L + messagesWritten = 0L + } +} + +/** + * Helper class for a log, its topic/partition, and the last clean position + */ +private case class LogToClean(topicPartition: TopicAndPartition, log: Log, firstDirtyOffset: Long) extends Ordered[LogToClean] { + val cleanBytes = log.logSegments(-1, firstDirtyOffset-1).map(_.size).sum + val dirtyBytes = log.logSegments(firstDirtyOffset, math.max(firstDirtyOffset, log.activeSegment.baseOffset)).map(_.size).sum + val cleanableRatio = dirtyBytes / totalBytes.toDouble + def totalBytes = cleanBytes + dirtyBytes + override def compare(that: LogToClean): Int = math.signum(this.cleanableRatio - that.cleanableRatio).toInt +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala new file mode 100644 index 0000000000000..1612c8dbcaba8 --- /dev/null +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -0,0 +1,188 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +import java.io.File +import kafka.utils.{Logging, Pool} +import kafka.server.OffsetCheckpoint +import collection.mutable +import java.util.concurrent.locks.ReentrantLock +import kafka.utils.Utils._ +import java.util.concurrent.TimeUnit +import kafka.common.{LogCleaningAbortedException, TopicAndPartition} + +private[log] sealed trait LogCleaningState +private[log] case object LogCleaningInProgress extends LogCleaningState +private[log] case object LogCleaningAborted extends LogCleaningState +private[log] case object LogCleaningPaused extends LogCleaningState + +/** + * Manage the state of each partition being cleaned. + * If a partition is to be cleaned, it enters the LogCleaningInProgress state. + * While a partition is being cleaned, it can be requested to be aborted and paused. Then the partition first enters + * the LogCleaningAborted state. Once the cleaning task is aborted, the partition enters the LogCleaningPaused state. + * While a partition is in the LogCleaningPaused state, it won't be scheduled for cleaning again, until cleaning is + * requested to be resumed. + */ +private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[TopicAndPartition, Log]) extends Logging { + /* the offset checkpoints holding the last cleaned point for each log */ + private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, "cleaner-offset-checkpoint")))).toMap + + /* the set of logs currently being cleaned */ + private val inProgress = mutable.HashMap[TopicAndPartition, LogCleaningState]() + + /* a global lock used to control all access to the in-progress set and the offset checkpoints */ + private val lock = new ReentrantLock + /* for coordinating the pausing and the cleaning of a partition */ + private val pausedCleaningCond = lock.newCondition() + + /** + * @return the position processed for all logs. + */ + def allCleanerCheckpoints(): Map[TopicAndPartition, Long] = + checkpoints.values.flatMap(_.read()).toMap + + /** + * Choose the log to clean next and add it to the in-progress set. We recompute this + * every time off the full set of logs to allow logs to be dynamically added to the pool of logs + * the log manager maintains. + */ + def grabFilthiestLog(): Option[LogToClean] = { + inLock(lock) { + val lastClean = allCleanerCheckpoints() + val cleanableLogs = logs.filter(l => l._2.config.dedupe) // skip any logs marked for delete rather than dedupe + .filterNot(l => inProgress.contains(l._1)) // skip any logs already in-progress + .map(l => LogToClean(l._1, l._2, lastClean.getOrElse(l._1, 0))) // create a LogToClean instance for each + val dirtyLogs = cleanableLogs.filter(l => l.totalBytes > 0) // must have some bytes + .filter(l => l.cleanableRatio > l.log.config.minCleanableRatio) // and must meet the minimum threshold for dirty byte ratio + if(dirtyLogs.isEmpty) { + None + } else { + val filthiest = dirtyLogs.max + inProgress.put(filthiest.topicPartition, LogCleaningInProgress) + Some(filthiest) + } + } + } + + /** + * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of + * the partition is aborted. + * This is implemented by first abortAndPausing and then resuming the cleaning of the partition. + */ + def abortCleaning(topicAndPartition: TopicAndPartition) { + inLock(lock) { + abortAndPauseCleaning(topicAndPartition) + resumeCleaning(topicAndPartition) + info("The cleaning for partition %s is aborted".format(topicAndPartition)) + } + } + + /** + * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition. + * This call blocks until the cleaning of the partition is aborted and paused. + * 1. If the partition is not in progress, mark it as paused. + * 2. Otherwise, first mark the state of the partition as aborted. + * 3. The cleaner thread checks the state periodically and if it sees the state of the partition is aborted, it + * throws a LogCleaningAbortedException to stop the cleaning task. + * 4. When the cleaning task is stopped, doneCleaning() is called, which sets the state of the partition as paused. + * 5. abortAndPauseCleaning() waits until the state of the partition is changed to paused. + */ + def abortAndPauseCleaning(topicAndPartition: TopicAndPartition) { + inLock(lock) { + inProgress.get(topicAndPartition) match { + case None => + inProgress.put(topicAndPartition, LogCleaningPaused) + case Some(state) => + state match { + case LogCleaningInProgress => + inProgress.put(topicAndPartition, LogCleaningAborted) + case s => + throw new IllegalStateException(("Partiiton %s can't be aborted and pasued since it's in %s state").format(topicAndPartition, s)) + } + } + while (!isCleaningInState(topicAndPartition, LogCleaningPaused)) + pausedCleaningCond.await(100, TimeUnit.MILLISECONDS) + info("The cleaning for partition %s is aborted and paused".format(topicAndPartition)) + } + } + + /** + * Resume the cleaning of a paused partition. This call blocks until the cleaning of a partition is resumed. + */ + def resumeCleaning(topicAndPartition: TopicAndPartition) { + inLock(lock) { + inProgress.get(topicAndPartition) match { + case None => + throw new IllegalStateException(("Partiiton %s can't be resumed since it's never paused").format(topicAndPartition)) + case Some(state) => + state match { + case LogCleaningPaused => + inProgress.remove(topicAndPartition) + case s => + throw new IllegalStateException(("Partiiton %s can't be resumed since it's in %s state").format(topicAndPartition, s)) + } + } + } + info("The cleaning for partition %s is resumed".format(topicAndPartition)) + } + + /** + * Check if the cleaning for a partition is in a particular state. The caller is expected to hold lock while making the call. + */ + def isCleaningInState(topicAndPartition: TopicAndPartition, expectedState: LogCleaningState): Boolean = { + inProgress.get(topicAndPartition) match { + case None => return false + case Some(state) => + if (state == expectedState) + return true + else + return false + } + } + + /** + * Check if the cleaning for a partition is aborted. If so, throw an exception. + */ + def checkCleaningAborted(topicAndPartition: TopicAndPartition) { + inLock(lock) { + if (isCleaningInState(topicAndPartition, LogCleaningAborted)) + throw new LogCleaningAbortedException() + } + } + + /** + * Save out the endOffset and remove the given log from the in-progress set, if not aborted. + */ + def doneCleaning(topicAndPartition: TopicAndPartition, dataDir: File, endOffset: Long) { + inLock(lock) { + inProgress(topicAndPartition) match { + case LogCleaningInProgress => + val checkpoint = checkpoints(dataDir) + val offsets = checkpoint.read() + ((topicAndPartition, endOffset)) + checkpoint.write(offsets) + inProgress.remove(topicAndPartition) + case LogCleaningAborted => + inProgress.put(topicAndPartition, LogCleaningPaused) + pausedCleaningCond.signalAll() + case s => + throw new IllegalStateException(("In-progress partiiton %s can't be in %s state").format(topicAndPartition, s)) + } + } + } +} diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala new file mode 100644 index 0000000000000..0b32aeeffcd9d --- /dev/null +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -0,0 +1,151 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +import java.util.Properties +import scala.collection._ +import kafka.common._ + +/** + * Configuration settings for a log + * @param segmentSize The soft maximum for the size of a segment file in the log + * @param segmentMs The soft maximum on the amount of time before a new log segment is rolled + * @param flushInterval The number of messages that can be written to the log before a flush is forced + * @param flushMs The amount of time the log can have dirty data before a flush is forced + * @param retentionSize The approximate total number of bytes this log can use + * @param retentionMs The age approximate maximum age of the last segment that is retained + * @param maxIndexSize The maximum size of an index file + * @param indexInterval The approximate number of bytes between index entries + * @param fileDeleteDelayMs The time to wait before deleting a file from the filesystem + * @param deleteRetentionMs The time to retain delete markers in the log. Only applicable for logs that are being compacted. + * @param minCleanableRatio The ratio of bytes that are available for cleaning to the bytes already cleaned + * @param dedupe Should old segments in this log be deleted or deduplicated? + */ +case class LogConfig(val segmentSize: Int = 1024*1024, + val segmentMs: Long = Long.MaxValue, + val flushInterval: Long = Long.MaxValue, + val flushMs: Long = Long.MaxValue, + val retentionSize: Long = Long.MaxValue, + val retentionMs: Long = Long.MaxValue, + val maxMessageSize: Int = Int.MaxValue, + val maxIndexSize: Int = 1024*1024, + val indexInterval: Int = 4096, + val fileDeleteDelayMs: Long = 60*1000, + val deleteRetentionMs: Long = 24 * 60 * 60 * 1000L, + val minCleanableRatio: Double = 0.5, + val dedupe: Boolean = false) { + + def toProps: Properties = { + val props = new Properties() + import LogConfig._ + props.put(SegmentBytesProp, segmentSize.toString) + props.put(SegmentMsProp, segmentMs.toString) + props.put(SegmentIndexBytesProp, maxIndexSize.toString) + props.put(FlushMessagesProp, flushInterval.toString) + props.put(FlushMsProp, flushMs.toString) + props.put(RetentionBytesProp, retentionSize.toString) + props.put(RententionMsProp, retentionMs.toString) + props.put(MaxMessageBytesProp, maxMessageSize.toString) + props.put(IndexIntervalBytesProp, indexInterval.toString) + props.put(DeleteRetentionMsProp, deleteRetentionMs.toString) + props.put(FileDeleteDelayMsProp, fileDeleteDelayMs.toString) + props.put(MinCleanableDirtyRatioProp, minCleanableRatio.toString) + props.put(CleanupPolicyProp, if(dedupe) "dedupe" else "delete") + props + } + +} + +object LogConfig { + val SegmentBytesProp = "segment.bytes" + val SegmentMsProp = "segment.ms" + val SegmentIndexBytesProp = "segment.index.bytes" + val FlushMessagesProp = "flush.messages" + val FlushMsProp = "flush.ms" + val RetentionBytesProp = "retention.bytes" + val RententionMsProp = "retention.ms" + val MaxMessageBytesProp = "max.message.bytes" + val IndexIntervalBytesProp = "index.interval.bytes" + val DeleteRetentionMsProp = "delete.retention.ms" + val FileDeleteDelayMsProp = "file.delete.delay.ms" + val MinCleanableDirtyRatioProp = "min.cleanable.dirty.ratio" + val CleanupPolicyProp = "cleanup.policy" + + val ConfigNames = Set(SegmentBytesProp, + SegmentMsProp, + SegmentIndexBytesProp, + FlushMessagesProp, + FlushMsProp, + RetentionBytesProp, + RententionMsProp, + MaxMessageBytesProp, + IndexIntervalBytesProp, + FileDeleteDelayMsProp, + DeleteRetentionMsProp, + MinCleanableDirtyRatioProp, + CleanupPolicyProp) + + + /** + * Parse the given properties instance into a LogConfig object + */ + def fromProps(props: Properties): LogConfig = { + new LogConfig(segmentSize = props.getProperty(SegmentBytesProp).toInt, + segmentMs = props.getProperty(SegmentMsProp).toLong, + maxIndexSize = props.getProperty(SegmentIndexBytesProp).toInt, + flushInterval = props.getProperty(FlushMessagesProp).toLong, + flushMs = props.getProperty(FlushMsProp).toLong, + retentionSize = props.getProperty(RetentionBytesProp).toLong, + retentionMs = props.getProperty(RententionMsProp).toLong, + maxMessageSize = props.getProperty(MaxMessageBytesProp).toInt, + indexInterval = props.getProperty(IndexIntervalBytesProp).toInt, + fileDeleteDelayMs = props.getProperty(FileDeleteDelayMsProp).toInt, + deleteRetentionMs = props.getProperty(DeleteRetentionMsProp).toLong, + minCleanableRatio = props.getProperty(MinCleanableDirtyRatioProp).toDouble, + dedupe = props.getProperty(CleanupPolicyProp).trim.toLowerCase == "dedupe") + } + + /** + * Create a log config instance using the given properties and defaults + */ + def fromProps(defaults: Properties, overrides: Properties): LogConfig = { + val props = new Properties(defaults) + props.putAll(overrides) + fromProps(props) + } + + /** + * Check that property names are valid + */ + def validateNames(props: Properties) { + import JavaConversions._ + for(name <- props.keys) + require(LogConfig.ConfigNames.contains(name), "Unknown configuration \"%s\".".format(name)) + } + + /** + * Check that the given properties contain only valid log config names, and that all values can be parsed. + */ + def validate(props: Properties) { + validateNames(props) + LogConfig.fromProps(LogConfig().toProps, props) // check that we can parse the values + } + +} + + \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 739e22aff352f..10062af1e02af 100644 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -18,12 +18,11 @@ package kafka.log import java.io._ +import java.util.concurrent.TimeUnit import kafka.utils._ import scala.collection._ -import kafka.log.Log._ import kafka.common.{TopicAndPartition, KafkaException} -import kafka.server.KafkaConfig - +import kafka.server.OffsetCheckpoint /** * The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning. @@ -36,35 +35,40 @@ import kafka.server.KafkaConfig * A background thread handles log retention by periodically truncating excess log segments. */ @threadsafe -private[kafka] class LogManager(val config: KafkaConfig, - scheduler: KafkaScheduler, - private val time: Time) extends Logging { +class LogManager(val logDirs: Array[File], + val topicConfigs: Map[String, LogConfig], + val defaultConfig: LogConfig, + val cleanerConfig: CleanerConfig, + val flushCheckMs: Long, + val flushCheckpointMs: Long, + val retentionCheckMs: Long, + scheduler: Scheduler, + private val time: Time) extends Logging { - val CleanShutdownFile = ".kafka_cleanshutdown" + val RecoveryPointCheckpointFile = "recovery-point-offset-checkpoint" val LockFile = ".lock" - val logDirs: Array[File] = config.logDirs.map(new File(_)).toArray - private val logFileSizeMap = config.logSegmentBytesPerTopicMap - private val logFlushInterval = config.logFlushIntervalMessages - private val logFlushIntervals = config.logFlushIntervalMsPerTopicMap - private val logCreationLock = new Object - private val logRetentionSizeMap = config.logRetentionBytesPerTopicMap - private val logRetentionMsMap = config.logRetentionHoursPerTopicMap.map(e => (e._1, e._2 * 60 * 60 * 1000L)) // convert hours to ms - private val logRollMsMap = config.logRollHoursPerTopicMap.map(e => (e._1, e._2 * 60 * 60 * 1000L)) - private val logRollDefaultIntervalMs = 1000L * 60 * 60 * config.logRollHours - private val logCleanupIntervalMs = 1000L * 60 * config.logCleanupIntervalMins - private val logCleanupDefaultAgeMs = 1000L * 60 * 60 * config.logRetentionHours - - this.logIdent = "[Log Manager on Broker " + config.brokerId + "] " + val InitialTaskDelayMs = 30*1000 + private val logCreationOrDeletionLock = new Object private val logs = new Pool[TopicAndPartition, Log]() - + createAndValidateLogDirs(logDirs) private var dirLocks = lockLogDirs(logDirs) + private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, RecoveryPointCheckpointFile)))).toMap loadLogs(logDirs) + private val cleaner: LogCleaner = + if(cleanerConfig.enableCleaner) + new LogCleaner(cleanerConfig, logDirs, logs, time = time) + else + null + /** - * 1. Ensure that there are no duplicates in the directory list - * 2. Create each directory if it doesn't exist - * 3. Check that each path is a readable directory + * Create and check validity of the given directories, specifically: + *
    + *
  1. Ensure that there are no duplicates in the directory list + *
  2. Create each directory if it doesn't exist + *
  3. Check that each path is a readable directory + *
*/ private def createAndValidateLogDirs(dirs: Seq[File]) { if(dirs.map(_.getCanonicalPath).toSet.size < dirs.size) @@ -95,86 +99,159 @@ private[kafka] class LogManager(val config: KafkaConfig, } /** - * Recovery and load all logs in the given data directories + * Recover and load all logs in the given data directories */ private def loadLogs(dirs: Seq[File]) { for(dir <- dirs) { - /* check if this set of logs was shut down cleanly */ - val cleanShutDownFile = new File(dir, CleanShutdownFile) - val needsRecovery = !cleanShutDownFile.exists - cleanShutDownFile.delete + val recoveryPoints = this.recoveryPointCheckpoints(dir).read /* load the logs */ val subDirs = dir.listFiles() if(subDirs != null) { + val cleanShutDownFile = new File(dir, Log.CleanShutdownFile) + if(cleanShutDownFile.exists()) + info("Found clean shutdown file. Skipping recovery for all logs in data directory '%s'".format(dir.getAbsolutePath)) for(dir <- subDirs) { - if(dir.isDirectory){ + if(dir.isDirectory) { info("Loading log '" + dir.getName + "'") - val topicPartition = parseTopicPartitionName(dir.getName) - val rollIntervalMs = logRollMsMap.get(topicPartition.topic).getOrElse(this.logRollDefaultIntervalMs) - val maxLogFileSize = logFileSizeMap.get(topicPartition.topic).getOrElse(config.logSegmentBytes) + val topicPartition = Log.parseTopicPartitionName(dir.getName) + val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig) val log = new Log(dir, - maxLogFileSize, - config.messageMaxBytes, - logFlushInterval, - rollIntervalMs, - needsRecovery, - config.logIndexSizeMaxBytes, - config.logIndexIntervalBytes, - time, - config.brokerId) + config, + recoveryPoints.getOrElse(topicPartition, 0L), + scheduler, + time) val previous = this.logs.put(topicPartition, log) if(previous != null) throw new IllegalArgumentException("Duplicate log directories found: %s, %s!".format(log.dir.getAbsolutePath, previous.dir.getAbsolutePath)) } } + cleanShutDownFile.delete() } } } /** - * Start the log flush thread + * Start the background threads to flush logs and do log cleanup */ def startup() { /* Schedule the cleanup task to delete old logs */ if(scheduler != null) { - info("Starting log cleaner every " + logCleanupIntervalMs + " ms") - scheduler.scheduleWithRate(cleanupLogs, "kafka-logcleaner-", 60 * 1000, logCleanupIntervalMs, false) - info("Starting log flusher every " + config.logFlushSchedulerIntervalMs + - " ms with the following overrides " + logFlushIntervals) - scheduler.scheduleWithRate(flushDirtyLogs, "kafka-logflusher-", - config.logFlushSchedulerIntervalMs, config.logFlushSchedulerIntervalMs, false) + info("Starting log cleanup with a period of %d ms.".format(retentionCheckMs)) + scheduler.schedule("kafka-log-retention", + cleanupLogs, + delay = InitialTaskDelayMs, + period = retentionCheckMs, + TimeUnit.MILLISECONDS) + info("Starting log flusher with a default period of %d ms.".format(flushCheckMs)) + scheduler.schedule("kafka-log-flusher", + flushDirtyLogs, + delay = InitialTaskDelayMs, + period = flushCheckMs, + TimeUnit.MILLISECONDS) + scheduler.schedule("kafka-recovery-point-checkpoint", + checkpointRecoveryPointOffsets, + delay = InitialTaskDelayMs, + period = flushCheckpointMs, + TimeUnit.MILLISECONDS) } + if(cleanerConfig.enableCleaner) + cleaner.startup() } /** - * Get the log if it exists + * Close all the logs */ - def getLog(topic: String, partition: Int): Option[Log] = { - val topicAndPartiton = TopicAndPartition(topic, partition) - val log = logs.get(topicAndPartiton) - if (log == null) - None - else - Some(log) + def shutdown() { + debug("Shutting down.") + try { + // stop the cleaner first + if(cleaner != null) + Utils.swallow(cleaner.shutdown()) + // flush the logs to ensure latest possible recovery point + allLogs.foreach(_.flush()) + // close the logs + allLogs.foreach(_.close()) + // update the last flush point + checkpointRecoveryPointOffsets() + // mark that the shutdown was clean by creating the clean shutdown marker file + logDirs.foreach(dir => Utils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile())) + } finally { + // regardless of whether the close succeeded, we need to unlock the data directories + dirLocks.foreach(_.destroy()) + } + debug("Shutdown complete.") + } + + /** + * Truncate the partition logs to the specified offsets and checkpoint the recovery point to this offset + * + * @param partitionAndOffsets Partition logs that need to be truncated + */ + def truncateTo(partitionAndOffsets: Map[TopicAndPartition, Long]) { + for ((topicAndPartition, truncateOffset) <- partitionAndOffsets) { + val log = logs.get(topicAndPartition) + // If the log does not exist, skip it + if (log != null) { + //May need to abort and pause the cleaning of the log, and resume after truncation is done. + val needToStopCleaner: Boolean = (truncateOffset < log.activeSegment.baseOffset) + if (needToStopCleaner && cleaner != null) + cleaner.abortAndPauseCleaning(topicAndPartition) + log.truncateTo(truncateOffset) + if (needToStopCleaner && cleaner != null) + cleaner.resumeCleaning(topicAndPartition) + } + } + checkpointRecoveryPointOffsets() } /** - * Create the log if it does not exist, if it exists just return it + * Delete all data in a partition and start the log at the new offset + * @param newOffset The new offset to start the log with */ - def getOrCreateLog(topic: String, partition: Int): Log = { - val topicAndPartition = TopicAndPartition(topic, partition) - logs.get(topicAndPartition) match { - case null => createLogIfNotExists(topicAndPartition) - case log: Log => log + def truncateFullyAndStartAt(topicAndPartition: TopicAndPartition, newOffset: Long) { + val log = logs.get(topicAndPartition) + // If the log does not exist, skip it + if (log != null) { + //Abort and pause the cleaning of the log, and resume after truncation is done. + if (cleaner != null) + cleaner.abortAndPauseCleaning(topicAndPartition) + log.truncateFullyAndStartAt(newOffset) + if (cleaner != null) + cleaner.resumeCleaning(topicAndPartition) } + checkpointRecoveryPointOffsets() + } + + /** + * Write out the current recovery point for all logs to a text file in the log directory + * to avoid recovering the whole log on startup. + */ + def checkpointRecoveryPointOffsets() { + val recoveryPointsByDir = this.logsByTopicPartition.groupBy(_._2.dir.getParent.toString) + for(dir <- logDirs) { + val recoveryPoints = recoveryPointsByDir.get(dir.toString) + if(recoveryPoints.isDefined) + this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint)) + } + } + + /** + * Get the log if it exists, otherwise return None + */ + def getLog(topicAndPartition: TopicAndPartition): Option[Log] = { + val log = logs.get(topicAndPartition) + if (log == null) + None + else + Some(log) } /** * Create a log for the given topic and the given partition * If the log already exists, just return a copy of the existing log */ - private def createLogIfNotExists(topicAndPartition: TopicAndPartition): Log = { - logCreationLock synchronized { + def createLog(topicAndPartition: TopicAndPartition, config: LogConfig): Log = { + logCreationOrDeletionLock synchronized { var log = logs.get(topicAndPartition) // check if the log has already been created in another thread @@ -185,24 +262,41 @@ private[kafka] class LogManager(val config: KafkaConfig, val dataDir = nextLogDir() val dir = new File(dataDir, topicAndPartition.topic + "-" + topicAndPartition.partition) dir.mkdirs() - val rollIntervalMs = logRollMsMap.get(topicAndPartition.topic).getOrElse(this.logRollDefaultIntervalMs) - val maxLogFileSize = logFileSizeMap.get(topicAndPartition.topic).getOrElse(config.logSegmentBytes) log = new Log(dir, - maxLogFileSize, - config.messageMaxBytes, - logFlushInterval, - rollIntervalMs, - needsRecovery = false, - config.logIndexSizeMaxBytes, - config.logIndexIntervalBytes, - time, - config.brokerId) - info("Created log for partition [%s,%d] in %s.".format(topicAndPartition.topic, topicAndPartition.partition, dataDir.getAbsolutePath)) + config, + recoveryPoint = 0L, + scheduler, + time) logs.put(topicAndPartition, log) + info("Created log for partition [%s,%d] in %s with properties {%s}." + .format(topicAndPartition.topic, + topicAndPartition.partition, + dataDir.getAbsolutePath, + {import JavaConversions._; config.toProps.mkString(", ")})) log } } - + + /** + * Delete a log. + */ + def deleteLog(topicAndPartition: TopicAndPartition) { + var removedLog: Log = null + logCreationOrDeletionLock synchronized { + removedLog = logs.remove(topicAndPartition) + } + if (removedLog != null) { + //We need to wait until there is no more cleaning task on the log to be deleted before actually deleting it. + if (cleaner != null) + cleaner.abortCleaning(topicAndPartition) + removedLog.delete() + info("Deleted log for partition [%s,%d] in %s." + .format(topicAndPartition.topic, + topicAndPartition.partition, + removedLog.dir.getAbsolutePath)) + } + } + /** * Choose the next directory in which to create a log. Currently this is done * by calculating the number of partitions in each directory and then choosing the @@ -223,24 +317,12 @@ private[kafka] class LogManager(val config: KafkaConfig, } } - def getOffsets(topicAndPartition: TopicAndPartition, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { - val log = getLog(topicAndPartition.topic, topicAndPartition.partition) - log match { - case Some(l) => l.getOffsetsBefore(timestamp, maxNumOffsets) - case None => getEmptyOffsets(timestamp) - } - } - /** * Runs through the log removing segments older than a certain age */ private def cleanupExpiredSegments(log: Log): Int = { val startMs = time.milliseconds - val topic = parseTopicPartitionName(log.name).topic - val logCleanupThresholdMs = logRetentionMsMap.get(topic).getOrElse(this.logCleanupDefaultAgeMs) - val toBeDeleted = log.markDeletedWhile(startMs - _.messageSet.file.lastModified > logCleanupThresholdMs) - val total = log.deleteSegments(toBeDeleted) - total + log.deleteOldSegments(startMs - _.lastModified > log.config.retentionMs) } /** @@ -248,10 +330,9 @@ private[kafka] class LogManager(val config: KafkaConfig, * is at least logRetentionSize bytes in size */ private def cleanupSegmentsToMaintainSize(log: Log): Int = { - val topic = parseTopicPartitionName(log.dir.getName).topic - val maxLogRetentionSize = logRetentionSizeMap.get(topic).getOrElse(config.logRetentionBytes) - if(maxLogRetentionSize < 0 || log.size < maxLogRetentionSize) return 0 - var diff = log.size - maxLogRetentionSize + if(log.config.retentionSize < 0 || log.size < log.config.retentionSize) + return 0 + var diff = log.size - log.config.retentionSize def shouldDelete(segment: LogSegment) = { if(diff - segment.size >= 0) { diff -= segment.size @@ -260,9 +341,7 @@ private[kafka] class LogManager(val config: KafkaConfig, false } } - val toBeDeleted = log.markDeletedWhile( shouldDelete ) - val total = log.deleteSegments(toBeDeleted) - total + log.deleteOldSegments(shouldDelete) } /** @@ -272,7 +351,7 @@ private[kafka] class LogManager(val config: KafkaConfig, debug("Beginning log cleanup...") var total = 0 val startMs = time.milliseconds - for(log <- allLogs) { + for(log <- allLogs; if !log.config.dedupe) { debug("Garbage collecting '" + log.name + "'") total += cleanupExpiredSegments(log) + cleanupSegmentsToMaintainSize(log) } @@ -280,61 +359,33 @@ private[kafka] class LogManager(val config: KafkaConfig, (time.milliseconds - startMs) / 1000 + " seconds") } - /** - * Close all the logs - */ - def shutdown() { - debug("Shutting down.") - try { - // close the logs - allLogs.foreach(_.close()) - // mark that the shutdown was clean by creating the clean shutdown marker file - logDirs.foreach(dir => Utils.swallow(new File(dir, CleanShutdownFile).createNewFile())) - } finally { - // regardless of whether the close succeeded, we need to unlock the data directories - dirLocks.foreach(_.destroy()) - } - debug("Shutdown complete.") - } - /** * Get all the partition logs */ def allLogs(): Iterable[Log] = logs.values + + /** + * Get a map of TopicAndPartition => Log + */ + def logsByTopicPartition = logs.toMap /** * Flush any log which has exceeded its flush interval and has unwritten messages. */ private def flushDirtyLogs() = { debug("Checking for dirty logs to flush...") - for (log <- allLogs) { + + for ((topicAndPartition, log) <- logs) { try { - val timeSinceLastFlush = System.currentTimeMillis - log.getLastFlushedTime - var logFlushInterval = config.logFlushIntervalMs - if(logFlushIntervals.contains(log.topicName)) - logFlushInterval = logFlushIntervals(log.topicName) - debug(log.topicName + " flush interval " + logFlushInterval + - " last flushed " + log.getLastFlushedTime + " time since last flush: " + timeSinceLastFlush) - if(timeSinceLastFlush >= logFlushInterval) + val timeSinceLastFlush = time.milliseconds - log.lastFlushTime + debug("Checking if flush is needed on " + topicAndPartition.topic + " flush interval " + log.config.flushMs + + " last flushed " + log.lastFlushTime + " time since last flush: " + timeSinceLastFlush) + if(timeSinceLastFlush >= log.config.flushMs) log.flush } catch { case e: Throwable => - error("Error flushing topic " + log.topicName, e) - e match { - case _: IOException => - fatal("Halting due to unrecoverable I/O error while flushing logs: " + e.getMessage, e) - System.exit(1) - case _ => - } + error("Error flushing topic " + topicAndPartition.topic, e) } } } - - private def parseTopicPartitionName(name: String): TopicAndPartition = { - val index = name.lastIndexOf('-') - TopicAndPartition(name.substring(0,index), name.substring(index+1).toInt) - } - - def topics(): Iterable[String] = logs.keys.map(_.topic) - } diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 6f11a8b6c8fa6..0d6926ea105a9 100644 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -19,6 +19,7 @@ import scala.math._ import java.io.File import kafka.message._ +import kafka.common._ import kafka.utils._ /** @@ -28,87 +29,100 @@ import kafka.utils._ * any previous segment. * * A segment with a base offset of [base_offset] would be stored in two files, a [base_offset].index and a [base_offset].log file. + * + * @param log The message set containing log entries + * @param index The offset index + * @param baseOffset A lower bound on the offsets in this segment + * @param indexIntervalBytes The approximate number of bytes between entries in the index + * @param time The time instance */ @nonthreadsafe -class LogSegment(val messageSet: FileMessageSet, +class LogSegment(val log: FileMessageSet, val index: OffsetIndex, - val start: Long, + val baseOffset: Long, val indexIntervalBytes: Int, - time: Time) extends Range with Logging { + time: Time) extends Logging { - var firstAppendTime: Option[Long] = - if (messageSet.sizeInBytes > 0) - Some(time.milliseconds) - else - None + var created = time.milliseconds /* the number of bytes since we last added an entry in the offset index */ - var bytesSinceLastIndexEntry = 0 - - @volatile var deleted = false + private var bytesSinceLastIndexEntry = 0 - def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int) = + def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, time: Time) = this(new FileMessageSet(file = Log.logFilename(dir, startOffset)), new OffsetIndex(file = Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), startOffset, indexIntervalBytes, - SystemTime) + time) /* Return the size in bytes of this log segment */ - def size: Long = messageSet.sizeInBytes() - - def updateFirstAppendTime() { - if (firstAppendTime.isEmpty) - firstAppendTime = Some(time.milliseconds) - } - + def size: Long = log.sizeInBytes() + /** * Append the given messages starting with the given offset. Add * an entry to the index if needed. * - * It is assumed this method is being called from within a lock + * It is assumed this method is being called from within a lock. + * + * @param offset The first offset in the message set. + * @param messages The messages to append. */ + @nonthreadsafe def append(offset: Long, messages: ByteBufferMessageSet) { if (messages.sizeInBytes > 0) { - trace("Inserting %d bytes at offset %d at position %d".format(messages.sizeInBytes, offset, messageSet.sizeInBytes())) + trace("Inserting %d bytes at offset %d at position %d".format(messages.sizeInBytes, offset, log.sizeInBytes())) // append an entry to the index (if needed) if(bytesSinceLastIndexEntry > indexIntervalBytes) { - index.append(offset, messageSet.sizeInBytes()) + index.append(offset, log.sizeInBytes()) this.bytesSinceLastIndexEntry = 0 } // append the messages - messageSet.append(messages) - updateFirstAppendTime() + log.append(messages) this.bytesSinceLastIndexEntry += messages.sizeInBytes } } /** - * Find the physical file position for the least offset >= the given offset. If no offset is found - * that meets this criteria before the end of the log, return null. + * Find the physical file position for the first message with offset >= the requested offset. + * + * The lowerBound argument is an optimization that can be used if we already know a valid starting position + * in the file higher than the greast-lower-bound from the index. + * + * @param offset The offset we want to translate + * @param startingFilePosition A lower bound on the file position from which to begin the search. This is purely an optimization and + * when omitted, the search will begin at the position in the offset index. + * + * @return The position in the log storing the message with the least offset >= the requested offset or null if no message meets this criteria. */ - private def translateOffset(offset: Long): OffsetPosition = { + @threadsafe + private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): OffsetPosition = { val mapping = index.lookup(offset) - messageSet.searchFor(offset, mapping.position) + log.searchFor(offset, max(mapping.position, startingFilePosition)) } /** - * Read a message set from this segment beginning with the first offset - * greater than or equal to the startOffset. The message set will include + * Read a message set from this segment beginning with the first offset >= startOffset. The message set will include * no more than maxSize bytes and will end before maxOffset if a maxOffset is specified. + * + * @param startOffset A lower bound on the first offset to include in the message set we read + * @param maxSize The maximum number of bytes to include in the message set we read + * @param maxOffset An optional maximum offset for the message set we read + * + * @return The message set read or null if the startOffset is larger than the largest offset in this log. */ - def read(startOffset: Long, maxSize: Int, maxOffset: Option[Long]): MessageSet = { + @threadsafe + def read(startOffset: Long, maxOffset: Option[Long], maxSize: Int): MessageSet = { if(maxSize < 0) throw new IllegalArgumentException("Invalid max size for log read (%d)".format(maxSize)) if(maxSize == 0) return MessageSet.Empty - val logSize = messageSet.sizeInBytes // this may change, need to save a consistent copy + val logSize = log.sizeInBytes // this may change, need to save a consistent copy val startPosition = translateOffset(startOffset) - // if the start position is already off the end of the log, return MessageSet.Empty + // if the start position is already off the end of the log, return null if(startPosition == null) - return MessageSet.Empty + return null // calculate the length of the message set to read based on whether or not they gave us a maxOffset val length = @@ -120,7 +134,7 @@ class LogSegment(val messageSet: FileMessageSet, // there is a max offset, translate it to a file position and use that to calculate the max read size if(offset < startOffset) throw new IllegalArgumentException("Attempt to read with a maximum offset (%d) less than the start offset (%d).".format(offset, startOffset)) - val mapping = translateOffset(offset) + val mapping = translateOffset(offset, startPosition.position) val endPosition = if(mapping == null) logSize // the max offset is off the end of the log, use the end of the file @@ -129,45 +143,113 @@ class LogSegment(val messageSet: FileMessageSet, min(endPosition - startPosition.position, maxSize) } } - messageSet.read(startPosition.position, length) + log.read(startPosition.position, length) + } + + /** + * Run recovery on the given segment. This will rebuild the index from the log file and lop off any invalid bytes from the end of the log and index. + * + * @param maxMessageSize A bound the memory allocation in the case of a corrupt message size--we will assume any message larger than this + * is corrupt. + * + * @return The number of bytes truncated from the log + */ + @nonthreadsafe + def recover(maxMessageSize: Int): Int = { + index.truncate() + index.resize(index.maxIndexSize) + var validBytes = 0 + var lastIndexEntry = 0 + val iter = log.iterator(maxMessageSize) + try { + while(iter.hasNext) { + val entry = iter.next + entry.message.ensureValid() + if(validBytes - lastIndexEntry > indexIntervalBytes) { + // we need to decompress the message, if required, to get the offset of the first uncompressed message + val startOffset = + entry.message.compressionCodec match { + case NoCompressionCodec => + entry.offset + case _ => + ByteBufferMessageSet.decompress(entry.message).head.offset + } + index.append(startOffset, validBytes) + lastIndexEntry = validBytes + } + validBytes += MessageSet.entrySize(entry.message) + } + } catch { + case e: InvalidMessageException => + logger.warn("Found invalid messages in log segment %s at byte offset %d: %s.".format(log.file.getAbsolutePath, validBytes, e.getMessage)) + } + val truncated = log.sizeInBytes - validBytes + log.truncateTo(validBytes) + index.trimToValidSize() + truncated } - override def toString() = "LogSegment(start=" + start + ", size=" + size + ")" + override def toString() = "LogSegment(baseOffset=" + baseOffset + ", size=" + size + ")" /** - * Truncate off all index and log entries with offsets greater than or equal to the current offset. + * Truncate off all index and log entries with offsets >= the given offset. + * If the given offset is larger than the largest message in this segment, do nothing. + * @param offset The offset to truncate to + * @return The number of log bytes truncated */ - def truncateTo(offset: Long) { + @nonthreadsafe + def truncateTo(offset: Long): Int = { val mapping = translateOffset(offset) if(mapping == null) - return + return 0 index.truncateTo(offset) // after truncation, reset and allocate more space for the (new currently active) index index.resize(index.maxIndexSize) - messageSet.truncateTo(mapping.position) - if (messageSet.sizeInBytes == 0) - firstAppendTime = None + val bytesTruncated = log.truncateTo(mapping.position) + if(log.sizeInBytes == 0) + created = time.milliseconds bytesSinceLastIndexEntry = 0 + bytesTruncated } /** * Calculate the offset that would be used for the next message to be append to this segment. * Note that this is expensive. */ + @threadsafe def nextOffset(): Long = { - val ms = read(index.lastOffset, messageSet.sizeInBytes, None) - ms.lastOption match { - case None => start - case Some(last) => last.nextOffset + val ms = read(index.lastOffset, None, log.sizeInBytes) + if(ms == null) { + baseOffset + } else { + ms.lastOption match { + case None => baseOffset + case Some(last) => last.nextOffset + } } } /** * Flush this log segment to disk */ + @threadsafe def flush() { - messageSet.flush() - index.flush() + LogFlushStats.logFlushTimer.time { + log.flush() + index.flush() + } + } + + /** + * Change the suffix for the index and log file for this log segment + */ + def changeFileSuffixes(oldSuffix: String, newSuffix: String) { + val logRenamed = log.renameTo(new File(Utils.replaceSuffix(log.file.getPath, oldSuffix, newSuffix))) + if(!logRenamed) + throw new KafkaStorageException("Failed to change the log file suffix from %s to %s for log segment %d".format(oldSuffix, newSuffix, baseOffset)) + val indexRenamed = index.renameTo(new File(Utils.replaceSuffix(index.file.getPath, oldSuffix, newSuffix))) + if(!indexRenamed) + throw new KafkaStorageException("Failed to change the index file suffix from %s to %s for log segment %d".format(oldSuffix, newSuffix, baseOffset)) } /** @@ -175,7 +257,32 @@ class LogSegment(val messageSet: FileMessageSet, */ def close() { Utils.swallow(index.close) - Utils.swallow(messageSet.close) + Utils.swallow(log.close) } + /** + * Delete this log segment from the filesystem. + * @throws KafkaStorageException if the delete fails. + */ + def delete() { + val deletedLog = log.delete() + val deletedIndex = index.delete() + if(!deletedLog && log.file.exists) + throw new KafkaStorageException("Delete of log " + log.file.getName + " failed.") + if(!deletedIndex && index.file.exists) + throw new KafkaStorageException("Delete of index " + index.file.getName + " failed.") + } + + /** + * The last modified time of this log segment as a unix time stamp + */ + def lastModified = log.file.lastModified + + /** + * Change the last modified time for this log segment + */ + def lastModified_=(ms: Long) = { + log.file.setLastModified(ms) + index.file.setLastModified(ms) + } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index 9de3d31d7c8e9..8a62dfad9bc48 100644 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -21,8 +21,10 @@ import scala.math._ import java.io._ import java.nio._ import java.nio.channels._ +import java.util.concurrent.locks._ import java.util.concurrent.atomic._ import kafka.utils._ +import kafka.utils.Utils.inLock import kafka.common.InvalidOffsetException /** @@ -50,9 +52,11 @@ import kafka.common.InvalidOffsetException * All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal * storage format. */ -class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = -1) extends Logging { +class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSize: Int = -1) extends Logging { - /* the memory mapping */ + private val lock = new ReentrantLock + + /* initialize the memory mapping for this index */ private var mmap: MappedByteBuffer = { val newlyCreated = file.createNewFile() @@ -65,12 +69,8 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = raf.setLength(roundToExactMultiple(maxIndexSize, 8)) } - val len = raf.length() - if(len < 0 || len % 8 != 0) - throw new IllegalStateException("Index file " + file.getName + " is corrupt, found " + len + - " bytes which is not positive or not a multiple of 8.") - /* memory-map the file */ + val len = raf.length() val idx = raf.getChannel.map(FileChannel.MapMode.READ_WRITE, 0, len) /* set the position in the index for the next entry */ @@ -84,50 +84,63 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = Utils.swallow(raf.close()) } } - - /* the number of entries in the index */ + + /* the number of eight-byte entries currently in the index */ private var size = new AtomicInteger(mmap.position / 8) + /** + * The maximum number of eight-byte entries this index can hold + */ + @volatile + var maxEntries = mmap.limit / 8 + /* the last offset in the index */ - var lastOffset = readLastOffset() + var lastOffset = readLastEntry.offset debug("Loaded index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d, file position = %d" .format(file.getAbsolutePath, maxEntries, maxIndexSize, entries(), lastOffset, mmap.position)) - /* the maximum number of entries this index can hold */ - def maxEntries = mmap.limit / 8 - /** - * The last offset written to the index + * The last entry in the index */ - private def readLastOffset(): Long = { - val offset = + def readLastEntry(): OffsetPosition = { + inLock(lock) { size.get match { - case 0 => 0 - case s => relativeOffset(this.mmap, s-1) + case 0 => OffsetPosition(baseOffset, 0) + case s => OffsetPosition(baseOffset + relativeOffset(this.mmap, s-1), physical(this.mmap, s-1)) } - baseOffset + offset + } } /** * Find the largest offset less than or equal to the given targetOffset * and return a pair holding this offset and it's corresponding physical file position. + * + * @param targetOffset The offset to look up. + * + * @return The offset found and the corresponding file position for this offset. * If the target offset is smaller than the least entry in the index (or the index is empty), * the pair (baseOffset, 0) is returned. */ def lookup(targetOffset: Long): OffsetPosition = { - val idx = mmap.duplicate - val slot = indexSlotFor(idx, targetOffset) - if(slot == -1) - OffsetPosition(baseOffset, 0) - else - OffsetPosition(baseOffset + relativeOffset(idx, slot), physical(idx, slot)) + maybeLock(lock) { + val idx = mmap.duplicate + val slot = indexSlotFor(idx, targetOffset) + if(slot == -1) + OffsetPosition(baseOffset, 0) + else + OffsetPosition(baseOffset + relativeOffset(idx, slot), physical(idx, slot)) + } } /** * Find the slot in which the largest offset less than or equal to the given * target offset is stored. - * Return -1 if the least entry in the index is larger than the target offset or the index is empty + * + * @param idx The index buffer + * @param targetOffset The offset to look for + * + * @return The slot found or -1 if the least entry in the index is larger than the target offset or the index is empty */ private def indexSlotFor(idx: ByteBuffer, targetOffset: Long): Int = { // we only store the difference from the base offset so calculate that @@ -160,24 +173,28 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = /* return the nth offset relative to the base offset */ private def relativeOffset(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * 8) - /* return the nth physical offset */ + /* return the nth physical position */ private def physical(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * 8 + 4) /** * Get the nth offset mapping from the index + * @param n The entry number in the index + * @return The offset/position pair at that entry */ def entry(n: Int): OffsetPosition = { - if(n >= entries) - throw new IllegalArgumentException("Attempt to fetch the %dth entry from an index of size %d.".format(n, entries)) - val idx = mmap.duplicate - OffsetPosition(relativeOffset(idx, n), physical(idx, n)) + maybeLock(lock) { + if(n >= entries) + throw new IllegalArgumentException("Attempt to fetch the %dth entry from an index of size %d.".format(n, entries)) + val idx = mmap.duplicate + OffsetPosition(relativeOffset(idx, n), physical(idx, n)) + } } /** - * Append entry for the given offset/location pair to the index. This entry must have a larger offset than all subsequent entries. + * Append an entry for the given offset/location pair to the index. This entry must have a larger offset than all subsequent entries. */ def append(offset: Long, position: Int) { - this synchronized { + inLock(lock) { require(!isFull, "Attempt to append to a full index (size = " + size + ").") if (size.get == 0 || offset > lastOffset) { debug("Adding index entry %d => %d to %s.".format(offset, position, file.getName)) @@ -186,10 +203,9 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = this.size.incrementAndGet() this.lastOffset = offset require(entries * 8 == mmap.position, entries + " entries but file position in index is " + mmap.position + ".") - } - else { + } else { throw new InvalidOffsetException("Attempt to append an offset (%d) to position %d no larger than the last offset appended (%d) to %s." - .format(offset, entries, lastOffset, file.getName)) + .format(offset, entries, lastOffset, file.getAbsolutePath)) } } } @@ -200,7 +216,7 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = def isFull: Boolean = entries >= this.maxEntries /** - * Truncate the entire index + * Truncate the entire index, deleting all entries */ def truncate() = truncateToEntries(0) @@ -209,7 +225,7 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = * Truncating to an offset larger than the largest in the index has no effect. */ def truncateTo(offset: Long) { - this synchronized { + inLock(lock) { val idx = mmap.duplicate val slot = indexSlotFor(idx, offset) @@ -233,9 +249,11 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = * Truncates index to a known number of entries. */ private def truncateToEntries(entries: Int) { - this.size.set(entries) - mmap.position(this.size.get * 8) - this.lastOffset = readLastOffset + inLock(lock) { + this.size.set(entries) + mmap.position(this.size.get * 8) + this.lastOffset = readLastEntry.offset + } } /** @@ -243,7 +261,7 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = * the file. */ def trimToValidSize() { - this synchronized { + inLock(lock) { resize(entries * 8) } } @@ -255,14 +273,18 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = * we want to reset the index size to maximum index size to avoid rolling new segment. */ def resize(newSize: Int) { - this synchronized { - flush() + inLock(lock) { val raf = new RandomAccessFile(file, "rws") val roundedNewSize = roundToExactMultiple(newSize, 8) + val position = this.mmap.position + + /* Windows won't let us modify the file length while the file is mmapped :-( */ + if(Os.isWindows) + forceUnmap(this.mmap) try { raf.setLength(roundedNewSize) - val position = this.mmap.position this.mmap = raf.getChannel().map(FileChannel.MapMode.READ_WRITE, 0, roundedNewSize) + this.maxEntries = this.mmap.limit / 8 this.mmap.position(position) } finally { Utils.swallow(raf.close()) @@ -270,11 +292,23 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = } } + /** + * Forcefully free the buffer's mmap. We do this only on windows. + */ + private def forceUnmap(m: MappedByteBuffer) { + try { + if(m.isInstanceOf[sun.nio.ch.DirectBuffer]) + (m.asInstanceOf[sun.nio.ch.DirectBuffer]).cleaner().clean() + } catch { + case t: Throwable => warn("Error when freeing index buffer", t) + } + } + /** * Flush the data in the index to disk */ def flush() { - this synchronized { + inLock(lock) { mmap.force() } } @@ -290,14 +324,59 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int = /** The number of entries in this index */ def entries() = size.get + /** + * The number of bytes actually used by this index + */ + def sizeInBytes() = 8 * entries + /** Close the index */ def close() { trimToValidSize() } + /** + * Rename the file that backs this offset index + * @return true iff the rename was successful + */ + def renameTo(f: File): Boolean = { + val success = this.file.renameTo(f) + this.file = f + success + } + + /** + * Do a basic sanity check on this index to detect obvious problems + * @throw IllegalArgumentException if any problems are found + */ + def sanityCheck() { + require(entries == 0 || lastOffset > baseOffset, + "Corrupt index found, index file (%s) has non-zero size but the last offset is %d and the base offset is %d" + .format(file.getAbsolutePath, lastOffset, baseOffset)) + val len = file.length() + require(len % 8 == 0, + "Index file " + file.getName + " is corrupt, found " + len + + " bytes which is not positive or not a multiple of 8.") + } + /** * Round a number to the greatest exact multiple of the given factor less than the given number. * E.g. roundToExactMultiple(67, 8) == 64 */ private def roundToExactMultiple(number: Int, factor: Int) = factor * (number / factor) + + /** + * Execute the given function in a lock only if we are running on windows. We do this + * because Windows won't let us resize a file while it is mmapped. As a result we have to force unmap it + * and this requires synchronizing reads. + */ + private def maybeLock[T](lock: Lock)(fun: => T): T = { + if(Os.isWindows) + lock.lock() + try { + return fun + } finally { + if(Os.isWindows) + lock.unlock() + } + } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/OffsetMap.scala b/core/src/main/scala/kafka/log/OffsetMap.scala new file mode 100644 index 0000000000000..42cdfbb6100b5 --- /dev/null +++ b/core/src/main/scala/kafka/log/OffsetMap.scala @@ -0,0 +1,179 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +import java.util.Arrays +import java.security.MessageDigest +import java.nio.ByteBuffer +import kafka.utils._ + +trait OffsetMap { + def slots: Int + def put(key: ByteBuffer, offset: Long) + def get(key: ByteBuffer): Long + def clear() + def size: Int + def utilization: Double = size.toDouble / slots +} + +/** + * An hash table used for deduplicating the log. This hash table uses a cryptographicly secure hash of the key as a proxy for the key + * for comparisons and to save space on object overhead. Collisions are resolved by probing. This hash table does not support deletes. + * @param memory The amount of memory this map can use + * @param hashAlgorithm The hash algorithm instance to use: MD2, MD5, SHA-1, SHA-256, SHA-384, SHA-512 + */ +@nonthreadsafe +class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extends OffsetMap { + private val bytes = ByteBuffer.allocate(memory) + + /* the hash algorithm instance to use, defualt is MD5 */ + private val digest = MessageDigest.getInstance(hashAlgorithm) + + /* the number of bytes for this hash algorithm */ + private val hashSize = digest.getDigestLength + + /* create some hash buffers to avoid reallocating each time */ + private val hash1 = new Array[Byte](hashSize) + private val hash2 = new Array[Byte](hashSize) + + /* number of entries put into the map */ + private var entries = 0 + + /* number of lookups on the map */ + private var lookups = 0L + + /* the number of probes for all lookups */ + private var probes = 0L + + /** + * The number of bytes of space each entry uses (the number of bytes in the hash plus an 8 byte offset) + */ + val bytesPerEntry = hashSize + 8 + + /** + * The maximum number of entries this map can contain + */ + val slots: Int = (memory / bytesPerEntry).toInt + + /** + * Associate this offset to the given key. + * @param key The key + * @param offset The offset + */ + override def put(key: ByteBuffer, offset: Long) { + require(entries < slots, "Attempt to add a new entry to a full offset map.") + lookups += 1 + hashInto(key, hash1) + // probe until we find the first empty slot + var attempt = 0 + var pos = positionOf(hash1, attempt) + while(!isEmpty(pos)) { + bytes.position(pos) + bytes.get(hash2) + if(Arrays.equals(hash1, hash2)) { + // we found an existing entry, overwrite it and return (size does not change) + bytes.putLong(offset) + return + } + attempt += 1 + pos = positionOf(hash1, attempt) + } + // found an empty slot, update it--size grows by 1 + bytes.position(pos) + bytes.put(hash1) + bytes.putLong(offset) + entries += 1 + } + + /** + * Check that there is no entry at the given position + */ + private def isEmpty(position: Int): Boolean = + bytes.getLong(position) == 0 && bytes.getLong(position + 8) == 0 && bytes.getLong(position + 16) == 0 + + /** + * Get the offset associated with this key. + * @param key The key + * @return The offset associated with this key or -1 if the key is not found + */ + override def get(key: ByteBuffer): Long = { + lookups += 1 + hashInto(key, hash1) + // search for the hash of this key by repeated probing until we find the hash we are looking for or we find an empty slot + var attempt = 0 + var pos = 0 + do { + pos = positionOf(hash1, attempt) + bytes.position(pos) + if(isEmpty(pos)) + return -1L + bytes.get(hash2) + attempt += 1 + } while(!Arrays.equals(hash1, hash2)) + bytes.getLong() + } + + /** + * Change the salt used for key hashing making all existing keys unfindable. + * Doesn't actually zero out the array. + */ + override def clear() { + this.entries = 0 + this.lookups = 0L + this.probes = 0L + Arrays.fill(bytes.array, bytes.arrayOffset, bytes.arrayOffset + bytes.limit, 0.toByte) + } + + /** + * The number of entries put into the map (note that not all may remain) + */ + override def size: Int = entries + + /** + * The rate of collisions in the lookups + */ + def collisionRate: Double = + (this.probes - this.lookups) / this.lookups.toDouble + + /** + * Calculate the ith probe position. We first try reading successive integers from the hash itself + * then if all of those fail we degrade to linear probing. + * @param hash The hash of the key to find the position for + * @param attempt The ith probe + * @return The byte offset in the buffer at which the ith probing for the given hash would reside + */ + private def positionOf(hash: Array[Byte], attempt: Int): Int = { + val probe = Utils.readInt(hash, math.min(attempt, hashSize - 4)) + math.max(0, attempt - hashSize + 4) + val slot = Utils.abs(probe) % slots + this.probes += 1 + slot * bytesPerEntry + } + + /** + * The offset at which we have stored the given key + * @param key The key to hash + * @param buffer The buffer to store the hash into + */ + private def hashInto(key: ByteBuffer, buffer: Array[Byte]) { + key.mark() + digest.update(key) + key.reset() + digest.digest(buffer, 0, hashSize) + } + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/log/SegmentList.scala b/core/src/main/scala/kafka/log/SegmentList.scala deleted file mode 100644 index 5c7b0bd351ca5..0000000000000 --- a/core/src/main/scala/kafka/log/SegmentList.scala +++ /dev/null @@ -1,100 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.log - -import java.util.concurrent.atomic._ -import reflect._ -import scala.math._ -import kafka.common.KafkaException - -private[log] object SegmentList { - val MaxAttempts: Int = 20 -} - -/** - * A copy-on-write list implementation that provides consistent views. The view() method - * provides an immutable sequence representing a consistent state of the list. The user can do - * iterative operations on this sequence such as binary search without locking all access to the list. - * Even if the range of the underlying list changes no change will be made to the view - */ -private[log] class SegmentList[T](seq: Seq[T])(implicit m: ClassManifest[T]) { - - val contents: AtomicReference[Array[T]] = new AtomicReference(seq.toArray) - - /** - * Append the given items to the end of the list - */ - def append(ts: T*)(implicit m: ClassManifest[T]) { - val curr = contents.get() - val updated = new Array[T](curr.length + ts.length) - Array.copy(curr, 0, updated, 0, curr.length) - for(i <- 0 until ts.length) - updated(curr.length + i) = ts(i) - contents.set(updated) - } - - - /** - * Delete the first n items from the list - */ - def trunc(newStart: Int): Seq[T] = { - if(newStart < 0) - throw new KafkaException("Starting index must be positive."); - var deleted: Array[T] = null - val curr = contents.get() - if (curr.length > 0) { - val newLength = max(curr.length - newStart, 0) - val updated = new Array[T](newLength) - Array.copy(curr, min(newStart, curr.length - 1), updated, 0, newLength) - contents.set(updated) - deleted = new Array[T](newStart) - Array.copy(curr, 0, deleted, 0, curr.length - newLength) - } - deleted - } - - /** - * Delete the items from position (newEnd + 1) until end of list - */ - def truncLast(newEnd: Int): Seq[T] = { - if (newEnd < 0 || newEnd >= contents.get().length) - throw new KafkaException("Attempt to truncate segment list of length %d to %d.".format(contents.get().size, newEnd)); - var deleted: Array[T] = null - val curr = contents.get() - if (curr.length > 0) { - val newLength = newEnd + 1 - val updated = new Array[T](newLength) - Array.copy(curr, 0, updated, 0, newLength) - contents.set(updated) - deleted = new Array[T](curr.length - newLength) - Array.copy(curr, min(newEnd + 1, curr.length - 1), deleted, 0, curr.length - newLength) - } - deleted - } - - /** - * Get a consistent view of the sequence - */ - def view: Array[T] = contents.get() - - /** - * Nicer toString method - */ - override def toString(): String = "SegmentList(%s)".format(view.mkString(", ")) - -} diff --git a/core/src/main/scala/kafka/log/package.html b/core/src/main/scala/kafka/log/package.html index 0880be724dec4..d8521a1abbc90 100644 --- a/core/src/main/scala/kafka/log/package.html +++ b/core/src/main/scala/kafka/log/package.html @@ -1 +1,6 @@ -The log management system for Kafka. \ No newline at end of file +The log management system for Kafka. + +The entry point for this system is LogManager. LogManager is responsible for holding all the logs, and handing them out by topic/partition. It also handles the enforcement of the +flush policy and retention policies. + +The Log itself is made up of log segments. A log is a FileMessageSet that contains the data and an OffsetIndex that supports reads by offset on the log. \ No newline at end of file diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 078ebb4b470f7..73401c5ff34d0 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -77,7 +77,7 @@ object ByteBufferMessageSet { new ByteBufferMessageSet(outputBuffer) } - private def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) { + private[kafka] def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) { buffer.putLong(offset) buffer.putInt(message.size) buffer.put(message.buffer) diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 9b6eece16cb83..52c082f5213ba 100644 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -102,7 +102,9 @@ class Message(val buffer: ByteBuffer) { Message.KeySizeLength + (if(key == null) 0 else key.length) + Message.ValueSizeLength + - (if(payloadSize >= 0) payloadSize else bytes.length - payloadOffset))) + (if(bytes == null) 0 + else if(payloadSize >= 0) payloadSize + else bytes.length - payloadOffset))) // skip crc, we will fill that in at the end buffer.position(MagicOffset) buffer.put(CurrentMagicValue) @@ -116,9 +118,12 @@ class Message(val buffer: ByteBuffer) { buffer.putInt(key.length) buffer.put(key, 0, key.length) } - val size = if(payloadSize >= 0) payloadSize else bytes.length - payloadOffset + val size = if(bytes == null) -1 + else if(payloadSize >= 0) payloadSize + else bytes.length - payloadOffset buffer.putInt(size) - buffer.put(bytes, payloadOffset, size) + if(bytes != null) + buffer.put(bytes, payloadOffset, size) buffer.rewind() // now compute the checksum and fill it in @@ -186,6 +191,11 @@ class Message(val buffer: ByteBuffer) { */ def payloadSize: Int = buffer.getInt(payloadSizeOffset) + /** + * Is the payload of this message null + */ + def isNull(): Boolean = payloadSize < 0 + /** * The magic version of this message */ diff --git a/core/src/main/scala/kafka/message/MessageAndMetadata.scala b/core/src/main/scala/kafka/message/MessageAndMetadata.scala index 20c0e7004282c..d693abc620aef 100644 --- a/core/src/main/scala/kafka/message/MessageAndMetadata.scala +++ b/core/src/main/scala/kafka/message/MessageAndMetadata.scala @@ -17,5 +17,18 @@ package kafka.message -case class MessageAndMetadata[K, V](key: K, message: V, topic: String, partition: Int, offset: Long) +import kafka.serializer.Decoder +import kafka.utils.Utils + +case class MessageAndMetadata[K, V](topic: String, partition: Int, + private val rawMessage: Message, offset: Long, + keyDecoder: Decoder[K], valueDecoder: Decoder[V]) { + + /** + * Return the decoded message key and payload + */ + def key(): K = if(rawMessage.key == null) null.asInstanceOf[K] else keyDecoder.fromBytes(Utils.readBytes(rawMessage.key)) + + def message(): V = if(rawMessage.isNull) null.asInstanceOf[V] else valueDecoder.fromBytes(Utils.readBytes(rawMessage.payload)) +} diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala index 7ef92ab3a1804..a1b5c63b4d9de 100644 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ b/core/src/main/scala/kafka/message/MessageSet.scala @@ -92,15 +92,23 @@ abstract class MessageSet extends Iterable[MessageAndOffset] { } /** - * Print this message set's contents + * Print this message set's contents. If the message set has more than 100 messages, just + * print the first 100. */ override def toString: String = { val builder = new StringBuilder() builder.append(getClass.getSimpleName + "(") - for(message <- this) { + val iter = this.iterator + var i = 0 + while(iter.hasNext && i < 100) { + val message = iter.next builder.append(message) - builder.append(", ") + if(iter.hasNext) + builder.append(", ") + i += 1 } + if(iter.hasNext) + builder.append("...") builder.append(")") builder.toString } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 77d7ec034d853..a6ec970d72fc8 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -42,9 +42,10 @@ object RequestChannel extends Logging { } case class Request(processor: Int, requestKey: Any, private var buffer: ByteBuffer, startTimeMs: Long, remoteAddress: SocketAddress = new InetSocketAddress(0)) { - @volatile var dequeueTimeMs = -1L + @volatile var requestDequeueTimeMs = -1L @volatile var apiLocalCompleteTimeMs = -1L @volatile var responseCompleteTimeMs = -1L + @volatile var responseDequeueTimeMs = -1L val requestId = buffer.getShort() val requestObj: RequestOrResponse = RequestKeys.deserializerForKey(requestId)(buffer) buffer = null @@ -57,10 +58,11 @@ object RequestChannel extends Logging { // processing time is really small. In this case, use responseCompleteTimeMs as apiLocalCompleteTimeMs. if (apiLocalCompleteTimeMs < 0) apiLocalCompleteTimeMs = responseCompleteTimeMs - val queueTime = (dequeueTimeMs - startTimeMs).max(0L) - val apiLocalTime = (apiLocalCompleteTimeMs - dequeueTimeMs).max(0L) + val requestQueueTime = (requestDequeueTimeMs - startTimeMs).max(0L) + val apiLocalTime = (apiLocalCompleteTimeMs - requestDequeueTimeMs).max(0L) val apiRemoteTime = (responseCompleteTimeMs - apiLocalCompleteTimeMs).max(0L) - val responseSendTime = (endTimeMs - responseCompleteTimeMs).max(0L) + val responseQueueTime = (responseDequeueTimeMs - responseCompleteTimeMs).max(0L) + val responseSendTime = (endTimeMs - responseDequeueTimeMs).max(0L) val totalTime = endTimeMs - startTimeMs var metricsList = List(RequestMetrics.metricsMap(RequestKeys.nameForKey(requestId))) if (requestId == RequestKeys.FetchKey) { @@ -72,15 +74,20 @@ object RequestChannel extends Logging { } metricsList.foreach{ m => m.requestRate.mark() - m.queueTimeHist.update(queueTime) + m.requestQueueTimeHist.update(requestQueueTime) m.localTimeHist.update(apiLocalTime) m.remoteTimeHist.update(apiRemoteTime) + m.responseQueueTimeHist.update(responseQueueTime) m.responseSendTimeHist.update(responseSendTime) m.totalTimeHist.update(totalTime) } if(requestLogger.isTraceEnabled) - requestLogger.trace("Completed request:%s from client %s;totalTime:%d,queueTime:%d,localTime:%d,remoteTime:%d,sendTime:%d" - .format(requestObj, remoteAddress, totalTime, queueTime, apiLocalTime, apiRemoteTime, responseSendTime)) + requestLogger.trace("Completed request:%s from client %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" + .format(requestObj.describe(true), remoteAddress, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) + else if(requestLogger.isDebugEnabled) { + requestLogger.debug("Completed request:%s from client %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" + .format(requestObj.describe(false), remoteAddress, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) + } } } @@ -154,8 +161,12 @@ class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMe requestQueue.take() /** Get a response for the given processor if there is one */ - def receiveResponse(processor: Int): RequestChannel.Response = - responseQueues(processor).poll() + def receiveResponse(processor: Int): RequestChannel.Response = { + val response = responseQueues(processor).poll() + if (response != null) + response.request.responseDequeueTimeMs = SystemTime.milliseconds + response + } def addResponseListener(onResponse: Int => Unit) { responseListeners ::= onResponse @@ -177,11 +188,13 @@ object RequestMetrics { class RequestMetrics(name: String) extends KafkaMetricsGroup { val requestRate = newMeter(name + "-RequestsPerSec", "requests", TimeUnit.SECONDS) // time a request spent in a request queue - val queueTimeHist = newHistogram(name + "-QueueTimeMs") + val requestQueueTimeHist = newHistogram(name + "-RequestQueueTimeMs") // time a request takes to be processed at the local broker val localTimeHist = newHistogram(name + "-LocalTimeMs") // time a request takes to wait on remote brokers (only relevant to fetch and produce requests) val remoteTimeHist = newHistogram(name + "-RemoteTimeMs") + // time a response spent in a response queue + val responseQueueTimeHist = newHistogram(name + "-ResponseQueueTimeMs") // time to send the response to the requester val responseSendTimeHist = newHistogram(name + "-ResponseSendTimeMs") val totalTimeHist = newHistogram(name + "-TotalTimeMs") diff --git a/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala b/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala index 752a4fc0707aa..988e4374d8c9b 100644 --- a/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala +++ b/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala @@ -20,8 +20,8 @@ package kafka.producer import kafka.utils._ -private class ByteArrayPartitioner(props: VerifiableProperties = null) extends Partitioner[Array[Byte]] { - def partition(key: Array[Byte], numPartitions: Int): Int = { - Utils.abs(java.util.Arrays.hashCode(key)) % numPartitions +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 00cb2e8d2b281..28de57304db6a 100644 --- a/core/src/main/scala/kafka/producer/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/producer/ConsoleProducer.scala @@ -68,7 +68,7 @@ object ConsoleProducer { .withRequiredArg .describedAs("queue enqueuetimeout ms") .ofType(classOf[java.lang.Long]) - .defaultsTo(0) + .defaultsTo(Int.MaxValue) val requestRequiredAcksOpt = parser.accepts("request-required-acks", "The required acks of the producer requests") .withRequiredArg .describedAs("request required acks") diff --git a/core/src/main/scala/kafka/producer/DefaultPartitioner.scala b/core/src/main/scala/kafka/producer/DefaultPartitioner.scala index 9bffeb65369aa..3afb22eeb4e3b 100644 --- a/core/src/main/scala/kafka/producer/DefaultPartitioner.scala +++ b/core/src/main/scala/kafka/producer/DefaultPartitioner.scala @@ -20,10 +20,10 @@ package kafka.producer import kafka.utils._ -private class DefaultPartitioner[T](props: VerifiableProperties = null) extends Partitioner[T] { +class DefaultPartitioner(props: VerifiableProperties = null) extends Partitioner { private val random = new java.util.Random - def partition(key: T, numPartitions: Int): Int = { + def partition(key: Any, numPartitions: Int): Int = { Utils.abs(key.hashCode) % numPartitions } } diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index 88ae78425c8ee..42239b27ab267 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -87,15 +87,19 @@ class KafkaLog4jAppender extends AppenderSkeleton with Logging { } override def append(event: LoggingEvent) { - val message : String = if( this.layout == null) { - event.getRenderedMessage - } - else this.layout.format(event) + val message = subAppend(event) LogLog.debug("[" + new Date(event.getTimeStamp).toString + "]" + message) val messageData = new KeyedMessage[String, String](topic, message) producer.send(messageData); } + def subAppend(event: LoggingEvent): String = { + if(this.layout == null) + event.getRenderedMessage + else + this.layout.format(event) + } + override def close() { if(!this.closed) { this.closed = true diff --git a/core/src/main/scala/kafka/producer/KeyedMessage.scala b/core/src/main/scala/kafka/producer/KeyedMessage.scala index b13c4ecd58fbe..388bc9bbd9a69 100644 --- a/core/src/main/scala/kafka/producer/KeyedMessage.scala +++ b/core/src/main/scala/kafka/producer/KeyedMessage.scala @@ -18,13 +18,25 @@ package kafka.producer /** - * A topic, key, and value + * A topic, key, and value. + * If a partition key is provided it will override the key for the purpose of partitioning but will not be stored. */ -case class KeyedMessage[K, V](val topic: String, val key: K, val message: V) { +case class KeyedMessage[K, V](val topic: String, val key: K, val partKey: Any, val message: V) { if(topic == null) throw new IllegalArgumentException("Topic cannot be null.") - def this(topic: String, message: V) = this(topic, null.asInstanceOf[K], message) + def this(topic: String, message: V) = this(topic, null.asInstanceOf[K], null, message) + + def this(topic: String, key: K, message: V) = this(topic, key, key, message) + + def partitionKey = { + if(partKey != null) + partKey + else if(hasKey) + key + else + null + } def hasKey = key != null } \ No newline at end of file diff --git a/core/src/main/scala/kafka/producer/Partitioner.scala b/core/src/main/scala/kafka/producer/Partitioner.scala index 9ee61c770fa88..efe6d6df87b0e 100644 --- a/core/src/main/scala/kafka/producer/Partitioner.scala +++ b/core/src/main/scala/kafka/producer/Partitioner.scala @@ -23,11 +23,11 @@ package kafka.producer * Implementations will be constructed via reflection and are required to have a constructor that takes a single * VerifiableProperties instance--this allows passing configuration properties into the partitioner implementation. */ -trait Partitioner[T] { +trait Partitioner { /** * Uses the key to calculate a partition bucket id for routing * the data to the appropriate broker partition * @return an integer between 0 and numPartitions-1 */ - def partition(key: T, numPartitions: Int): Int + 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 f5829198ebe75..4798481d573bb 100644 --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -57,7 +57,7 @@ class Producer[K,V](val config: ProducerConfig, def this(config: ProducerConfig) = this(config, new DefaultEventHandler[K,V](config, - Utils.createObject[Partitioner[K]](config.partitionerClass, config.props), + Utils.createObject[Partitioner](config.partitionerClass, config.props), Utils.createObject[Encoder[V]](config.serializerClass, config.props), Utils.createObject[Encoder[K]](config.keySerializerClass, config.props), new ProducerPool(config))) diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index 419156eb143fb..041cfa59c18fa 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -120,11 +120,6 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { } } - private def reconnect() { - disconnect() - connect() - } - /** * Disconnect from current channel, closing connection. * Side effect: channel field is set to null on successful disconnect diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index eba375b776f19..d8ac915de31a2 100644 --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -29,7 +29,7 @@ import java.util.concurrent.atomic._ import kafka.api.{TopicMetadata, ProducerRequest} class DefaultEventHandler[K,V](config: ProducerConfig, - private val partitioner: Partitioner[K], + private val partitioner: Partitioner, private val encoder: Encoder[V], private val keyEncoder: Encoder[K], private val producerPool: ProducerPool, @@ -125,9 +125,9 @@ class DefaultEventHandler[K,V](config: ProducerConfig, events.map{e => try { if(e.hasKey) - serializedMessages += KeyedMessage[K,Message](topic = e.topic, key = e.key, message = new Message(key = keyEncoder.toBytes(e.key), bytes = encoder.toBytes(e.message))) + serializedMessages += new KeyedMessage[K,Message](topic = e.topic, key = e.key, partKey = e.partKey, message = new Message(key = keyEncoder.toBytes(e.key), bytes = encoder.toBytes(e.message))) else - serializedMessages += KeyedMessage[K,Message](topic = e.topic, key = null.asInstanceOf[K], message = new Message(bytes = encoder.toBytes(e.message))) + serializedMessages += new KeyedMessage[K,Message](topic = e.topic, key = e.key, partKey = e.partKey, message = new Message(bytes = encoder.toBytes(e.message))) } catch { case t: Throwable => producerStats.serializationErrorRate.mark() @@ -148,7 +148,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, try { for (message <- messages) { val topicPartitionsList = getPartitionListForTopic(message) - val partitionIndex = getPartition(message.topic, message.key, topicPartitionsList) + val partitionIndex = getPartition(message.topic, message.partitionKey, topicPartitionsList) val brokerPartition = topicPartitionsList(partitionIndex) // postpone the failure until the send operation, so that requests for other brokers are handled correctly @@ -195,11 +195,12 @@ class DefaultEventHandler[K,V](config: ProducerConfig, /** * Retrieves the partition id and throws an UnknownTopicOrPartitionException if * the value of partition is not between 0 and numPartitions-1 + * @param topic The topic * @param key the partition key * @param topicPartitionList the list of available partitions * @return the partition id */ - private def getPartition(topic: String, key: K, topicPartitionList: Seq[PartitionAndLeader]): Int = { + private def getPartition(topic: String, key: Any, topicPartitionList: Seq[PartitionAndLeader]): Int = { val numPartitions = topicPartitionList.size if(numPartitions <= 0) throw new UnknownTopicOrPartitionException("Topic " + topic + " doesn't exist") @@ -260,7 +261,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, if (logger.isTraceEnabled) { val successfullySentData = response.status.filter(_._2.error == ErrorMapping.NoError) successfullySentData.foreach(m => messagesPerTopic(m._1).foreach(message => - trace("Successfully sent message: %s".format(Utils.readString(message.message.payload))))) + trace("Successfully sent message: %s".format(if(message.message.isNull) null else Utils.readString(message.message.payload))))) } val failedPartitionsAndStatus = response.status.filter(_._2.error != ErrorMapping.NoError).toSeq failedTopicPartitions = failedPartitionsAndStatus.map(partitionStatus => partitionStatus._1) @@ -275,8 +276,9 @@ class DefaultEventHandler[K,V](config: ProducerConfig, warn("Produce request with correlation id %d failed due to %s".format(currentCorrelationId, errorString)) } failedTopicPartitions - } else + } else { Seq.empty[TopicAndPartition] + } } catch { case t: Throwable => warn("Failed to send producer request with correlation id %d to broker %d with data for partitions %s" diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 15b7bd31446ff..9390edf37dae9 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -18,14 +18,17 @@ package kafka.server import scala.collection.mutable -import kafka.utils.Logging +import scala.collection.Set +import scala.collection.Map +import kafka.utils.{Utils, Logging} import kafka.cluster.Broker import kafka.metrics.KafkaMetricsGroup +import kafka.common.TopicAndPartition import com.yammer.metrics.core.Gauge abstract class AbstractFetcherManager(protected val name: String, metricPrefix: String, numFetchers: Int = 1) extends Logging with KafkaMetricsGroup { - // map of (source brokerid, fetcher Id per source broker) => fetcher + // map of (source broker_id, fetcher_id per source broker) => fetcher private val fetcherThreadMap = new mutable.HashMap[BrokerAndFetcherId, AbstractFetcherThread] private val mapLock = new Object this.logIdent = "[" + name + "] " @@ -60,36 +63,43 @@ abstract class AbstractFetcherManager(protected val name: String, metricPrefix: ) private def getFetcherId(topic: String, partitionId: Int) : Int = { - (topic.hashCode() + 31 * partitionId) % numFetchers + Utils.abs(31 * topic.hashCode() + partitionId) % numFetchers } // to be defined in subclass to create a specific fetcher def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread - def addFetcher(topic: String, partitionId: Int, initialOffset: Long, sourceBroker: Broker) { + def addFetcherForPartitions(partitionAndOffsets: Map[TopicAndPartition, BrokerAndInitialOffset]) { mapLock synchronized { - var fetcherThread: AbstractFetcherThread = null - val key = new BrokerAndFetcherId(sourceBroker, getFetcherId(topic, partitionId)) - fetcherThreadMap.get(key) match { - case Some(f) => fetcherThread = f - case None => - fetcherThread = createFetcherThread(key.fetcherId, sourceBroker) - fetcherThreadMap.put(key, fetcherThread) - fetcherThread.start + val partitionsPerFetcher = partitionAndOffsets.groupBy{ case(topicAndPartition, brokerAndInitialOffset) => + BrokerAndFetcherId(brokerAndInitialOffset.broker, getFetcherId(topicAndPartition.topic, topicAndPartition.partition))} + for ((brokerAndFetcherId, partitionAndOffsets) <- partitionsPerFetcher) { + var fetcherThread: AbstractFetcherThread = null + fetcherThreadMap.get(brokerAndFetcherId) match { + case Some(f) => fetcherThread = f + case None => + fetcherThread = createFetcherThread(brokerAndFetcherId.fetcherId, brokerAndFetcherId.broker) + fetcherThreadMap.put(brokerAndFetcherId, fetcherThread) + fetcherThread.start + } + + fetcherThreadMap(brokerAndFetcherId).addPartitions(partitionAndOffsets.map { case (topicAndPartition, brokerAndInitOffset) => + topicAndPartition -> brokerAndInitOffset.initOffset + }) } - fetcherThread.addPartition(topic, partitionId, initialOffset) - info("Adding fetcher for partition [%s,%d], initOffset %d to broker %d with fetcherId %d" - .format(topic, partitionId, initialOffset, sourceBroker.id, key.fetcherId)) } + + info("Added fetcher for partitions %s".format(partitionAndOffsets.map{ case (topicAndPartition, brokerAndInitialOffset) => + "[" + topicAndPartition + ", initOffset " + brokerAndInitialOffset.initOffset + " to broker " + brokerAndInitialOffset.broker + "] "})) } - def removeFetcher(topic: String, partitionId: Int) { - info("Removing fetcher for partition [%s,%d]".format(topic, partitionId)) + def removeFetcherForPartitions(partitions: Set[TopicAndPartition]) { mapLock synchronized { for ((key, fetcher) <- fetcherThreadMap) { - fetcher.removePartition(topic, partitionId) + fetcher.removePartitions(partitions) } } + info("Removed fetcher for partitions %s".format(partitions.mkString(","))) } def shutdownIdleFetcherThreads() { @@ -115,4 +125,6 @@ abstract class AbstractFetcherManager(protected val name: String, metricPrefix: } } -case class BrokerAndFetcherId(broker: Broker, fetcherId: Int) \ No newline at end of file +case class BrokerAndFetcherId(broker: Broker, fetcherId: Int) + +case class BrokerAndInitialOffset(broker: Broker, initOffset: Long) \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index a5fc96df72a0d..db7017b8a8ff0 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -19,16 +19,19 @@ package kafka.server import kafka.cluster.Broker import collection.mutable +import scala.collection.Set +import scala.collection.Map import kafka.message.{InvalidMessageException, ByteBufferMessageSet, MessageAndOffset} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge -import java.util.concurrent.atomic.AtomicLong import kafka.utils.{Pool, ShutdownableThread} -import java.util.concurrent.TimeUnit -import java.util.concurrent.locks.ReentrantLock import kafka.consumer.{PartitionTopicInfo, SimpleConsumer} import kafka.api.{FetchRequest, FetchResponse, FetchResponsePartitionData, FetchRequestBuilder} import kafka.common.{KafkaException, ClientIdAndBroker, TopicAndPartition, ErrorMapping} +import kafka.utils.Utils.inLock +import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock +import java.util.concurrent.atomic.AtomicLong /** @@ -70,8 +73,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke } override def doWork() { - partitionMapLock.lock() - try { + inLock(partitionMapLock) { if (partitionMap.isEmpty) partitionMapCond.await(200L, TimeUnit.MILLISECONDS) partitionMap.foreach { @@ -79,8 +81,6 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition, offset, fetchSize) } - } finally { - partitionMapLock.unlock() } val fetchRequest = fetchRequestBuilder.build() @@ -97,7 +97,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke } catch { case t: Throwable => if (isRunning.get) { - warn("Error in fetch %s".format(fetchRequest), t) + error("Error in fetch %s".format(fetchRequest), t) partitionMapLock synchronized { partitionsWithError ++= partitionMap.keys } @@ -107,8 +107,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke if (response != null) { // process fetched data - partitionMapLock.lock() - try { + inLock(partitionMapLock) { response.data.foreach { case(topicAndPartition, partitionData) => val (topic, partitionId) = topicAndPartition.asTuple @@ -135,7 +134,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke // 1. If there is a corrupt message in a topic partition, it does not bring the fetcher thread down and cause other topic partition to also lag // 2. If the message is corrupt due to a transient state in the log (truncation, partial writes can cause this), we simply continue and // should get fixed in the subsequent fetches - logger.warn("Found invalid messages during fetch for partition [" + topic + "," + partitionId + "] offset " + currentOffset.get + " error " + ime.getMessage) + logger.error("Found invalid messages during fetch for partition [" + topic + "," + partitionId + "] offset " + currentOffset.get + " error " + ime.getMessage) case e: Throwable => throw new KafkaException("error processing data for partition [%s,%d] offset %d" .format(topic, partitionId, currentOffset.get), e) @@ -144,24 +143,22 @@ 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" + error("Current offset %d for partition [%s,%d] out of range; reset offset to %d" .format(currentOffset.get, topic, partitionId, newOffset)) } catch { case e: Throwable => - warn("Error getting offset for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), e) + error("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:%s".format(topic, partitionId, sourceBroker.id, + error("Error for partition [%s,%d] to broker %d:%s".format(topic, partitionId, sourceBroker.id, ErrorMapping.exceptionFor(partitionData.error).getClass)) partitionsWithError += topicAndPartition } } } } - } finally { - partitionMapLock.unlock() } } @@ -171,23 +168,26 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke } } - def addPartition(topic: String, partitionId: Int, initialOffset: Long) { + def addPartitions(partitionAndOffsets: Map[TopicAndPartition, Long]) { partitionMapLock.lockInterruptibly() try { - val topicPartition = TopicAndPartition(topic, partitionId) - partitionMap.put( - topicPartition, - if (PartitionTopicInfo.isOffsetInvalid(initialOffset)) handleOffsetOutOfRange(topicPartition) else initialOffset) + for ((topicAndPartition, offset) <- partitionAndOffsets) { + // If the partitionMap already has the topic/partition, then do not update the map with the old offset + if (!partitionMap.contains(topicAndPartition)) + partitionMap.put( + topicAndPartition, + if (PartitionTopicInfo.isOffsetInvalid(offset)) handleOffsetOutOfRange(topicAndPartition) else offset) + } partitionMapCond.signalAll() } finally { partitionMapLock.unlock() } } - def removePartition(topic: String, partitionId: Int) { + def removePartitions(topicAndPartitions: Set[TopicAndPartition]) { partitionMapLock.lockInterruptibly() try { - partitionMap.remove(TopicAndPartition(topic, partitionId)) + topicAndPartitions.foreach(tp => partitionMap.remove(tp)) } finally { partitionMapLock.unlock() } @@ -204,7 +204,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke } class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMetricsGroup { - private[this] var lagVal = new AtomicLong(-1L) + private[this] val lagVal = new AtomicLong(-1L) newGauge( metricId + "-ConsumerLag", new Gauge[Long] { diff --git a/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala b/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala deleted file mode 100644 index e18ecab4857ed..0000000000000 --- a/core/src/main/scala/kafka/server/HighwaterMarkCheckpoint.scala +++ /dev/null @@ -1,121 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.server - -import kafka.utils.Logging -import kafka.common._ -import java.util.concurrent.locks.ReentrantLock -import java.io._ - -/** - * This class handles the read/write to the highwaterMark checkpoint file. The file stores the high watermark value for - * all topics and partitions that this broker hosts. The format of this file is as follows - - * version - * number of entries - * topic partition highwatermark - */ - -object HighwaterMarkCheckpoint { - val highWatermarkFileName = "replication-offset-checkpoint" - val currentHighwaterMarkFileVersion = 0 -} - -class HighwaterMarkCheckpoint(val path: String) extends Logging { - /* create the highwatermark file handle for all partitions */ - val name = path + File.separator + HighwaterMarkCheckpoint.highWatermarkFileName - private val hwFile = new File(name) - private val hwFileLock = new ReentrantLock() - // recover from previous tmp file, if required - - def write(highwaterMarksPerPartition: Map[TopicAndPartition, Long]) { - hwFileLock.lock() - try { - // write to temp file and then swap with the highwatermark file - val tempHwFile = new File(hwFile + ".tmp") - - val hwFileWriter = new BufferedWriter(new FileWriter(tempHwFile)) - // checkpoint highwatermark for all partitions - // write the current version - hwFileWriter.write(HighwaterMarkCheckpoint.currentHighwaterMarkFileVersion.toString) - hwFileWriter.newLine() - // write the number of entries in the highwatermark file - hwFileWriter.write(highwaterMarksPerPartition.size.toString) - hwFileWriter.newLine() - - highwaterMarksPerPartition.foreach { partitionAndHw => - hwFileWriter.write("%s %s %s".format(partitionAndHw._1.topic, partitionAndHw._1.partition, partitionAndHw._2)) - hwFileWriter.newLine() - } - hwFileWriter.flush() - hwFileWriter.close() - // swap new high watermark file with previous one - if(!tempHwFile.renameTo(hwFile)) { - // renameTo() fails on Windows if the destination file exists. - hwFile.delete() - if(!tempHwFile.renameTo(hwFile)) { - fatal("Attempt to swap the new high watermark file with the old one failed") - System.exit(1) - } - } - }finally { - hwFileLock.unlock() - } - } - - def read(topic: String, partition: Int): Long = { - hwFileLock.lock() - try { - hwFile.length() match { - case 0 => - warn("No highwatermark file is found. Returning 0 as the highwatermark for partition [%s,%d]".format(topic, partition)) - 0L - case _ => - val hwFileReader = new BufferedReader(new FileReader(hwFile)) - val version = hwFileReader.readLine().toShort - version match { - case HighwaterMarkCheckpoint.currentHighwaterMarkFileVersion => - val numberOfHighWatermarks = hwFileReader.readLine().toInt - val partitionHighWatermarks = - for(i <- 0 until numberOfHighWatermarks) yield { - val nextHwEntry = hwFileReader.readLine() - val partitionHwInfo = nextHwEntry.split(" ") - val topic = partitionHwInfo(0) - val partitionId = partitionHwInfo(1).toInt - val highWatermark = partitionHwInfo(2).toLong - (TopicAndPartition(topic, partitionId) -> highWatermark) - } - hwFileReader.close() - val hwOpt = partitionHighWatermarks.toMap.get(TopicAndPartition(topic, partition)) - hwOpt match { - case Some(hw) => - debug("Read hw %d for partition [%s,%d] from highwatermark checkpoint file".format(hw, topic, partition)) - hw - case None => - warn("No previously checkpointed highwatermark value found for topic %s ".format(topic) + - "partition %d. Returning 0 as the highwatermark".format(partition)) - 0L - } - case _ => fatal("Unrecognized version of the highwatermark checkpoint file " + version) - System.exit(1) - -1L - } - } - }finally { - hwFileLock.unlock() - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4679e1839bf64..b705a1a2006b5 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,11 +17,15 @@ package kafka.server -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import kafka.api._ import kafka.message._ import kafka.network._ +import kafka.log._ +import kafka.utils.ZKGroupTopicDirs +import org.apache.log4j.Logger import scala.collection._ +import java.util.Properties import java.util.concurrent.TimeUnit import java.util.concurrent.atomic._ import kafka.metrics.KafkaMetricsGroup @@ -39,7 +43,8 @@ import kafka.controller.KafkaController class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, val zkClient: ZkClient, - brokerId: Int, + val brokerId: Int, + val config: KafkaConfig, val controller: KafkaController) extends Logging { private val producerRequestPurgatory = @@ -70,7 +75,9 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.StopReplicaKey => handleStopReplicaRequest(request) case RequestKeys.UpdateMetadataKey => handleUpdateMetadataRequest(request) case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request) - case requestId => throw new KafkaException("No mapping found for handler id " + requestId) + case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request) + case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request) + case requestId => throw new KafkaException("Unknown api code " + requestId) } } catch { case e: Throwable => @@ -238,16 +245,22 @@ class KafkaApis(val requestChannel: RequestChannel, try { val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) - val (start, end) = + val info = partitionOpt match { case Some(partition) => partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet]) case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" .format(topicAndPartition, brokerId)) } + val numAppendedMessages = if (info.firstOffset == -1L || info.lastOffset == -1L) 0 else (info.lastOffset - info.firstOffset + 1) + + // update stats + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) + BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) + trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" - .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, start, end)) - ProduceResult(topicAndPartition, start, end) + .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) + ProduceResult(topicAndPartition, info.firstOffset, info.lastOffset) } catch { // NOTE: Failed produce requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException // since failed produce requests metric is supposed to indicate failure of a broker in handling a produce request @@ -378,11 +391,11 @@ class KafkaApis(val requestChannel: RequestChannel, else replicaManager.getLeaderReplicaIfLocal(topic, partition) trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize)) - val maxOffsetOpt = if (fromReplicaId == Request.OrdinaryConsumerId) { - Some(localReplica.highWatermark) - } else { - None - } + val maxOffsetOpt = + if (Request.isReplicaIdFromFollower(fromReplicaId)) + None + else + Some(localReplica.highWatermark) val messages = localReplica.log match { case Some(log) => log.read(offset, maxSize, maxOffsetOpt) @@ -407,15 +420,18 @@ class KafkaApis(val requestChannel: RequestChannel, else replicaManager.getReplicaOrException(topicAndPartition.topic, topicAndPartition.partition) val offsets = { - val allOffsets = replicaManager.logManager.getOffsets(topicAndPartition, - partitionOffsetRequestInfo.time, - partitionOffsetRequestInfo.maxNumOffsets) - if (!offsetRequest.isFromOrdinaryClient) allOffsets - else { + val allOffsets = fetchOffsets(replicaManager.logManager, + topicAndPartition, + partitionOffsetRequestInfo.time, + partitionOffsetRequestInfo.maxNumOffsets) + if (!offsetRequest.isFromOrdinaryClient) { + allOffsets + } else { val hw = localReplica.highWatermark if (allOffsets.exists(_ > hw)) hw +: allOffsets.dropWhile(_ > hw) - else allOffsets + else + allOffsets } } (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.NoError, offsets)) @@ -438,6 +454,59 @@ class KafkaApis(val requestChannel: RequestChannel, val response = OffsetResponse(offsetRequest.correlationId, responseMap) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } + + def fetchOffsets(logManager: LogManager, topicAndPartition: TopicAndPartition, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { + logManager.getLog(topicAndPartition) match { + case Some(log) => + fetchOffsetsBefore(log, timestamp, maxNumOffsets) + case None => + if (timestamp == OffsetRequest.LatestTime || timestamp == OffsetRequest.EarliestTime) + Seq(0L) + else + Nil + } + } + + def fetchOffsetsBefore(log: Log, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { + val segsArray = log.logSegments.toArray + var offsetTimeArray: Array[(Long, Long)] = null + if(segsArray.last.size > 0) + offsetTimeArray = new Array[(Long, Long)](segsArray.length + 1) + else + offsetTimeArray = new Array[(Long, Long)](segsArray.length) + + for(i <- 0 until segsArray.length) + offsetTimeArray(i) = (segsArray(i).baseOffset, segsArray(i).lastModified) + if(segsArray.last.size > 0) + offsetTimeArray(segsArray.length) = (log.logEndOffset, SystemTime.milliseconds) + + var startIndex = -1 + timestamp match { + case OffsetRequest.LatestTime => + startIndex = offsetTimeArray.length - 1 + case OffsetRequest.EarliestTime => + startIndex = 0 + case _ => + var isFound = false + debug("Offset time array = " + offsetTimeArray.foreach(o => "%d, %d".format(o._1, o._2))) + startIndex = offsetTimeArray.length - 1 + while (startIndex >= 0 && !isFound) { + if (offsetTimeArray(startIndex)._2 <= timestamp) + isFound = true + else + startIndex -=1 + } + } + + val retSize = maxNumOffsets.min(startIndex + 1) + val ret = new Array[Long](retSize) + for(j <- 0 until retSize) { + ret(j) = offsetTimeArray(startIndex)._1 + startIndex -= 1 + } + // ensure that the returned seq is in descending order of offsets + ret.toSeq.sortBy(- _) + } /** * Service the topic metadata request API @@ -450,8 +519,11 @@ class KafkaApis(val requestChannel: RequestChannel, uniqueTopics = { if(metadataRequest.topics.size > 0) metadataRequest.topics.toSet - else - ZkUtils.getAllTopics(zkClient).toSet + else { + partitionMetadataLock synchronized { + leaderCache.keySet.map(_.topic) + } + } } val topicMetadataList = partitionMetadataLock synchronized { @@ -502,7 +574,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ErrorMapping.UnknownTopicOrPartitionCode => if (config.autoCreateTopicsEnable) { try { - CreateTopicCommand.createTopic(zkClient, topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor) + AdminUtils.createTopic(zkClient, topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor, config.defaultMaxRackReplication) info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" .format(topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor)) } catch { @@ -523,6 +595,59 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } + /* + * Service the Offset commit API + */ + def handleOffsetCommitRequest(request: RequestChannel.Request) { + val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] + val responseInfo = offsetCommitRequest.requestInfo.map{ + case (topicAndPartition, metaAndError) => { + val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicAndPartition.topic) + try { + if(metaAndError.metadata != null && metaAndError.metadata.length > config.offsetMetadataMaxSize) { + (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) + } else { + ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + + topicAndPartition.partition, metaAndError.offset.toString) + (topicAndPartition, ErrorMapping.NoError) + } + } catch { + case e: Throwable => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) + } + } + } + val response = new OffsetCommitResponse(responseInfo, + offsetCommitRequest.correlationId) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } + + /* + * Service the Offset fetch API + */ + def handleOffsetFetchRequest(request: RequestChannel.Request) { + val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] + val responseInfo = offsetFetchRequest.requestInfo.map( t => { + val topicDirs = new ZKGroupTopicDirs(offsetFetchRequest.groupId, t.topic) + try { + val payloadOpt = ZkUtils.readDataMaybeNull(zkClient, topicDirs.consumerOffsetDir + "/" + t.partition)._1 + payloadOpt match { + case Some(payload) => { + (t, OffsetMetadataAndError(offset=payload.toLong, error=ErrorMapping.NoError)) + } + case None => (t, OffsetMetadataAndError(OffsetMetadataAndError.InvalidOffset, OffsetMetadataAndError.NoMetadata, + ErrorMapping.UnknownTopicOrPartitionCode)) + } + } catch { + case e: Throwable => + (t, OffsetMetadataAndError(OffsetMetadataAndError.InvalidOffset, OffsetMetadataAndError.NoMetadata, + ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))) + } + }) + val response = new OffsetFetchResponse(collection.immutable.Map(responseInfo: _*), + offsetFetchRequest.correlationId) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } + def close() { debug("Shutting down.") fetchRequestPurgatory.shutdown() diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 41c962604854a..c171185606376 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -31,12 +31,29 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro this(new VerifiableProperties(originalProps)) props.verify() } + + private def getLogRetentionTimeMillis(): Long = { + var millisInMinute = 60L * 1000L + val millisInHour = 60L * millisInMinute + if(props.containsKey("log.retention.minutes")){ + millisInMinute * props.getIntInRange("log.retention.minutes", (1, Int.MaxValue)) + } else { + millisInHour * props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue)) + } + + } /*********** General Configuration ***********/ /* the broker id for this server */ val brokerId: Int = props.getIntInRange("broker.id", (0, Int.MaxValue)) + /* the rack id for this server */ + val rackId: Int = props.getIntInRange("broker.rack", 0, (0, Int.MaxValue)) + + /* the default max-rack-replication for auto topic creation */ + val defaultMaxRackReplication = props.getIntInRange("default.max.rack.replication", -1, (-1, Int.MaxValue)) + /* the maximum size of message that the server can receive */ val messageMaxBytes = props.getIntInRange("message.max.bytes", 1000000 + MessageSet.LogOverhead, (0, Int.MaxValue)) @@ -46,6 +63,9 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the number of io threads that the server uses for carrying out network requests */ val numIoThreads = props.getIntInRange("num.io.threads", 8, (1, Int.MaxValue)) + /* the number of threads to use for various background processing tasks */ + val backgroundThreads = props.getIntInRange("background.threads", 4, (1, Int.MaxValue)) + /* the number of queued requests allowed before blocking the network threads */ val queuedMaxRequests = props.getIntInRange("queued.max.requests", 500, (1, Int.MaxValue)) @@ -55,8 +75,19 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val port: Int = props.getInt("port", 6667) /* hostname of broker. If this is set, it will only bind to this address. If this is not set, - * it will bind to all interfaces, and publish one to ZK */ + * it will bind to all interfaces */ val hostName: String = props.getString("host.name", null) + + /* hostname to publish to ZooKeeper for clients to use. In IaaS environments, this may + * need to be different from the interface to which the broker binds. If this is not set, + * it will use the value for "host.name" if configured. Otherwise + * it will use the value returned from java.net.InetAddress.getCanonicalHostName(). */ + val advertisedHostName: String = props.getString("advertised.host.name", hostName) + + /* the port to publish to ZooKeeper for clients to use. In IaaS environments, this may + * need to be different from the port to which the broker binds. If this is not set, + * it will publish the same port that the broker binds to. */ + val advertisedPort: Int = props.getInt("advertised.port", port) /* the SO_SNDBUFF buffer of the socket sever sockets */ val socketSendBufferBytes: Int = props.getInt("socket.send.buffer.bytes", 100*1024) @@ -79,29 +110,49 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the maximum size of a single log file */ val logSegmentBytes = props.getIntInRange("log.segment.bytes", 1*1024*1024*1024, (Message.MinHeaderSize, Int.MaxValue)) - /* the maximum size of a single log file for some specific topic */ - val logSegmentBytesPerTopicMap = props.getMap("log.segment.bytes.per.topic", _.toInt > 0).mapValues(_.toInt) - /* the maximum time before a new log segment is rolled out */ val logRollHours = props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue)) - /* the number of hours before rolling out a new log segment for some specific topic */ - val logRollHoursPerTopicMap = props.getMap("log.roll.hours.per.topic", _.toInt > 0).mapValues(_.toInt) - /* the number of hours to keep a log file before deleting it */ - val logRetentionHours = props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue)) - - /* the number of hours to keep a log file before deleting it for some specific topic*/ - val logRetentionHoursPerTopicMap = props.getMap("log.retention.hours.per.topic", _.toInt > 0).mapValues(_.toInt) + val logRetentionTimeMillis = getLogRetentionTimeMillis /* the maximum size of the log before deleting it */ val logRetentionBytes = props.getLong("log.retention.bytes", -1) - /* the maximum size of the log for some specific topic before deleting it */ - val logRetentionBytesPerTopicMap = props.getMap("log.retention.bytes.per.topic", _.toLong > 0).mapValues(_.toLong) - /* the frequency in minutes that the log cleaner checks whether any log is eligible for deletion */ - val logCleanupIntervalMins = props.getIntInRange("log.cleanup.interval.mins", 10, (1, Int.MaxValue)) + val logCleanupIntervalMs = props.getLongInRange("log.retention.check.interval.ms", 5*60*1000, (1, Long.MaxValue)) + + /* the default cleanup policy for segments beyond the retention window, must be either "delete" or "dedupe" */ + val logCleanupPolicy = props.getString("log.cleanup.policy", "delete") + + /* the number of background threads to use for log cleaning */ + val logCleanerThreads = props.getIntInRange("log.cleaner.threads", 1, (0, Int.MaxValue)) + + /* the log cleaner will be throttled so that the sum of its read and write i/o will be less than this value on average */ + val logCleanerIoMaxBytesPerSecond = props.getDouble("log.cleaner.io.max.bytes.per.second", Double.MaxValue) + + /* the total memory used for log deduplication across all cleaner threads */ + val logCleanerDedupeBufferSize = props.getLongInRange("log.cleaner.dedupe.buffer.size", 500*1024*1024L, (0, Long.MaxValue)) + require(logCleanerDedupeBufferSize / logCleanerThreads > 1024*1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.") + + /* the total memory used for log cleaner I/O buffers across all cleaner threads */ + val logCleanerIoBufferSize = props.getIntInRange("log.cleaner.io.buffer.size", 512*1024, (0, Int.MaxValue)) + + /* log cleaner dedupe buffer load factor. The percentage full the dedupe buffer can become. A higher value + * will allow more log to be cleaned at once but will lead to more hash collisions */ + val logCleanerDedupeBufferLoadFactor = props.getDouble("log.cleaner.io.buffer.load.factor", 0.9d) + + /* the amount of time to sleep when there are no logs to clean */ + val logCleanerBackoffMs = props.getLongInRange("log.cleaner.backoff.ms", 30*1000, (0L, Long.MaxValue)) + + /* the minimum ratio of dirty log to total log for a log to eligible for cleaning */ + val logCleanerMinCleanRatio = props.getDouble("log.cleaner.min.cleanable.ratio", 0.5) + + /* should we enable log cleaning? */ + val logCleanerEnable = props.getBoolean("log.cleaner.enable", false) + + /* how long are delete records retained? */ + val logCleanerDeleteRetentionMs = props.getLong("log.cleaner.delete.retention.ms", 24 * 60 * 60 * 1000L) /* the maximum size in bytes of the offset index */ val logIndexSizeMaxBytes = props.getIntInRange("log.index.size.max.bytes", 10*1024*1024, (4, Int.MaxValue)) @@ -110,16 +161,19 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val logIndexIntervalBytes = props.getIntInRange("log.index.interval.bytes", 4096, (0, Int.MaxValue)) /* the number of messages accumulated on a log partition before messages are flushed to disk */ - val logFlushIntervalMessages = props.getIntInRange("log.flush.interval.messages", 10000, (1, Int.MaxValue)) + val logFlushIntervalMessages = props.getLongInRange("log.flush.interval.messages", Long.MaxValue, (1, Long.MaxValue)) - /* the maximum time in ms that a message in selected topics is kept in memory before flushed to disk, e.g., topic1:3000,topic2: 6000 */ - val logFlushIntervalMsPerTopicMap = props.getMap("log.flush.interval.ms.per.topic", _.toInt > 0).mapValues(_.toInt) + /* the amount of time to wait before deleting a file from the filesystem */ + val logDeleteDelayMs = props.getLongInRange("log.segment.delete.delay.ms", 60000, (0, Long.MaxValue)) /* the frequency in ms that the log flusher checks whether any log needs to be flushed to disk */ - val logFlushSchedulerIntervalMs = props.getInt("log.flush.scheduler.interval.ms", 3000) + val logFlushSchedulerIntervalMs = props.getLong("log.flush.scheduler.interval.ms", Long.MaxValue) /* the maximum time in ms that a message in any topic is kept in memory before flushed to disk */ - val logFlushIntervalMs = props.getInt("log.flush.interval.ms", logFlushSchedulerIntervalMs) + val logFlushIntervalMs = props.getLong("log.flush.interval.ms", logFlushSchedulerIntervalMs) + + /* the frequency with which we update the persistent record of the last flush which acts as the log recovery point */ + val logFlushOffsetCheckpointIntervalMs = props.getIntInRange("log.flush.offset.checkpoint.interval.ms", 60000, (0, Int.MaxValue)) /* enable auto creation of topic on the server */ val autoCreateTopicsEnable = props.getBoolean("auto.create.topics.enable", true) @@ -148,10 +202,13 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val replicaSocketReceiveBufferBytes = props.getInt("replica.socket.receive.buffer.bytes", ConsumerConfig.SocketBufferSize) /* the number of byes of messages to attempt to fetch */ - val replicaFetchMaxBytes = props.getInt("replica.fetch.max.bytes", ConsumerConfig.FetchSize) + val replicaFetchMaxBytes = props.getIntInRange("replica.fetch.max.bytes", ConsumerConfig.FetchSize, (messageMaxBytes, Int.MaxValue)) - /* max wait time for each fetcher request issued by follower replicas*/ + /* max wait time for each fetcher request issued by follower replicas. This value should always be less than the + * replica.lag.time.max.ms at all times to prevent frequent shrinking of ISR for low throughput topics */ val replicaFetchWaitMaxMs = props.getInt("replica.fetch.wait.max.ms", 500) + require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs, "replica.fetch.wait.max.ms should always be at least replica.lag.time.max.ms" + + " to prevent frequent changes in ISR") /* minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs */ val replicaFetchMinBytes = props.getInt("replica.fetch.min.bytes", 1) @@ -169,6 +226,18 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the purge interval (in number of requests) of the producer request purgatory */ val producerPurgatoryPurgeIntervalRequests = props.getInt("producer.purgatory.purge.interval.requests", 10000) + /* Enables auto leader balancing. A background thread checks and triggers leader + * balance if required at regular intervals */ + val autoLeaderRebalanceEnable = props.getBoolean("auto.leader.rebalance.enable", false) + + /* the ratio of leader imbalance allowed per broker. The controller would trigger a leader balance if it goes above + * this value per broker. The value is specified in percentage. */ + val leaderImbalancePerBrokerPercentage = props.getInt("leader.imbalance.per.broker.percentage", 10) + + /* the frequency with which the partition rebalance check is triggered by the controller */ + val leaderImbalanceCheckIntervalSeconds = props.getInt("leader.imbalance.check.interval.seconds", 300) + + /*********** Controlled shutdown configuration ***********/ /** Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens */ @@ -181,4 +250,8 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* enable controlled shutdown of the server */ val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", false) + /*********** Misc configuration ***********/ + + /* the maximum size for a metadata entry associated with an offset commit */ + val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", 1024) } diff --git a/core/src/main/scala/kafka/server/KafkaZooKeeper.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala similarity index 61% rename from core/src/main/scala/kafka/server/KafkaZooKeeper.scala rename to core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 553640f43dc7e..263f9b058570a 100644 --- a/core/src/main/scala/kafka/server/KafkaZooKeeper.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -25,30 +25,38 @@ import java.net.InetAddress /** - * Handles registering broker with zookeeper in the following path: - * /brokers/[0...N] --> host:port + * This class registers the broker in zookeeper to allow + * other brokers and consumers to detect failures. It uses an ephemeral znode with the path: + * /brokers/[0...N] --> advertisedHost:advertisedPort + * + * Right now our definition of health is fairly naive. If we register in zk we are healthy, otherwise + * we are dead. */ -class KafkaZooKeeper(config: KafkaConfig) extends Logging { +class KafkaHealthcheck(private val brokerId: Int, + private val rackId: Int, + private val advertisedHost: String, + private val advertisedPort: Int, + private val zkSessionTimeoutMs: Int, + private val zkClient: ZkClient) extends Logging { - val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + config.brokerId - private var zkClient: ZkClient = null - - def startup() { - /* start client */ - info("connecting to ZK: " + config.zkConnect) - zkClient = KafkaZookeeperClient.getZookeeperClient(config) - zkClient.subscribeStateChanges(new SessionExpireListener) - registerBrokerInZk() - } + val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId + + def startup() { + zkClient.subscribeStateChanges(new SessionExpireListener) + register() + } - private def registerBrokerInZk() { - val hostName = - if(config.hostName == null || config.hostName.trim.isEmpty) + /** + * Register this broker as "alive" in zookeeper + */ + def register() { + val advertisedHostName = + if(advertisedHost == null || advertisedHost.trim.isEmpty) InetAddress.getLocalHost.getCanonicalHostName else - config.hostName + advertisedHost val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt - ZkUtils.registerBrokerInZk(zkClient, config.brokerId, hostName, config.port, config.zkSessionTimeoutMs, jmxPort) + ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedHostName, advertisedPort, rackId, zkSessionTimeoutMs, jmxPort) } /** @@ -70,21 +78,11 @@ class KafkaZooKeeper(config: KafkaConfig) extends Logging { */ @throws(classOf[Exception]) def handleNewSession() { - info("re-registering broker info in ZK for broker " + config.brokerId) - registerBrokerInZk() + info("re-registering broker info in ZK for broker " + brokerId) + register() info("done re-registering broker") info("Subscribing to %s path to watch for new topics".format(ZkUtils.BrokerTopicsPath)) } } - def shutdown() { - if (zkClient != null) { - info("Closing zookeeper client...") - zkClient.close() - } - } - - def getZookeeperClient = { - zkClient - } } diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index 40281186b1ffd..d0f05cb3953a0 100644 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -37,7 +37,7 @@ class KafkaRequestHandler(id: Int, brokerId: Int, val requestChannel: RequestCha id, brokerId)) return } - req.dequeueTimeMs = SystemTime.milliseconds + req.requestDequeueTimeMs = SystemTime.milliseconds trace("Kafka request handler %d on broker %d handling request %s".format(id, brokerId, req)) apis.handle(req) } catch { @@ -68,7 +68,7 @@ class KafkaRequestHandlerPool(val brokerId: Int, handler.shutdown for(thread <- threads) thread.join - info("shutted down completely") + info("shut down completely") } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index a26de880a8927..3e55ce59356cf 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -17,15 +17,20 @@ package kafka.server -import kafka.network.{Receive, BlockingChannel, SocketServer} +import kafka.admin._ +import kafka.log.LogConfig +import kafka.log.CleanerConfig import kafka.log.LogManager import kafka.utils._ import java.util.concurrent._ import atomic.{AtomicInteger, AtomicBoolean} +import java.io.File +import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} import kafka.cluster.Broker import kafka.api.{ControlledShutdownResponse, ControlledShutdownRequest} import kafka.common.ErrorMapping +import kafka.network.{Receive, BlockingChannel, SocketServer} /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required @@ -40,29 +45,31 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var socketServer: SocketServer = null var requestHandlerPool: KafkaRequestHandlerPool = null var logManager: LogManager = null - var kafkaZookeeper: KafkaZooKeeper = null + var kafkaHealthcheck: KafkaHealthcheck = null + var topicConfigManager: TopicConfigManager = null var replicaManager: ReplicaManager = null var apis: KafkaApis = null var kafkaController: KafkaController = null - val kafkaScheduler = new KafkaScheduler(4) - + val kafkaScheduler = new KafkaScheduler(config.backgroundThreads) + var zkClient: ZkClient = null /** * Start up API for bringing up a single instance of the Kafka server. * Instantiates the LogManager, the SocketServer and the request handlers - KafkaRequestHandlers */ def startup() { - info("Starting") + info("starting") isShuttingDown = new AtomicBoolean(false) shutdownLatch = new CountDownLatch(1) /* start scheduler */ - kafkaScheduler.startup + kafkaScheduler.startup() + + /* setup zookeeper */ + zkClient = initZk() /* start log manager */ - logManager = new LogManager(config, - kafkaScheduler, - time) + logManager = createLogManager(zkClient) logManager.startup() socketServer = new SocketServer(config.brokerId, @@ -73,31 +80,39 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.socketSendBufferBytes, config.socketReceiveBufferBytes, config.socketRequestMaxBytes) + socketServer.startup() - socketServer.startup - - /* start client */ - kafkaZookeeper = new KafkaZooKeeper(config) - // starting relevant replicas and leader election for partitions assigned to this broker - kafkaZookeeper.startup - - info("Connecting to ZK: " + config.zkConnect) - - replicaManager = new ReplicaManager(config, time, kafkaZookeeper.getZookeeperClient, kafkaScheduler, logManager, isShuttingDown) - - kafkaController = new KafkaController(config, kafkaZookeeper.getZookeeperClient) - apis = new KafkaApis(socketServer.requestChannel, replicaManager, kafkaZookeeper.getZookeeperClient, config.brokerId, kafkaController) + replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) + kafkaController = new KafkaController(config, zkClient) + + /* start processing requests */ + apis = new KafkaApis(socketServer.requestChannel, replicaManager, zkClient, config.brokerId, config, kafkaController) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) - Mx4jLoader.maybeLoad + + Mx4jLoader.maybeLoad() - // start the replica manager replicaManager.startup() - // start the controller + kafkaController.startup() - // register metrics beans + + topicConfigManager = new TopicConfigManager(zkClient, logManager) + topicConfigManager.startup() + + /* tell everyone we are alive */ + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.rackId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) + kafkaHealthcheck.startup() + + registerStats() startupComplete.set(true); - info("Started") + info("started") + } + + private def initZk(): ZkClient = { + info("Connecting to zookeeper on " + config.zkConnect) + val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + ZkUtils.setupCommonPaths(zkClient) + zkClient } /** @@ -130,8 +145,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg // Get the current controller info. This is to ensure we use the most recent info to issue the // controlled shutdown request - val controllerId = ZkUtils.getController(kafkaZookeeper.getZookeeperClient) - ZkUtils.getBrokerInfo(kafkaZookeeper.getZookeeperClient, controllerId) match { + val controllerId = ZkUtils.getController(zkClient) + ZkUtils.getBrokerInfo(zkClient, controllerId) match { case Some(broker) => if (channel == null || prevController == null || !prevController.equals(broker)) { // if this is the first attempt or if the controller has changed, create a channel to the most recent @@ -199,12 +214,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg * Shuts down the LogManager, the SocketServer and the log cleaner scheduler thread */ def shutdown() { - info("Shutting down") + info("shutting down") val canShutdown = isShuttingDown.compareAndSet(false, true); if (canShutdown) { Utils.swallow(controlledShutdown()) - if(kafkaZookeeper != null) - Utils.swallow(kafkaZookeeper.shutdown()) if(socketServer != null) Utils.swallow(socketServer.shutdown()) if(requestHandlerPool != null) @@ -216,13 +229,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg Utils.swallow(replicaManager.shutdown()) if(logManager != null) Utils.swallow(logManager.shutdown()) - if(kafkaController != null) Utils.swallow(kafkaController.shutdown()) + if(zkClient != null) + Utils.swallow(zkClient.close()) shutdownLatch.countDown() startupComplete.set(false); - info("Shut down completed") + info("shut down completed") } } @@ -232,6 +246,43 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def awaitShutdown(): Unit = shutdownLatch.await() def getLogManager(): LogManager = logManager + + private def createLogManager(zkClient: ZkClient): LogManager = { + val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, + segmentMs = 60L * 60L * 1000L * config.logRollHours, + flushInterval = config.logFlushIntervalMessages, + flushMs = config.logFlushIntervalMs.toLong, + retentionSize = config.logRetentionBytes, + retentionMs = config.logRetentionTimeMillis, + maxMessageSize = config.messageMaxBytes, + maxIndexSize = config.logIndexSizeMaxBytes, + indexInterval = config.logIndexIntervalBytes, + deleteRetentionMs = config.logCleanerDeleteRetentionMs, + fileDeleteDelayMs = config.logDeleteDelayMs, + minCleanableRatio = config.logCleanerMinCleanRatio, + dedupe = config.logCleanupPolicy.trim.toLowerCase == "dedupe") + val defaultProps = defaultLogConfig.toProps + val configs = AdminUtils.fetchAllTopicConfigs(zkClient).mapValues(LogConfig.fromProps(defaultProps, _)) + // read the log configurations from zookeeper + val cleanerConfig = CleanerConfig(numThreads = config.logCleanerThreads, + dedupeBufferSize = config.logCleanerDedupeBufferSize, + dedupeBufferLoadFactor = config.logCleanerDedupeBufferLoadFactor, + ioBufferSize = config.logCleanerIoBufferSize, + maxMessageSize = config.messageMaxBytes, + maxIoBytesPerSecond = config.logCleanerIoMaxBytesPerSecond, + backOffMs = config.logCleanerBackoffMs, + enableCleaner = config.logCleanerEnable) + new LogManager(logDirs = config.logDirs.map(new File(_)).toArray, + topicConfigs = configs, + defaultConfig = defaultLogConfig, + cleanerConfig = cleanerConfig, + flushCheckMs = config.logFlushSchedulerIntervalMs, + flushCheckpointMs = config.logFlushOffsetCheckpointIntervalMs, + retentionCheckMs = config.logCleanupIntervalMs, + scheduler = kafkaScheduler, + time = time) + } + } diff --git a/core/src/main/scala/kafka/server/LeaderElector.scala b/core/src/main/scala/kafka/server/LeaderElector.scala index a0b79e47c0519..14b3fa4be8e65 100644 --- a/core/src/main/scala/kafka/server/LeaderElector.scala +++ b/core/src/main/scala/kafka/server/LeaderElector.scala @@ -27,8 +27,6 @@ trait LeaderElector extends Logging { def amILeader : Boolean -// def electAndBecomeLeader: Unit -// def elect: Boolean def close diff --git a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala new file mode 100644 index 0000000000000..b5719f89f79b9 --- /dev/null +++ b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server + +import scala.collection._ +import kafka.utils.Logging +import kafka.common._ +import java.util.concurrent.locks.ReentrantLock +import java.io._ + +/** + * This class saves out a map of topic/partition=>offsets to a file + */ +class OffsetCheckpoint(val file: File) extends Logging { + private val lock = new Object() + new File(file + ".tmp").delete() // try to delete any existing temp files for cleanliness + file.createNewFile() // in case the file doesn't exist + + def write(offsets: Map[TopicAndPartition, Long]) { + lock synchronized { + // write to temp file and then swap with the existing file + val temp = new File(file.getAbsolutePath + ".tmp") + + val writer = new BufferedWriter(new FileWriter(temp)) + try { + // write the current version + writer.write(0.toString) + writer.newLine() + + // write the number of entries + writer.write(offsets.size.toString) + writer.newLine() + + // write the entries + offsets.foreach { case (topicPart, offset) => + writer.write("%s %d %d".format(topicPart.topic, topicPart.partition, offset)) + writer.newLine() + } + + // flush and overwrite old file + writer.flush() + } finally { + writer.close() + } + + // swap new offset checkpoint file with previous one + if(!temp.renameTo(file)) { + // renameTo() fails on Windows if the destination file exists. + file.delete() + if(!temp.renameTo(file)) + throw new IOException("File rename from %s to %s failed.".format(temp.getAbsolutePath, file.getAbsolutePath)) + } + } + } + + def read(): Map[TopicAndPartition, Long] = { + lock synchronized { + val reader = new BufferedReader(new FileReader(file)) + try { + var line = reader.readLine() + if(line == null) + return Map.empty + val version = line.toInt + version match { + case 0 => + line = reader.readLine() + if(line == null) + return Map.empty + val expectedSize = line.toInt + var offsets = Map[TopicAndPartition, Long]() + line = reader.readLine() + while(line != null) { + val pieces = line.split("\\s+") + if(pieces.length != 3) + throw new IOException("Malformed line in offset checkpoint file: '%s'.".format(line)) + + val topic = pieces(0) + val partition = pieces(1).toInt + val offset = pieces(2).toLong + offsets += (TopicAndPartition(pieces(0), partition) -> offset) + line = reader.readLine() + } + if(offsets.size != expectedSize) + throw new IOException("Expected %d entries but found only %d".format(expectedSize, offsets.size)) + offsets + case _ => + throw new IOException("Unrecognized version of the highwatermark checkpoint file: " + version) + } + } finally { + reader.close() + } + } + } + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index a84da137700df..73e605eb31bc7 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -68,7 +68,6 @@ class ReplicaFetcherThread(name:String, */ def handleOffsetOutOfRange(topicAndPartition: TopicAndPartition): Long = { val replica = replicaMgr.getReplica(topicAndPartition.topic, topicAndPartition.partition).get - val log = replica.log.get /** * Unclean leader election: A follower goes down, in the meanwhile the leader keeps appending messages. The follower comes back up @@ -81,8 +80,8 @@ class ReplicaFetcherThread(name:String, * There is a potential for a mismatch between the logs of the two replicas here. We don't fix this mismatch as of now. */ val leaderEndOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, OffsetRequest.LatestTime, brokerConfig.brokerId) - if (leaderEndOffset < log.logEndOffset) { - log.truncateTo(leaderEndOffset) + if (leaderEndOffset < replica.logEndOffset) { + replicaMgr.logManager.truncateTo(Map(topicAndPartition -> leaderEndOffset)) warn("Replica %d for partition %s reset its fetch offset to current leader %d's latest offset %d" .format(brokerConfig.brokerId, topicAndPartition, sourceBroker.id, leaderEndOffset)) leaderEndOffset @@ -94,7 +93,7 @@ class ReplicaFetcherThread(name:String, * Roll out a new log at the follower with the start offset equal to the current leader's start offset and continue fetching. */ val leaderStartOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, OffsetRequest.EarliestTime, brokerConfig.brokerId) - log.truncateAndStartWithNewOffset(leaderStartOffset) + replicaMgr.logManager.truncateFullyAndStartAt(topicAndPartition, leaderStartOffset) warn("Replica %d for partition %s reset its fetch offset to current leader %d's start offset %d" .format(brokerConfig.brokerId, topicAndPartition, sourceBroker.id, leaderStartOffset)) leaderStartOffset diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 03ba60e82cdb3..f9d10d385cee4 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -20,6 +20,7 @@ import kafka.cluster.{Broker, Partition, Replica} import collection._ import mutable.HashMap import org.I0Itec.zkclient.ZkClient +import java.io.{File, IOException} import java.util.concurrent.atomic.AtomicBoolean import kafka.utils._ import kafka.log.LogManager @@ -34,12 +35,13 @@ import org.apache.log4j.Logger object ReplicaManager { val UnknownLogEndOffset = -1L + val HighWatermarkFilename = "replication-offset-checkpoint" } class ReplicaManager(val config: KafkaConfig, time: Time, val zkClient: ZkClient, - kafkaScheduler: KafkaScheduler, + scheduler: Scheduler, val logManager: LogManager, val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup { /* epoch of the controller that last changed the leader */ @@ -48,9 +50,10 @@ class ReplicaManager(val config: KafkaConfig, private val allPartitions = new Pool[(String, Int), Partition] private var leaderPartitions = new mutable.HashSet[Partition]() private val leaderPartitionsLock = new Object + private val replicaStateChangeLock = new Object val replicaFetcherManager = new ReplicaFetcherManager(config, this) private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false) - val highWatermarkCheckpoints = config.logDirs.map(dir => (dir, new HighwaterMarkCheckpoint(dir))).toMap + val highWatermarkCheckpoints = config.logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap private var hwThreadInitialized = false this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) @@ -74,20 +77,21 @@ class ReplicaManager(val config: KafkaConfig, newGauge( "UnderReplicatedPartitions", new Gauge[Int] { - def value = { - leaderPartitionsLock synchronized { - leaderPartitions.count(_.isUnderReplicated) - } - } + def value = underReplicatedPartitionCount() } ) val isrExpandRate = newMeter("IsrExpandsPerSec", "expands", TimeUnit.SECONDS) - val isrShrinkRate = newMeter("ISRShrinksPerSec", "shrinks", TimeUnit.SECONDS) + val isrShrinkRate = newMeter("IsrShrinksPerSec", "shrinks", TimeUnit.SECONDS) + def underReplicatedPartitionCount(): Int = { + leaderPartitionsLock synchronized { + leaderPartitions.count(_.isUnderReplicated) + } + } def startHighWaterMarksCheckPointThread() = { if(highWatermarkCheckPointThreadStarted.compareAndSet(false, true)) - kafkaScheduler.scheduleWithRate(checkpointHighWatermarks, "highwatermark-checkpoint-thread", 0, config.replicaHighWatermarkCheckpointIntervalMs) + scheduler.schedule("highwatermark-checkpoint", checkpointHighWatermarks, period = config.replicaHighWatermarkCheckpointIntervalMs, unit = TimeUnit.MILLISECONDS) } /** @@ -106,23 +110,22 @@ class ReplicaManager(val config: KafkaConfig, def startup() { // start ISR expiration thread - kafkaScheduler.scheduleWithRate(maybeShrinkIsr, "isr-expiration-thread-", 0, config.replicaLagTimeMaxMs) + scheduler.schedule("isr-expiration", maybeShrinkIsr, period = config.replicaLagTimeMaxMs, unit = TimeUnit.MILLISECONDS) } def stopReplica(topic: String, partitionId: Int, deletePartition: Boolean): Short = { stateChangeLogger.trace("Broker %d handling stop replica for partition [%s,%d]".format(localBrokerId, topic, partitionId)) val errorCode = ErrorMapping.NoError - getReplica(topic, partitionId) match { - case Some(replica) => - replicaFetcherManager.removeFetcher(topic, partitionId) - /* TODO: handle deleteLog in a better way */ - //if (deletePartition) - // logManager.deleteLog(topic, partition) + getPartition(topic, partitionId) match { + case Some(partition) => leaderPartitionsLock synchronized { - leaderPartitions -= replica.partition + leaderPartitions -= partition + } + if(deletePartition) { + val removedPartition = allPartitions.remove((topic, partitionId)) + if (removedPartition != null) + removedPartition.delete() // this will delete the local log } - if(deletePartition) - allPartitions.remove((topic, partitionId)) case None => //do nothing if replica no longer exists } stateChangeLogger.trace("Broker %d finished handling stop replica for partition [%s,%d]".format(localBrokerId, topic, partitionId)) @@ -130,20 +133,26 @@ class ReplicaManager(val config: KafkaConfig, } def stopReplicas(stopReplicaRequest: StopReplicaRequest): (mutable.Map[(String, Int), Short], Short) = { - val responseMap = new collection.mutable.HashMap[(String, Int), Short] - if(stopReplicaRequest.controllerEpoch < controllerEpoch) { - stateChangeLogger.warn("Broker %d received stop replica request from an old controller epoch %d." - .format(localBrokerId, stopReplicaRequest.controllerEpoch) + - " Latest known controller epoch is %d " + controllerEpoch) - (responseMap, ErrorMapping.StaleControllerEpochCode) - } else { - controllerEpoch = stopReplicaRequest.controllerEpoch - val responseMap = new HashMap[(String, Int), Short] - for((topic, partitionId) <- stopReplicaRequest.partitions){ - val errorCode = stopReplica(topic, partitionId, stopReplicaRequest.deletePartitions) - responseMap.put((topic, partitionId), errorCode) + replicaStateChangeLock synchronized { + val responseMap = new collection.mutable.HashMap[(String, Int), Short] + if(stopReplicaRequest.controllerEpoch < controllerEpoch) { + stateChangeLogger.warn("Broker %d received stop replica request from an old controller epoch %d." + .format(localBrokerId, stopReplicaRequest.controllerEpoch) + + " Latest known controller epoch is %d " + controllerEpoch) + (responseMap, ErrorMapping.StaleControllerEpochCode) + } else { + controllerEpoch = stopReplicaRequest.controllerEpoch + val responseMap = new HashMap[(String, Int), Short] + // First stop fetchers for all partitions, then stop the corresponding replicas + replicaFetcherManager.removeFetcherForPartitions(stopReplicaRequest.partitions.map { + case (topic, partition) => TopicAndPartition(topic, partition) + }) + for((topic, partitionId) <- stopReplicaRequest.partitions){ + val errorCode = stopReplica(topic, partitionId, stopReplicaRequest.deletePartitions) + responseMap.put((topic, partitionId), errorCode) + } + (responseMap, ErrorMapping.NoError) } - (responseMap, ErrorMapping.NoError) } } @@ -182,7 +191,7 @@ class ReplicaManager(val config: KafkaConfig, case Some(leaderReplica) => leaderReplica case None => throw new NotLeaderForPartitionException("Leader not local for partition [%s,%d] on broker %d" - .format(topic, partitionId, config.brokerId)) + .format(topic, partitionId, config.brokerId)) } } } @@ -196,88 +205,215 @@ class ReplicaManager(val config: KafkaConfig, } def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): (collection.Map[(String, Int), Short], Short) = { - leaderAndISRRequest.partitionStateInfos.foreach(p => - stateChangeLogger.trace("Broker %d handling LeaderAndIsr request correlation id %d received from controller %d epoch %d for partition [%s,%d]" - .format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, - leaderAndISRRequest.controllerEpoch, p._1._1, p._1._2))) - info("Handling LeaderAndIsr request %s".format(leaderAndISRRequest)) - - val responseMap = new collection.mutable.HashMap[(String, Int), Short] - if(leaderAndISRRequest.controllerEpoch < controllerEpoch) { - stateChangeLogger.warn("Broker %d received LeaderAndIsr request correlation id %d with an old controller epoch %d. Latest known controller epoch is %d" - .format(localBrokerId, leaderAndISRRequest.controllerEpoch, leaderAndISRRequest.correlationId, controllerEpoch)) - (responseMap, ErrorMapping.StaleControllerEpochCode) - }else { - val controllerId = leaderAndISRRequest.controllerId - controllerEpoch = leaderAndISRRequest.controllerEpoch - for((topicAndPartition, partitionStateInfo) <- leaderAndISRRequest.partitionStateInfos) { - var errorCode = ErrorMapping.NoError - val topic = topicAndPartition._1 - val partitionId = topicAndPartition._2 - - val requestedLeaderId = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader - try { - if(requestedLeaderId == config.brokerId) - makeLeader(controllerId, controllerEpoch, topic, partitionId, partitionStateInfo, leaderAndISRRequest.correlationId) - else - makeFollower(controllerId, controllerEpoch, topic, partitionId, partitionStateInfo, leaderAndISRRequest.leaders, - leaderAndISRRequest.correlationId) - } catch { - case e: Throwable => - val errorMsg = ("Error on broker %d while processing LeaderAndIsr request correlationId %d received from controller %d " + - "epoch %d for partition %s").format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, - leaderAndISRRequest.controllerEpoch, topicAndPartition) - stateChangeLogger.error(errorMsg, e) - errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partition), stateInfo) => + stateChangeLogger.trace("Broker %d received LeaderAndIsr request %s correlation id %d from controller %d epoch %d for partition [%s,%d]" + .format(localBrokerId, stateInfo, leaderAndISRRequest.correlationId, + leaderAndISRRequest.controllerId, leaderAndISRRequest.controllerEpoch, topic, partition)) + } + replicaStateChangeLock synchronized { + val responseMap = new collection.mutable.HashMap[(String, Int), Short] + if(leaderAndISRRequest.controllerEpoch < controllerEpoch) { + leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partition), stateInfo) => + stateChangeLogger.warn(("Broker %d received LeaderAndIsr request correlation id %d with an old controller epoch %d." + + " Latest known controller epoch is %d").format(localBrokerId, leaderAndISRRequest.correlationId, + leaderAndISRRequest.controllerEpoch, controllerEpoch)) } - responseMap.put(topicAndPartition, errorCode) - stateChangeLogger.trace("Broker %d handled LeaderAndIsr request correlationId %d received from controller %d epoch %d for partition [%s,%d]" - .format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId, leaderAndISRRequest.controllerEpoch, - topicAndPartition._1, topicAndPartition._2)) - } - info("Handled leader and isr request %s".format(leaderAndISRRequest)) - // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions - // have been completely populated before starting the checkpointing there by avoiding weird race conditions - if (!hwThreadInitialized) { - startHighWaterMarksCheckPointThread() - hwThreadInitialized = true + (responseMap, ErrorMapping.StaleControllerEpochCode) + } else { + val controllerId = leaderAndISRRequest.controllerId + val correlationId = leaderAndISRRequest.correlationId + controllerEpoch = leaderAndISRRequest.controllerEpoch + + // First check partition's leader epoch + val partitionState = new HashMap[Partition, PartitionStateInfo]() + leaderAndISRRequest.partitionStateInfos.foreach{ case ((topic, partitionId), partitionStateInfo) => + val partition = getOrCreatePartition(topic, partitionId, partitionStateInfo.replicationFactor) + val partitionLeaderEpoch = partition.getLeaderEpoch() + // If the leader epoch is valid record the epoch of the controller that made the leadership decision. + // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path + if (partitionLeaderEpoch < partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch) { + if(partitionStateInfo.allReplicas.contains(config.brokerId)) + partitionState.put(partition, partitionStateInfo) + else { + stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request with correlation id %d from " + + "controller %d epoch %d as broker is not in assigned replica list %s for partition [%s,%d]") + .format(localBrokerId, correlationId, controllerId, leaderAndISRRequest.controllerEpoch, + partitionStateInfo.allReplicas.mkString(","), topic, partition.partitionId)) + } + } else { + // Otherwise record the error code in response + stateChangeLogger.warn(("Broker %d received invalid LeaderAndIsr request with correlation id %d from " + + "controller %d epoch %d with an older leader epoch %d for partition [%s,%d], current leader epoch is %d") + .format(localBrokerId, correlationId, controllerId, leaderAndISRRequest.controllerEpoch, + partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch, topic, partition.partitionId, partitionLeaderEpoch)) + responseMap.put((topic, partitionId), ErrorMapping.StaleLeaderEpochCode) + } + } + + val partitionsTobeLeader = partitionState + .filter{ case (partition, partitionStateInfo) => partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId} + val partitionsTobeFollower = (partitionState -- partitionsTobeLeader.keys) + + if (!partitionsTobeLeader.isEmpty) makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap) + if (!partitionsTobeFollower.isEmpty) makeFollowers(controllerId, controllerEpoch, partitionsTobeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap) + + // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions + // have been completely populated before starting the checkpointing there by avoiding weird race conditions + if (!hwThreadInitialized) { + startHighWaterMarksCheckPointThread() + hwThreadInitialized = true + } + replicaFetcherManager.shutdownIdleFetcherThreads() + (responseMap, ErrorMapping.NoError) } - replicaFetcherManager.shutdownIdleFetcherThreads() - (responseMap, ErrorMapping.NoError) } } - private def makeLeader(controllerId: Int, epoch:Int, topic: String, partitionId: Int, - partitionStateInfo: PartitionStateInfo, correlationId: Int) = { - val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch - stateChangeLogger.trace(("Broker %d received LeaderAndIsr request correlationId %d from controller %d epoch %d " + - "starting the become-leader transition for partition [%s,%d]") - .format(localBrokerId, correlationId, controllerId, epoch, topic, partitionId)) - val partition = getOrCreatePartition(topic, partitionId, partitionStateInfo.replicationFactor) - if (partition.makeLeader(controllerId, topic, partitionId, leaderIsrAndControllerEpoch, correlationId)) { - // also add this partition to the list of partitions for which the leader is the current broker + /* + * Make the current broker to become follower for a given set of partitions by: + * + * 1. Stop fetchers for these partitions + * 2. Update the partition metadata in cache + * 3. Add these partitions to the leader partitions set + * + * If an unexpected error is thrown in this function, it will be propagated to KafkaApis where + * the error message will be set on each partition since we do not know which partition caused it + * TODO: the above may need to be fixed later + */ + private def makeLeaders(controllerId: Int, epoch: Int, + partitionState: Map[Partition, PartitionStateInfo], + correlationId: Int, responseMap: mutable.Map[(String, Int), Short]) = { + partitionState.foreach(state => + stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + + "starting the become-leader transition for partition %s") + .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId)))) + + for (partition <- partitionState.keys) + responseMap.put((partition.topic, partition.partitionId), ErrorMapping.NoError) + + try { + // First stop fetchers for all the partitions + replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(new TopicAndPartition(_))) + partitionState.foreach { state => + stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-leader request from controller " + + "%d epoch %d with correlation id %d for partition %s") + .format(localBrokerId, controllerId, epoch, correlationId, TopicAndPartition(state._1.topic, state._1.partitionId))) + } + // Update the partition information to be the leader + partitionState.foreach{ case (partition, partitionStateInfo) => + partition.makeLeader(controllerId, partitionStateInfo, correlationId)} + + // Finally add these partitions to the list of partitions for which the leader is the current broker leaderPartitionsLock synchronized { - leaderPartitions += partition - } + leaderPartitions ++= partitionState.keySet + } + } catch { + case e: Throwable => + partitionState.foreach { state => + val errorMsg = ("Error on broker %d while processing LeaderAndIsr request correlationId %d received from controller %d" + + " epoch %d for partition %s").format(localBrokerId, correlationId, controllerId, epoch, + TopicAndPartition(state._1.topic, state._1.partitionId)) + stateChangeLogger.error(errorMsg, e) + } + // Re-throw the exception for it to be caught in KafkaApis + throw e + } + + partitionState.foreach { state => + stateChangeLogger.trace(("Broker %d completed LeaderAndIsr request correlationId %d from controller %d epoch %d " + + "for the become-leader transition for partition %s") + .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) } - stateChangeLogger.trace("Broker %d completed become-leader transition for partition [%s,%d]".format(localBrokerId, topic, partitionId)) } - private def makeFollower(controllerId: Int, epoch: Int, topic: String, partitionId: 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)) + /* + * Make the current broker to become follower for a given set of partitions by: + * + * 1. Remove these partitions from the leader partitions set. + * 2. Mark the replicas as followers so that no more data can be added from the producer clients. + * 3. Stop fetchers for these partitions so that no more data can be added by the replica fetcher threads. + * 4. Truncate the log and checkpoint offsets for these partitions. + * 5. If the broker is not shutting down, add the fetcher to the new leaders. + * + * The ordering of doing these steps make sure that the replicas in transition will not + * take any more messages before checkpointing offsets so that all messages before the checkpoint + * are guaranteed to be flushed to disks + * + * If an unexpected error is thrown in this function, it will be propagated to KafkaApis where + * the error message will be set on each partition since we do not know which partition caused it + */ + private def makeFollowers(controllerId: Int, epoch: Int, partitionState: Map[Partition, PartitionStateInfo], + leaders: Set[Broker], correlationId: Int, responseMap: mutable.Map[(String, Int), Short]) { + partitionState.foreach(state => + stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + + "starting the become-follower transition for partition %s") + .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId)))) + + for (partition <- partitionState.keys) + responseMap.put((partition.topic, partition.partitionId), ErrorMapping.NoError) - val partition = getOrCreatePartition(topic, partitionId, partitionStateInfo.replicationFactor) - if (partition.makeFollower(controllerId, topic, partitionId, leaderIsrAndControllerEpoch, leaders, correlationId)) { - // remove this replica's partition from the ISR expiration queue + try { leaderPartitionsLock synchronized { - leaderPartitions -= partition + leaderPartitions --= partitionState.keySet } + + partitionState.foreach{ case (partition, leaderIsrAndControllerEpoch) => + partition.makeFollower(controllerId, leaderIsrAndControllerEpoch, leaders, correlationId)} + + replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(new TopicAndPartition(_))) + partitionState.foreach { state => + stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-follower request from controller " + + "%d epoch %d with correlation id %d for partition %s") + .format(localBrokerId, controllerId, epoch, correlationId, TopicAndPartition(state._1.topic, state._1.partitionId))) + } + + logManager.truncateTo(partitionState.map{ case(partition, leaderISRAndControllerEpoch) => + new TopicAndPartition(partition) -> partition.getOrCreateReplica().highWatermark + }) + partitionState.foreach { state => + stateChangeLogger.trace(("Broker %d truncated logs and checkpointed recovery boundaries for partition %s as part of " + + "become-follower request with correlation id %d from controller %d epoch %d").format(localBrokerId, + TopicAndPartition(state._1.topic, state._1.partitionId), correlationId, controllerId, epoch)) + } + if (!isShuttingDown.get()) { + val partitionAndOffsets = mutable.Map[TopicAndPartition, BrokerAndInitialOffset]() + partitionState.foreach { + case (partition, partitionStateInfo) => + val leader = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader + leaders.find(_.id == leader) match { + case Some(leaderBroker) => + partitionAndOffsets.put(new TopicAndPartition(partition), + BrokerAndInitialOffset(leaderBroker, partition.getReplica().get.logEndOffset)) + case None => + stateChangeLogger.trace(("Broker %d ignored the become-follower state change with correlation id %d " + + "controller %d epoch %d for partition %s since the designated leader %d " + + "cannot be found in live or shutting down brokers %s").format(localBrokerId, + correlationId, controllerId, epoch, partition, leader, leaders.mkString(","))) + } + } + replicaFetcherManager.addFetcherForPartitions(partitionAndOffsets) + } + else { + partitionState.foreach { state => + stateChangeLogger.trace(("Broker %d ignored the become-follower state change with correlation id %d from " + + "controller %d epoch %d for partition %s since it is shutting down").format(localBrokerId, correlationId, + controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) + } + } + } catch { + case e: Throwable => + val errorMsg = ("Error on broker %d while processing LeaderAndIsr request with correlationId %d received from controller %d " + + "epoch %d").format(localBrokerId, correlationId, controllerId, epoch) + stateChangeLogger.error(errorMsg, e) + // Re-throw the exception for it to be caught in KafkaApis + throw e + } + + partitionState.foreach { state => + stateChangeLogger.trace(("Broker %d completed LeaderAndIsr request correlationId %d from controller %d epoch %d " + + "for the become-follower transition for partition %s") + .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) } - stateChangeLogger.trace("Broker %d completed the become-follower transition for partition [%s,%d]".format(localBrokerId, topic, partitionId)) } private def maybeShrinkIsr(): Unit = { @@ -305,8 +441,14 @@ class ReplicaManager(val config: KafkaConfig, val replicas = allPartitions.values.map(_.getReplica(config.brokerId)).collect{case Some(replica) => replica} val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParent) for((dir, reps) <- replicasByDir) { - val hwms = reps.map(r => (TopicAndPartition(r.topic, r.partitionId) -> r.highWatermark)).toMap - highWatermarkCheckpoints(dir).write(hwms) + val hwms = reps.map(r => (new TopicAndPartition(r) -> r.highWatermark)).toMap + try { + highWatermarkCheckpoints(dir).write(hwms) + } catch { + case e: IOException => + fatal("Error writing to highwatermark file: ", e) + Runtime.getRuntime().halt(1) + } } } diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala new file mode 100644 index 0000000000000..42e98dd66f326 --- /dev/null +++ b/core/src/main/scala/kafka/server/TopicConfigManager.scala @@ -0,0 +1,134 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util.Properties +import scala.collection._ +import kafka.log._ +import kafka.utils._ +import kafka.admin.AdminUtils +import org.I0Itec.zkclient.{IZkChildListener, ZkClient} + +/** + * This class initiates and carries out topic config changes. + * + * It works as follows. + * + * Config is stored under the path + * /brokers/topics//config + * This znode stores the topic-overrides for this topic (but no defaults) in properties format. + * + * To avoid watching all topics for changes instead we have a notification path + * /brokers/config_changes + * The TopicConfigManager has a child watch on this path. + * + * To update a topic config we first update the topic config properties. Then we create a new sequential + * znode under the change path which contains the name of the topic that was updated, say + * /brokers/config_changes/config_change_13321 + * + * This will fire a watcher on all brokers. This watcher works as follows. It reads all the config change notifications. + * It keeps track of the highest config change suffix number it has applied previously. For any previously applied change it finds + * it checks if this notification is larger than a static expiration time (say 10mins) and if so it deletes this notification. + * For any new changes it reads the new configuration, combines it with the defaults, and updates the log config + * for all logs for that topic (if any) that it has. + * + * Note that config is always read from the config path in zk, the notification is just a trigger to do so. So if a broker is + * down and misses a change that is fine--when it restarts it will be loading the full config anyway. Note also that + * if there are two consecutive config changes it is possible that only the last one will be applied (since by the time the + * broker reads the config the both changes may have been made). In this case the broker would needlessly refresh the config twice, + * but that is harmless. + * + * On restart the config manager re-processes all notifications. This will usually be wasted work, but avoids any race conditions + * on startup where a change might be missed between the initial config load and registering for change notifications. + * + */ +class TopicConfigManager(private val zkClient: ZkClient, + private val logManager: LogManager, + private val changeExpirationMs: Long = 10*60*1000, + private val time: Time = SystemTime) extends Logging { + private var lastExecutedChange = -1L + + /** + * Begin watching for config changes + */ + def startup() { + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.TopicConfigChangesPath) + zkClient.subscribeChildChanges(ZkUtils.TopicConfigChangesPath, ConfigChangeListener) + processAllConfigChanges() + } + + /** + * Process all config changes + */ + private def processAllConfigChanges() { + val configChanges = zkClient.getChildren(ZkUtils.TopicConfigChangesPath) + import JavaConversions._ + processConfigChanges((configChanges: mutable.Buffer[String]).sorted) + } + + /** + * Process the given list of config changes + */ + private def processConfigChanges(notifications: Seq[String]) { + if (notifications.size > 0) { + info("Processing %d topic config change notification(s)...".format(notifications.size)) + val now = time.milliseconds + val logs = logManager.logsByTopicPartition.toBuffer + val logsByTopic = logs.groupBy(_._1.topic).mapValues(_.map(_._2)) + for (notification <- notifications) { + val changeId = changeNumber(notification) + if (changeId > lastExecutedChange) { + val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification + val (topicJson, stat) = ZkUtils.readData(zkClient, changeZnode) + val topic = topicJson.substring(1, topicJson.length - 1) // dequote + if (logsByTopic.contains(topic)) { + /* combine the default properties with the overrides in zk to create the new LogConfig */ + val props = new Properties(logManager.defaultConfig.toProps) + props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) + val logConfig = LogConfig.fromProps(props) + for (log <- logsByTopic(topic)) + log.config = logConfig + lastExecutedChange = changeId + info("Processed topic config change %d for topic %s, setting new config to %s.".format(changeId, topic, props)) + } else if (now - stat.getCtime > changeExpirationMs) { + /* this change is now obsolete, try to delete it unless it is the last change left */ + ZkUtils.deletePath(zkClient, changeZnode) + } + } + } + } + } + + /* get the change number from a change notification znode */ + private def changeNumber(name: String): Long = name.substring(AdminUtils.TopicConfigChangeZnodePrefix.length).toLong + + /** + * A listener that applies config changes to logs + */ + object ConfigChangeListener extends IZkChildListener { + override def handleChildChange(path: String, chillins: java.util.List[String]) { + try { + import JavaConversions._ + processConfigChanges(chillins: mutable.Buffer[String]) + } catch { + case e: Exception => error("Error processing config change:", e) + } + } + } + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala index 33b73609b1178..b189619bdc1b0 100644 --- a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala +++ b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala @@ -17,7 +17,7 @@ package kafka.server import kafka.utils.ZkUtils._ -import kafka.utils.{Utils, SystemTime, Logging} +import kafka.utils.{Json, Utils, SystemTime, Logging} import org.I0Itec.zkclient.exception.ZkNodeExistsException import org.I0Itec.zkclient.IZkDataListener import kafka.controller.ControllerContext @@ -30,7 +30,10 @@ import kafka.common.KafkaException * leader is dead, this class will handle automatic re-election and if it succeeds, it invokes the leader state change * callback */ -class ZookeeperLeaderElector(controllerContext: ControllerContext, electionPath: String, onBecomingLeader: () => Unit, +class ZookeeperLeaderElector(controllerContext: ControllerContext, + electionPath: String, + onBecomingLeader: () => Unit, + onResigningAsLeader: () => Unit, brokerId: Int) extends LeaderElector with Logging { var leaderId = -1 @@ -49,9 +52,7 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, electionPath: def elect: Boolean = { 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)) + val electString = Json.encode(Map("version" -> 1, "brokerid" -> brokerId, "timestamp" -> timestamp)) try { createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, electionPath, electString, brokerId, @@ -60,23 +61,22 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, electionPath: info(brokerId + " successfully elected as leader") leaderId = brokerId onBecomingLeader() - } 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 - } + } 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: Throwable => - error("Error while electing or becoming leader on broker %d".format(brokerId), e2) - leaderId = -1 + } + if (leaderId != -1) + debug("Broker %d was elected as leader instead of broker %d".format(leaderId, brokerId)) + case e2: Throwable => + error("Error while electing or becoming leader on broker %d".format(brokerId), e2) + resign() } - amILeader } @@ -118,6 +118,8 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, electionPath: controllerContext.controllerLock synchronized { debug("%s leader change listener fired for path %s to handle data deleted: trying to elect as a leader" .format(brokerId, dataPath)) + if(amILeader) + onResigningAsLeader() elect } } diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index dc3dcdf7e0b12..14f44d9305f34 100644 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -77,9 +77,9 @@ object DumpLogSegments { } } nonConsecutivePairsForLogFilesMap.foreach { - case (fileName, listOfNonSecutivePairs) => { + case (fileName, listOfNonConsecutivePairs) => { System.err.println("Non-secutive offsets in :" + fileName) - listOfNonSecutivePairs.foreach(m => { + listOfNonConsecutivePairs.foreach(m => { System.err.println(" %d is followed by %d".format(m._1, m._2)) }) } @@ -94,16 +94,16 @@ object DumpLogSegments { val startOffset = file.getName().split("\\.")(0).toLong val logFileName = file.getAbsolutePath.split("\\.")(0) + Log.LogFileSuffix val logFile = new File(logFileName) - val messageSet = new FileMessageSet(logFile) + val messageSet = new FileMessageSet(logFile, false) val index = new OffsetIndex(file = file, baseOffset = startOffset) for(i <- 0 until index.entries) { val entry = index.entry(i) val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, maxMessageSize) val messageAndOffset = getIterator(partialFileMessageSet.head, isDeepIteration = true).next() if(messageAndOffset.offset != entry.offset + index.baseOffset) { - var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getName, List[(Long, Long)]()) + var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]()) misMatchesSeq ::=(entry.offset + index.baseOffset, messageAndOffset.offset) - misMatchesForIndexFilesMap.put(file.getName, misMatchesSeq) + misMatchesForIndexFilesMap.put(file.getAbsolutePath, misMatchesSeq) } // since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one if(entry.offset == 0 && i > 0) @@ -120,7 +120,7 @@ object DumpLogSegments { isDeepIteration: Boolean) { val startOffset = file.getName().split("\\.")(0).toLong println("Starting offset: " + startOffset) - val messageSet = new FileMessageSet(file) + val messageSet = new FileMessageSet(file, false) var validBytes = 0L var lastOffset = -1l for(shallowMessageAndOffset <- messageSet) { // this only does shallow iteration @@ -132,9 +132,9 @@ object DumpLogSegments { lastOffset = messageAndOffset.offset // If we are iterating uncompressed messages, offsets must be consecutive else if (msg.compressionCodec == NoCompressionCodec && messageAndOffset.offset != lastOffset +1) { - var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getName, List[(Long, Long)]()) + var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]()) nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset) - nonConsecutivePairsForLogFilesMap.put(file.getName, nonConsecutivePairsSeq) + nonConsecutivePairsForLogFilesMap.put(file.getAbsolutePath, nonConsecutivePairsSeq) } lastOffset = messageAndOffset.offset @@ -146,7 +146,8 @@ object DumpLogSegments { if(printContents) { if(msg.hasKey) print(" key: " + Utils.readString(messageAndOffset.message.key, "UTF-8")) - print(" payload: " + Utils.readString(messageAndOffset.message.payload, "UTF-8")) + val payload = if(messageAndOffset.message.isNull) null else Utils.readString(messageAndOffset.message.payload, "UTF-8") + print(" payload: " + payload) } println() } diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 2b9438ae3a4da..fba652e3716a6 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -20,28 +20,29 @@ package kafka.tools import kafka.consumer._ import joptsimple._ -import java.net.URI import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} import kafka.common.TopicAndPartition +import kafka.client.ClientUtils +import kafka.utils.CommandLineUtils object GetOffsetShell { def main(args: Array[String]): Unit = { val parser = new OptionParser - val urlOpt = parser.accepts("server", "REQUIRED: The hostname of the server to connect to.") + val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.") .withRequiredArg - .describedAs("kafka://hostname:port") + .describedAs("hostname:port,...,hostname:port") .ofType(classOf[String]) val topicOpt = parser.accepts("topic", "REQUIRED: The topic to get offset from.") .withRequiredArg .describedAs("topic") .ofType(classOf[String]) - val partitionOpt = parser.accepts("partition", "partition id") + val partitionOpt = parser.accepts("partitions", "comma separated list of partition ids. If not specified, it will find offsets for all partitions") .withRequiredArg - .describedAs("partition id") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) + .describedAs("partition ids") + .ofType(classOf[String]) + .defaultsTo("") val timeOpt = parser.accepts("time", "timestamp of the offsets before that") .withRequiredArg .describedAs("timestamp/-1(latest)/-2(earliest)") @@ -51,28 +52,52 @@ object GetOffsetShell { .describedAs("count") .ofType(classOf[java.lang.Integer]) .defaultsTo(1) + val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.") + .withRequiredArg + .describedAs("ms") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1000) val options = parser.parse(args : _*) - for(arg <- List(urlOpt, topicOpt, timeOpt)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } + CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt, topicOpt, timeOpt) - val url = new URI(options.valueOf(urlOpt)) + val clientId = "GetOffsetShell" + val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt)) val topic = options.valueOf(topicOpt) - val partition = options.valueOf(partitionOpt).intValue + var partitionList = options.valueOf(partitionOpt) var time = options.valueOf(timeOpt).longValue val nOffsets = options.valueOf(nOffsetsOpt).intValue - val consumer = new SimpleConsumer(url.getHost, url.getPort, 10000, 100000, "GetOffsetShell") - val topicAndPartition = TopicAndPartition(topic, partition) - val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) - val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets - println("get " + offsets.length + " results") - for (offset <- offsets) - println(offset) + val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue() + + val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, clientId, maxWaitMs).topicsMetadata + if(topicsMetadata.size != 1 || !topicsMetadata(0).topic.equals(topic)) { + System.err.println(("Error: no valid topic metadata for topic: %s, " + " probably the topic does not exist, run ").format(topic) + + "kafka-list-topic.sh to verify") + System.exit(1) + } + val partitions = + if(partitionList == "") { + topicsMetadata.head.partitionsMetadata.map(_.partitionId) + } else { + partitionList.split(",").map(_.toInt).toSeq + } + partitions.foreach { partitionId => + val partitionMetadataOpt = topicsMetadata.head.partitionsMetadata.find(_.partitionId == partitionId) + partitionMetadataOpt match { + case Some(metadata) => + metadata.leader match { + case Some(leader) => + val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, clientId) + val topicAndPartition = TopicAndPartition(topic, partitionId) + val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) + val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets + + println("%s:%d:%s".format(topic, partitionId, offsets.mkString(","))) + case None => System.err.println("Error: partition %d does not have a leader. Skip getting offsets".format(partitionId)) + } + case None => System.err.println("Error: partition %d does not exist".format(partitionId)) + } + } } } diff --git a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java index 3c18286182f3a..7909d255ff47a 100644 --- a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java +++ b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java @@ -57,8 +57,9 @@ * The user need to provide the configuration file for 0.7 consumer and 0.8 producer. For 0.8 producer, * the "serializer.class" config is set to "kafka.serializer.DefaultEncoder" by the code. */ - -public class KafkaMigrationTool { +@SuppressWarnings({"unchecked", "rawtypes"}) +public class KafkaMigrationTool +{ private static final org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(KafkaMigrationTool.class.getName()); private static final String KAFKA_07_STATIC_CONSUMER_CLASS_NAME = "kafka.consumer.Consumer"; private static final String KAFKA_07_CONSUMER_CONFIG_CLASS_NAME = "kafka.consumer.ConsumerConfig"; diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala new file mode 100644 index 0000000000000..5e8c56d963174 --- /dev/null +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -0,0 +1,393 @@ +/** + * 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.tools + +import joptsimple.OptionParser +import kafka.cluster.Broker +import kafka.message.{MessageSet, MessageAndOffset, ByteBufferMessageSet} +import java.util.concurrent.CountDownLatch +import java.util.concurrent.atomic.AtomicReference +import kafka.client.ClientUtils +import java.util.regex.{PatternSyntaxException, Pattern} +import kafka.api._ +import java.text.SimpleDateFormat +import java.util.Date +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.utils._ +import kafka.consumer.{ConsumerConfig, Whitelist, SimpleConsumer} + +/** + * For verifying the consistency among replicas. + * + * 1. start a fetcher on every broker. + * 2. each fetcher does the following + * 2.1 issues fetch request + * 2.2 puts the fetched result in a shared buffer + * 2.3 waits for all other fetchers to finish step 2.2 + * 2.4 one of the fetchers verifies the consistency of fetched results among replicas + * + * The consistency verification is up to the high watermark. The tool reports the + * max lag between the verified offset and the high watermark among all partitions. + * + * If a broker goes down, the verification of the partitions on that broker is delayed + * until the broker is up again. + * + * Caveats: + * 1. The tools needs all brokers to be up at startup time. + * 2. The tool doesn't handle out of range offsets. + */ + +object ReplicaVerificationTool extends Logging { + val clientId= "replicaVerificationTool" + val dateFormatString = "yyyy-MM-dd HH:mm:ss,SSS" + val dateFormat = new SimpleDateFormat(dateFormatString) + + def getCurrentTimeString() = { + ReplicaVerificationTool.dateFormat.format(new Date(SystemTime.milliseconds)) + } + + def main(args: Array[String]): Unit = { + val parser = new OptionParser + val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.") + .withRequiredArg + .describedAs("hostname:port,...,hostname:port") + .ofType(classOf[String]) + val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.") + .withRequiredArg + .describedAs("bytes") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(ConsumerConfig.FetchSize) + val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.") + .withRequiredArg + .describedAs("ms") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1000) + val topicWhiteListOpt = parser.accepts("topic-white-list", "White list of topics to verify replica consistency. Defaults to all topics.") + .withRequiredArg + .describedAs("Java regex (String)") + .ofType(classOf[String]) + .defaultsTo(".*") + val initialOffsetTimeOpt = parser.accepts("time", "Timestamp for getting the initial offsets.") + .withRequiredArg + .describedAs("timestamp/-1(latest)/-2(earliest)") + .ofType(classOf[java.lang.Long]) + .defaultsTo(-1L) + val reportIntervalOpt = parser.accepts("report-interval-ms", "The reporting interval.") + .withRequiredArg + .describedAs("ms") + .ofType(classOf[java.lang.Long]) + .defaultsTo(30 * 1000L) + + + val options = parser.parse(args : _*) + CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt) + + val regex = options.valueOf(topicWhiteListOpt) + val topicWhiteListFiler = new Whitelist(regex) + + try { + Pattern.compile(regex) + } + catch { + case e: PatternSyntaxException => + throw new RuntimeException(regex + " is an invalid regex.") + } + + val fetchSize = options.valueOf(fetchSizeOpt).intValue + val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue + val initialOffsetTime = options.valueOf(initialOffsetTimeOpt).longValue + val reportInterval = options.valueOf(reportIntervalOpt).longValue + // getting topic metadata + info("Getting topic metatdata...") + val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt)) + val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, clientId, maxWaitMs) + val brokerMap = topicsMetadataResponse.extractBrokers(topicsMetadataResponse.topicsMetadata) + val filteredTopicMetadata = topicsMetadataResponse.topicsMetadata.filter( + topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic)) true else false + ) + val topicPartitionReplicaList: Seq[TopicPartitionReplica] = filteredTopicMetadata.flatMap( + topicMetadataResponse => + topicMetadataResponse.partitionsMetadata.flatMap( + partitionMetadata => + partitionMetadata.replicas.map(broker => + TopicPartitionReplica(topic = topicMetadataResponse.topic, partitionId = partitionMetadata.partitionId, replicaId = broker.id)) + ) + ) + debug("Selected topic partitions: " + topicPartitionReplicaList) + val topicAndPartitionsPerBroker: Map[Int, Seq[TopicAndPartition]] = topicPartitionReplicaList.groupBy(_.replicaId) + .map { case (brokerId, partitions) => + brokerId -> partitions.map { case partition => new TopicAndPartition(partition.topic, partition.partitionId) } } + debug("Topic partitions per broker: " + topicAndPartitionsPerBroker) + val expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int] = + topicPartitionReplicaList.groupBy(replica => new TopicAndPartition(replica.topic, replica.partitionId)) + .map { case (topicAndPartition, replicaSet) => topicAndPartition -> replicaSet.size } + debug("Expected replicas per topic partition: " + expectedReplicasPerTopicAndPartition) + val leadersPerBroker: Map[Int, Seq[TopicAndPartition]] = filteredTopicMetadata.flatMap( + topicMetadataResponse => + topicMetadataResponse.partitionsMetadata.map( + partitionMetadata => + (new TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id)) + ).groupBy(_._2) + .mapValues(topicAndPartitionAndLeaderIds => topicAndPartitionAndLeaderIds.map { + case(topicAndPartition, leaderId) => topicAndPartition }) + debug("Leaders per broker: " + leadersPerBroker) + + val replicaBuffer = new ReplicaBuffer(expectedReplicasPerTopicAndPartition, + leadersPerBroker, + topicAndPartitionsPerBroker.size, + brokerMap, + initialOffsetTime, + reportInterval) + // create all replica fetcher threads + val verificationBrokerId = topicAndPartitionsPerBroker.head._1 + val fetcherThreads: Iterable[ReplicaFetcher] = topicAndPartitionsPerBroker.map { + case (brokerId, topicAndPartitions) => + new ReplicaFetcher(name = "ReplicaFetcher-" + brokerId, + sourceBroker = brokerMap(brokerId), + topicAndPartitions = topicAndPartitions, + replicaBuffer = replicaBuffer, + socketTimeout = 30000, + socketBufferSize = 256000, + fetchSize = fetchSize, + maxWait = maxWaitMs, + minBytes = 1, + doVerification = (brokerId == verificationBrokerId)) + } + + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run() { + info("Stopping all fetchers") + fetcherThreads.foreach(_.shutdown()) + } + }) + fetcherThreads.foreach(_.start()) + println(ReplicaVerificationTool.getCurrentTimeString() + ": verification process is started.") + + } +} + +private case class TopicPartitionReplica(topic: String, partitionId: Int, replicaId: Int) + +private case class ReplicaAndMessageIterator(replicaId: Int, iterator: Iterator[MessageAndOffset]) + +private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, checksum: Long) + +private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int], + leadersPerBroker: Map[Int, Seq[TopicAndPartition]], + expectedNumFetchers: Int, + brokerMap: Map[Int, Broker], + initialOffsetTime: Long, + reportInterval: Long) extends Logging { + private val fetchOffsetMap = new Pool[TopicAndPartition, Long] + private val messageSetCache = new Pool[TopicAndPartition, Pool[Int, FetchResponsePartitionData]] + private val fetcherBarrier = new AtomicReference(new CountDownLatch(expectedNumFetchers)) + private val verificationBarrier = new AtomicReference(new CountDownLatch(1)) + @volatile private var lastReportTime = SystemTime.milliseconds + private var maxLag: Long = -1L + private var offsetWithMaxLag: Long = -1L + private var maxLagTopicAndPartition: TopicAndPartition = null + initialize() + + def createNewFetcherBarrier() { + fetcherBarrier.set(new CountDownLatch(expectedNumFetchers)) + } + + def getFetcherBarrier() = fetcherBarrier.get() + + def createNewVerificationBarrier() { + verificationBarrier.set(new CountDownLatch(1)) + } + + def getVerificationBarrier() = verificationBarrier.get() + + private def initialize() { + for (topicAndPartition <- expectedReplicasPerTopicAndPartition.keySet) + messageSetCache.put(topicAndPartition, new Pool[Int, FetchResponsePartitionData]) + setInitialOffsets() + } + + private def offsetResponseStringWithError(offsetResponse: OffsetResponse): String = { + offsetResponse.partitionErrorAndOffsets.filter { + case (topicAndPartition, partitionOffsetsResponse) => partitionOffsetsResponse.error != ErrorMapping.NoError + }.mkString + } + + private def setInitialOffsets() { + for ((brokerId, topicAndPartitions) <- leadersPerBroker) { + val broker = brokerMap(brokerId) + val consumer = new SimpleConsumer(broker.host, broker.port, 10000, 100000, ReplicaVerificationTool.clientId) + val initialOffsetMap: Map[TopicAndPartition, PartitionOffsetRequestInfo] = + topicAndPartitions.map(topicAndPartition => topicAndPartition -> PartitionOffsetRequestInfo(initialOffsetTime, 1)).toMap + val offsetRequest = OffsetRequest(initialOffsetMap) + val offsetResponse = consumer.getOffsetsBefore(offsetRequest) + assert(!offsetResponse.hasError, offsetResponseStringWithError(offsetResponse)) + offsetResponse.partitionErrorAndOffsets.foreach{ + case (topicAndPartition, partitionOffsetResponse) => + fetchOffsetMap.put(topicAndPartition, partitionOffsetResponse.offsets.head) + } + } + } + + def addFetchedData(topicAndPartition: TopicAndPartition, replicaId: Int, partitionData: FetchResponsePartitionData) { + messageSetCache.get(topicAndPartition).put(replicaId, partitionData) + } + + def getOffset(topicAndPartition: TopicAndPartition) = { + fetchOffsetMap.get(topicAndPartition) + } + + def verifyCheckSum() { + debug("Begin verification") + maxLag = -1L + for ((topicAndPartition, fetchResponsePerReplica) <- messageSetCache) { + debug("Verifying " + topicAndPartition) + assert(fetchResponsePerReplica.size == expectedReplicasPerTopicAndPartition(topicAndPartition), + "fetched " + fetchResponsePerReplica.size + " replicas for " + topicAndPartition + ", but expected " + + expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas") + val messageIteratorMap = fetchResponsePerReplica.map { + case(replicaId, fetchResponse) => + replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].shallowIterator} + val maxHw = fetchResponsePerReplica.values.map(_.hw).max + + // Iterate one message at a time from every replica, until high watermark is reached. + var isMessageInAllReplicas = true + while (isMessageInAllReplicas) { + var messageInfoFromFirstReplicaOpt: Option[MessageInfo] = None + for ( (replicaId, messageIterator) <- messageIteratorMap) { + try { + if (messageIterator.hasNext) { + val messageAndOffset = messageIterator.next() + + // only verify up to the high watermark + if (messageAndOffset.offset >= fetchResponsePerReplica.get(replicaId).hw) + isMessageInAllReplicas = false + else { + messageInfoFromFirstReplicaOpt match { + case None => + messageInfoFromFirstReplicaOpt = Some( + MessageInfo(replicaId, messageAndOffset.offset,messageAndOffset.nextOffset, messageAndOffset.message.checksum)) + case Some(messageInfoFromFirstReplica) => + if (messageInfoFromFirstReplica.offset != messageAndOffset.offset) { + println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicAndPartition + + ": replica " + messageInfoFromFirstReplica.replicaId + "'s offset " + + messageInfoFromFirstReplica.offset + " doesn't match replica " + + replicaId + "'s offset " + messageAndOffset.offset) + System.exit(1) + } + if (messageInfoFromFirstReplica.checksum != messageAndOffset.message.checksum) + println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + + topicAndPartition + " has unmatched checksum at offset " + messageAndOffset.offset + "; replica " + + messageInfoFromFirstReplica.replicaId + "'s checksum " + messageInfoFromFirstReplica.checksum + + "; replica " + replicaId + "'s checksum " + messageAndOffset.message.checksum) + } + } + } else + isMessageInAllReplicas = false + } catch { + case t => + throw new RuntimeException("Error in processing replica %d in partition %s at offset %d." + .format(replicaId, topicAndPartition, fetchOffsetMap.get(topicAndPartition)), t) + } + } + if (isMessageInAllReplicas) { + val nextOffset = messageInfoFromFirstReplicaOpt.get.nextOffset + fetchOffsetMap.put(topicAndPartition, nextOffset) + debug(expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas match at offset " + + nextOffset + " for " + topicAndPartition) + } + } + if (maxHw - fetchOffsetMap.get(topicAndPartition) > maxLag) { + offsetWithMaxLag = fetchOffsetMap.get(topicAndPartition) + maxLag = maxHw - offsetWithMaxLag + maxLagTopicAndPartition = topicAndPartition + } + fetchResponsePerReplica.clear() + } + val currentTimeMs = SystemTime.milliseconds + if (currentTimeMs - lastReportTime > reportInterval) { + println(ReplicaVerificationTool.dateFormat.format(new Date(currentTimeMs)) + ": max lag is " + + maxLag + " for partition " + maxLagTopicAndPartition + " at offset " + offsetWithMaxLag + + " among " + messageSetCache.size + " paritions") + lastReportTime = currentTimeMs + } + } +} + +private class ReplicaFetcher(name: String, sourceBroker: Broker, topicAndPartitions: Iterable[TopicAndPartition], + replicaBuffer: ReplicaBuffer, socketTimeout: Int, socketBufferSize: Int, + fetchSize: Int, maxWait: Int, minBytes: Int, doVerification: Boolean) + extends ShutdownableThread(name) { + val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, ReplicaVerificationTool.clientId) + val fetchRequestBuilder = new FetchRequestBuilder(). + clientId(ReplicaVerificationTool.clientId). + replicaId(Request.DebuggingConsumerId). + maxWait(maxWait). + minBytes(minBytes) + + override def doWork() { + + val fetcherBarrier = replicaBuffer.getFetcherBarrier() + val verificationBarrier = replicaBuffer.getVerificationBarrier() + + for (topicAndPartition <- topicAndPartitions) + fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition, + replicaBuffer.getOffset(topicAndPartition), fetchSize) + + val fetchRequest = fetchRequestBuilder.build() + debug("Issuing fetch request " + fetchRequest) + + var response: FetchResponse = null + try { + response = simpleConsumer.fetch(fetchRequest) + } catch { + case t: Throwable => + if (!isRunning.get) + throw t + } + + if (response != null) { + response.data.foreach { + case(topicAndPartition, partitionData) => + replicaBuffer.addFetchedData(topicAndPartition, sourceBroker.id, partitionData) + } + } else { + for (topicAndPartition <- topicAndPartitions) + replicaBuffer.addFetchedData(topicAndPartition, sourceBroker.id, new FetchResponsePartitionData(messages = MessageSet.Empty)) + } + + fetcherBarrier.countDown() + debug("Done fetching") + + // wait for all fetchers to finish + fetcherBarrier.await() + debug("Ready for verification") + + // one of the fetchers will do the verification + if (doVerification) { + debug("Do verification") + replicaBuffer.verifyCheckSum() + replicaBuffer.createNewFetcherBarrier() + replicaBuffer.createNewVerificationBarrier() + debug("Created new barrier") + verificationBarrier.countDown() + } + + verificationBarrier.await() + debug("Done verification") + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index c889835b88825..747e07280cce7 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -214,7 +214,7 @@ object SimpleConsumerShell extends Logging { System.out.println("next offset = " + offset) val message = messageAndOffset.message val key = if(message.hasKey) Utils.readBytes(message.key) else null - formatter.writeTo(key, Utils.readBytes(message.payload), System.out) + formatter.writeTo(key, if(message.isNull) null else Utils.readBytes(message.payload), System.out) numMessagesConsumed += 1 } catch { case e: Throwable => diff --git a/core/src/main/scala/kafka/utils/CommandLineUtils.scala b/core/src/main/scala/kafka/utils/CommandLineUtils.scala index 0018fb24dfcde..5f563ca93ed5c 100644 --- a/core/src/main/scala/kafka/utils/CommandLineUtils.scala +++ b/core/src/main/scala/kafka/utils/CommandLineUtils.scala @@ -26,7 +26,7 @@ object CommandLineUtils extends Logging { def checkRequiredArgs(parser: OptionParser, options: OptionSet, required: OptionSpec[_]*) { for(arg <- required) { if(!options.has(arg)) { - error("Missing required argument \"" + arg + "\"") + System.err.println("Missing required argument \"" + arg + "\"") parser.printHelpOn(System.err) System.exit(1) } diff --git a/core/src/main/scala/kafka/utils/Crc32.java b/core/src/main/scala/kafka/utils/Crc32.java new file mode 100644 index 0000000000000..af9fe0d7d4ab2 --- /dev/null +++ b/core/src/main/scala/kafka/utils/Crc32.java @@ -0,0 +1,637 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.utils; + +import java.util.zip.Checksum; + +/** + * This class was taken from Hadoop org.apache.hadoop.util.PureJavaCrc32 + * + * A pure-java implementation of the CRC32 checksum that uses + * the same polynomial as the built-in native CRC32. + * + * This is to avoid the JNI overhead for certain uses of Checksumming + * where many small pieces of data are checksummed in succession. + * + * The current version is ~10x to 1.8x as fast as Sun's native + * java.util.zip.CRC32 in Java 1.6 + * + * @see java.util.zip.CRC32 + */ +public class Crc32 implements Checksum { + + /** the current CRC value, bit-flipped */ + private int crc; + + /** Create a new PureJavaCrc32 object. */ + public Crc32() { + reset(); + } + + @Override + public long getValue() { + return (~crc) & 0xffffffffL; + } + + @Override + public void reset() { + crc = 0xffffffff; + } + + @Override + public void update(byte[] b, int off, int len) { + int localCrc = crc; + + while(len > 7) { + final int c0 =(b[off+0] ^ localCrc) & 0xff; + final int c1 =(b[off+1] ^ (localCrc >>>= 8)) & 0xff; + final int c2 =(b[off+2] ^ (localCrc >>>= 8)) & 0xff; + final int c3 =(b[off+3] ^ (localCrc >>>= 8)) & 0xff; + localCrc = (T[T8_7_start + c0] ^ T[T8_6_start + c1]) + ^ (T[T8_5_start + c2] ^ T[T8_4_start + c3]); + + final int c4 = b[off+4] & 0xff; + final int c5 = b[off+5] & 0xff; + final int c6 = b[off+6] & 0xff; + final int c7 = b[off+7] & 0xff; + + localCrc ^= (T[T8_3_start + c4] ^ T[T8_2_start + c5]) + ^ (T[T8_1_start + c6] ^ T[T8_0_start + c7]); + + off += 8; + len -= 8; + } + + /* loop unroll - duff's device style */ + switch(len) { + case 7: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + case 6: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + case 5: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + case 4: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + case 3: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + case 2: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + case 1: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + default: + /* nothing */ + } + + // Publish crc out to object + crc = localCrc; + } + + @Override + final public void update(int b) { + crc = (crc >>> 8) ^ T[T8_0_start + ((crc ^ b) & 0xff)]; + } + + /* + * CRC-32 lookup tables generated by the polynomial 0xEDB88320. + * See also TestPureJavaCrc32.Table. + */ + private static final int T8_0_start = 0*256; + private static final int T8_1_start = 1*256; + private static final int T8_2_start = 2*256; + private static final int T8_3_start = 3*256; + private static final int T8_4_start = 4*256; + private static final int T8_5_start = 5*256; + private static final int T8_6_start = 6*256; + private static final int T8_7_start = 7*256; + + private static final int[] T = new int[] { + /* T8_0 */ + 0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, + 0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, + 0x0EDB8832, 0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, + 0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, + 0x1DB71064, 0x6AB020F2, 0xF3B97148, 0x84BE41DE, + 0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, + 0x136C9856, 0x646BA8C0, 0xFD62F97A, 0x8A65C9EC, + 0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, + 0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172, + 0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, + 0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, + 0x32D86CE3, 0x45DF5C75, 0xDCD60DCF, 0xABD13D59, + 0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, + 0x21B4F4B5, 0x56B3C423, 0xCFBA9599, 0xB8BDA50F, + 0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, + 0x2F6F7C87, 0x58684C11, 0xC1611DAB, 0xB6662D3D, + 0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, + 0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433, + 0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, + 0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, + 0x6B6B51F4, 0x1C6C6162, 0x856530D8, 0xF262004E, + 0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, + 0x65B0D9C6, 0x12B7E950, 0x8BBEB8EA, 0xFCB9887C, + 0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, + 0x4DB26158, 0x3AB551CE, 0xA3BC0074, 0xD4BB30E2, + 0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, + 0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0, + 0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, + 0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, + 0x5768B525, 0x206F85B3, 0xB966D409, 0xCE61E49F, + 0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, + 0x59B33D17, 0x2EB40D81, 0xB7BD5C3B, 0xC0BA6CAD, + 0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, + 0xEAD54739, 0x9DD277AF, 0x04DB2615, 0x73DC1683, + 0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, + 0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1, + 0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, + 0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, + 0xFED41B76, 0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, + 0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, + 0xD6D6A3E8, 0xA1D1937E, 0x38D8C2C4, 0x4FDFF252, + 0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, + 0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6, 0x41047A60, + 0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, + 0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236, + 0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, + 0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, + 0xC2D7FFA7, 0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, + 0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, + 0x9C0906A9, 0xEB0E363F, 0x72076785, 0x05005713, + 0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, + 0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7, 0x0BDBDF21, + 0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, + 0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777, + 0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, + 0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, + 0xA00AE278, 0xD70DD2EE, 0x4E048354, 0x3903B3C2, + 0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, + 0xAED16A4A, 0xD9D65ADC, 0x40DF0B66, 0x37D83BF0, + 0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, + 0xBDBDF21C, 0xCABAC28A, 0x53B39330, 0x24B4A3A6, + 0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, + 0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94, + 0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D, + /* T8_1 */ + 0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, + 0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, + 0xC8D98A08, 0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, + 0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, + 0x4AC21251, 0x53D92310, 0x78F470D3, 0x61EF4192, + 0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, + 0x821B9859, 0x9B00A918, 0xB02DFADB, 0xA936CB9A, + 0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, + 0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761, + 0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, + 0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, + 0x39316BAE, 0x202A5AEF, 0x0B07092C, 0x121C386D, + 0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, + 0xBB2AF3F7, 0xA231C2B6, 0x891C9175, 0x9007A034, + 0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, + 0x73F379FF, 0x6AE848BE, 0x41C51B7D, 0x58DE2A3C, + 0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, + 0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2, + 0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, + 0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, + 0xBABB5D54, 0xA3A06C15, 0x888D3FD6, 0x91960E97, + 0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, + 0x7262D75C, 0x6B79E61D, 0x4054B5DE, 0x594F849F, + 0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, + 0x65FD6BA7, 0x7CE65AE6, 0x57CB0925, 0x4ED03864, + 0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, + 0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C, + 0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, + 0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, + 0x4B53BCF2, 0x52488DB3, 0x7965DE70, 0x607EEF31, + 0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, + 0x838A36FA, 0x9A9107BB, 0xB1BC5478, 0xA8A76539, + 0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, + 0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD, 0x74C20E8C, + 0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, + 0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484, + 0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, + 0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, + 0xB9980012, 0xA0833153, 0x8BAE6290, 0x92B553D1, + 0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, + 0xAE07BCE9, 0xB71C8DA8, 0x9C31DE6B, 0x852AEF2A, + 0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, + 0x66DE36E1, 0x7FC507A0, 0x54E85463, 0x4DF36522, + 0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, + 0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B, + 0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, + 0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, + 0x4870E1B4, 0x516BD0F5, 0x7A468336, 0x635DB277, + 0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, + 0xAF96124A, 0xB68D230B, 0x9DA070C8, 0x84BB4189, + 0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, + 0x674F9842, 0x7E54A903, 0x5579FAC0, 0x4C62CB81, + 0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, + 0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8, + 0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, + 0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, + 0x5E7EF3EC, 0x4765C2AD, 0x6C48916E, 0x7553A02F, + 0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, + 0x96A779E4, 0x8FBC48A5, 0xA4911B66, 0xBD8A2A27, + 0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, + 0x14BCE1BD, 0x0DA7D0FC, 0x268A833F, 0x3F91B27E, + 0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, + 0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876, + 0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72, + /* T8_2 */ + 0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, + 0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, + 0x0E1351B8, 0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, + 0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, + 0x1C26A370, 0x1DE4C947, 0x1FA2771E, 0x1E601D29, + 0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, + 0x1235F2C8, 0x13F798FF, 0x11B126A6, 0x10734C91, + 0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, + 0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9, + 0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, + 0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, + 0x3157BF84, 0x3095D5B3, 0x32D36BEA, 0x331101DD, + 0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, + 0x23624D4C, 0x22A0277B, 0x20E69922, 0x2124F315, + 0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, + 0x2D711CF4, 0x2CB376C3, 0x2EF5C89A, 0x2F37A2AD, + 0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, + 0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45, + 0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, + 0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, + 0x6CBC2EB0, 0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, + 0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, + 0x62AF7F08, 0x636D153F, 0x612BAB66, 0x60E9C151, + 0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, + 0x48D7CB20, 0x4915A117, 0x4B531F4E, 0x4A917579, + 0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, + 0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1, + 0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, + 0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, + 0x53F8C08C, 0x523AAABB, 0x507C14E2, 0x51BE7ED5, + 0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, + 0x5DEB9134, 0x5C29FB03, 0x5E6F455A, 0x5FAD2F6D, + 0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, + 0xE63CB35C, 0xE7FED96B, 0xE5B86732, 0xE47A0D05, + 0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, + 0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD, + 0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, + 0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, + 0xF300E948, 0xF2C2837F, 0xF0843D26, 0xF1465711, + 0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, + 0xD9785D60, 0xD8BA3757, 0xDAFC890E, 0xDB3EE339, + 0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, + 0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6, 0xD52DB281, + 0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, + 0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049, + 0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, + 0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, + 0xCC440774, 0xCD866D43, 0xCFC0D31A, 0xCE02B92D, + 0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, + 0x96A63E9C, 0x976454AB, 0x9522EAF2, 0x94E080C5, + 0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, + 0x98B56F24, 0x99770513, 0x9B31BB4A, 0x9AF3D17D, + 0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, + 0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5, + 0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, + 0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, + 0xA9E2D0A0, 0xA820BA97, 0xAA6604CE, 0xABA46EF9, + 0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, + 0xA7F18118, 0xA633EB2F, 0xA4755576, 0xA5B73F41, + 0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, + 0xB5C473D0, 0xB40619E7, 0xB640A7BE, 0xB782CD89, + 0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, + 0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31, + 0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED, + /* T8_3 */ + 0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, + 0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, + 0xC5B428EF, 0x7D084F8A, 0x6FBDE064, 0xD7018701, + 0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, + 0x5019579F, 0xE8A530FA, 0xFA109F14, 0x42ACF871, + 0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, + 0x95AD7F70, 0x2D111815, 0x3FA4B7FB, 0x8718D09E, + 0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, + 0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0, + 0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, + 0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, + 0xEAE41086, 0x525877E3, 0x40EDD80D, 0xF851BF68, + 0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, + 0x7F496FF6, 0xC7F50893, 0xD540A77D, 0x6DFCC018, + 0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, + 0xBAFD4719, 0x0241207C, 0x10F48F92, 0xA848E8F7, + 0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, + 0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084, + 0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, + 0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, + 0xCB0D0FA2, 0x73B168C7, 0x6104C729, 0xD9B8A04C, + 0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, + 0x0EB9274D, 0xB6054028, 0xA4B0EFC6, 0x1C0C88A3, + 0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, + 0x3B26F703, 0x839A9066, 0x912F3F88, 0x299358ED, + 0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, + 0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002, + 0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, + 0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, + 0xE45D37CB, 0x5CE150AE, 0x4E54FF40, 0xF6E89825, + 0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, + 0x21E91F24, 0x99557841, 0x8BE0D7AF, 0x335CB0CA, + 0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, + 0x623B216C, 0xDA874609, 0xC832E9E7, 0x708E8E82, + 0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, + 0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D, + 0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, + 0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, + 0x78F4C94B, 0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, + 0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, + 0x4D6B1905, 0xF5D77E60, 0xE762D18E, 0x5FDEB6EB, + 0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, + 0x88DF31EA, 0x3063568F, 0x22D6F961, 0x9A6A9E04, + 0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, + 0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174, + 0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, + 0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, + 0x57A4F122, 0xEF189647, 0xFDAD39A9, 0x45115ECC, + 0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, + 0xF92F7951, 0x41931E34, 0x5326B1DA, 0xEB9AD6BF, + 0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, + 0x3C9B51BE, 0x842736DB, 0x96929935, 0x2E2EFE50, + 0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, + 0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120, + 0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, + 0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, + 0xD67F4138, 0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, + 0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, + 0x13CB69D7, 0xAB770EB2, 0xB9C2A15C, 0x017EC639, + 0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, + 0x866616A7, 0x3EDA71C2, 0x2C6FDE2C, 0x94D3B949, + 0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, + 0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6, + 0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1, + /* T8_4 */ + 0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, + 0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, + 0x30704BC1, 0x0D106271, 0x4AB018A1, 0x77D03111, + 0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, + 0x60E09782, 0x5D80BE32, 0x1A20C4E2, 0x2740ED52, + 0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, + 0x5090DC43, 0x6DF0F5F3, 0x2A508F23, 0x1730A693, + 0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, + 0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4, + 0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, + 0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, + 0x0431C205, 0x3951EBB5, 0x7EF19165, 0x4391B8D5, + 0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, + 0x54A11E46, 0x69C137F6, 0x2E614D26, 0x13016496, + 0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, + 0x64D15587, 0x59B17C37, 0x1E1106E7, 0x23712F57, + 0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, + 0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459, + 0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, + 0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, + 0x3813CFCB, 0x0573E67B, 0x42D39CAB, 0x7FB3B51B, + 0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, + 0x0863840A, 0x3503ADBA, 0x72A3D76A, 0x4FC3FEDA, + 0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, + 0x9932774D, 0xA4525EFD, 0xE3F2242D, 0xDE920D9D, + 0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, + 0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C, + 0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, + 0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, + 0x0C52460F, 0x31326FBF, 0x7692156F, 0x4BF23CDF, + 0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, + 0x3C220DCE, 0x0142247E, 0x46E25EAE, 0x7B82771E, + 0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, + 0x44661652, 0x79063FE2, 0x3EA64532, 0x03C66C82, + 0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, + 0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743, + 0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, + 0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, + 0xE1766CD1, 0xDC164561, 0x9BB63FB1, 0xA6D61601, + 0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, + 0x70279F96, 0x4D47B626, 0x0AE7CCF6, 0x3787E546, + 0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, + 0x4057D457, 0x7D37FDE7, 0x3A978737, 0x07F7AE87, + 0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, + 0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4, + 0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, + 0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, + 0xD537E515, 0xE857CCA5, 0xAFF7B675, 0x92979FC5, + 0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, + 0x1C954E1B, 0x21F567AB, 0x66551D7B, 0x5B3534CB, + 0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, + 0x2CE505DA, 0x11852C6A, 0x562556BA, 0x6B457F0A, + 0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, + 0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349, + 0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, + 0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, + 0x28D4C7DF, 0x15B4EE6F, 0x521494BF, 0x6F74BD0F, + 0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, + 0x18A48C1E, 0x25C4A5AE, 0x6264DF7E, 0x5F04F6CE, + 0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, + 0x4834505D, 0x755479ED, 0x32F4033D, 0x0F942A8D, + 0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, + 0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C, + 0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C, + /* T8_5 */ + 0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, + 0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, + 0xEC53826D, 0x270F51C8, 0xA19B2366, 0x6AC7F0C3, + 0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, + 0x03D6029B, 0xC88AD13E, 0x4E1EA390, 0x85427035, + 0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, + 0xEF8580F6, 0x24D95353, 0xA24D21FD, 0x6911F258, + 0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, + 0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798, + 0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, + 0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, + 0x706EC54D, 0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, + 0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, + 0x9FEB45BB, 0x54B7961E, 0xD223E4B0, 0x197F3715, + 0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, + 0x73B8C7D6, 0xB8E41473, 0x3E7066DD, 0xF52CB578, + 0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, + 0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4, + 0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, + 0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, + 0x0C8E08F7, 0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, + 0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, + 0xE0DD8A9A, 0x2B81593F, 0xAD152B91, 0x6649F834, + 0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, + 0x08F40F5A, 0xC3A8DCFF, 0x453CAE51, 0x8E607DF4, + 0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, + 0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99, + 0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, + 0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, + 0x90B34FD7, 0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, + 0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, + 0x7CE0CDBA, 0xB7BC1E1F, 0x31286CB1, 0xFA74BF14, + 0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, + 0x852156CE, 0x4E7D856B, 0xC8E9F7C5, 0x03B52460, + 0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, + 0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D, + 0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, + 0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, + 0xF135942E, 0x3A69478B, 0xBCFD3525, 0x77A1E680, + 0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, + 0x191C11EE, 0xD240C24B, 0x54D4B0E5, 0x9F886340, + 0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, + 0xF54F9383, 0x3E134026, 0xB8873288, 0x73DBE12D, + 0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, + 0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB, + 0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, + 0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, + 0x6D08D30E, 0xA65400AB, 0x20C07205, 0xEB9CA1A0, + 0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, + 0x8A795CA2, 0x41258F07, 0xC7B1FDA9, 0x0CED2E0C, + 0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, + 0x662ADECF, 0xAD760D6A, 0x2BE27FC4, 0xE0BEAC61, + 0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, + 0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97, + 0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, + 0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, + 0x16441B82, 0xDD18C827, 0x5B8CBA89, 0x90D0692C, + 0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, + 0xFA1799EF, 0x314B4A4A, 0xB7DF38E4, 0x7C83EB41, + 0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, + 0x15921919, 0xDECECABC, 0x585AB812, 0x93066BB7, + 0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, + 0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA, + 0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC, + /* T8_6 */ + 0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, + 0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, + 0x33EF4E67, 0x959845D3, 0xA4705F4E, 0x020754FA, + 0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, + 0x67DE9CCE, 0xC1A9977A, 0xF0418DE7, 0x56368653, + 0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, + 0x5431D2A9, 0xF246D91D, 0xC3AEC380, 0x65D9C834, + 0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, + 0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301, + 0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, + 0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, + 0x081D53E8, 0xAE6A585C, 0x9F8242C1, 0x39F54975, + 0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, + 0x5C2C8141, 0xFA5B8AF5, 0xCBB39068, 0x6DC49BDC, + 0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, + 0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F, 0x5E2BD5BB, + 0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, + 0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7, + 0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, + 0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, + 0x23D5E9B7, 0x85A2E203, 0xB44AF89E, 0x123DF32A, + 0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, + 0x103AA7D0, 0xB64DAC64, 0x87A5B6F9, 0x21D2BD4D, + 0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, + 0x8BB64CE5, 0x2DC14751, 0x1C295DCC, 0xBA5E5678, + 0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, + 0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F, + 0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, + 0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, + 0x1827F438, 0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, + 0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, + 0x2BC8BA5F, 0x8DBFB1EB, 0xBC57AB76, 0x1A20A0C2, + 0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, + 0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8, 0x4DB1D47C, + 0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, + 0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B, + 0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, + 0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, + 0xDC27385B, 0x7A5033EF, 0x4BB82972, 0xEDCF22C6, + 0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, + 0x47ABD36E, 0xE1DCD8DA, 0xD034C247, 0x7643C9F3, + 0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, + 0x74449D09, 0xD23396BD, 0xE3DB8C20, 0x45AC8794, + 0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, + 0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D, + 0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, + 0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, + 0xE7D525D4, 0x41A22E60, 0x704A34FD, 0xD63D3F49, + 0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, + 0x3852BB98, 0x9E25B02C, 0xAFCDAAB1, 0x09BAA105, + 0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, + 0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6, 0x3A55EF62, + 0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, + 0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB, + 0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, + 0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, + 0x03A0A617, 0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, + 0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, + 0x304FE870, 0x9638E3C4, 0xA7D0F959, 0x01A7F2ED, + 0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, + 0x647E3AD9, 0xC209316D, 0xF3E12BF0, 0x55962044, + 0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, + 0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23, + 0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30, + /* T8_7 */ + 0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, + 0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, + 0xD3E51BB5, 0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, + 0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, + 0x7CBB312B, 0xB01131B5, 0x3E9E3656, 0xF23436C8, + 0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, + 0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3, 0x21D12D7D, + 0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, + 0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5, + 0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, + 0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, + 0xAED97719, 0x62737787, 0xECFC7064, 0x205670FA, + 0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, + 0x01875D87, 0xCD2D5D19, 0x43A25AFA, 0x8F085A64, + 0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, + 0xD2624632, 0x1EC846AC, 0x9047414F, 0x5CED41D1, + 0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, + 0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4, + 0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, + 0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, + 0x5526F3C6, 0x998CF358, 0x1703F4BB, 0xDBA9F425, + 0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, + 0x86C3E873, 0x4A69E8ED, 0xC4E6EF0E, 0x084CEF90, + 0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, + 0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6, 0x5E64A758, + 0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, + 0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED, + 0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, + 0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, + 0x281A9F6A, 0xE4B09FF4, 0x6A3F9817, 0xA6959889, + 0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, + 0xFBFF84DF, 0x37558441, 0xB9DA83A2, 0x7570833C, + 0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, + 0xD7718B20, 0x1BDB8BBE, 0x95548C5D, 0x59FE8CC3, + 0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, + 0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776, + 0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, + 0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, + 0xFC65AF44, 0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, + 0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, + 0xAA4DE78C, 0x66E7E712, 0xE868E0F1, 0x24C2E06F, + 0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, + 0x79A8FC39, 0xB502FCA7, 0x3B8DFB44, 0xF727FBDA, + 0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, + 0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144, + 0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, + 0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, + 0x8159C3E8, 0x4DF3C376, 0xC37CC495, 0x0FD6C40B, + 0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, + 0xFEEC49CD, 0x32464953, 0xBCC94EB0, 0x70634E2E, + 0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, + 0x2D095278, 0xE1A352E6, 0x6F2C5505, 0xA386559B, + 0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, + 0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05, + 0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, + 0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, + 0x83D02561, 0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, + 0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, + 0x50353ED4, 0x9C9F3E4A, 0x121039A9, 0xDEBA3937, + 0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, + 0xFF6B144A, 0x33C114D4, 0xBD4E1337, 0x71E413A9, + 0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, + 0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C, + 0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6 + }; +} diff --git a/core/src/main/scala/kafka/utils/FileLock.scala b/core/src/main/scala/kafka/utils/FileLock.scala index 14f6ffba36f69..b43b4b1055874 100644 --- a/core/src/main/scala/kafka/utils/FileLock.scala +++ b/core/src/main/scala/kafka/utils/FileLock.scala @@ -25,56 +25,56 @@ import java.nio.channels._ * The given path will be created and opened if it doesn't exist. */ class FileLock(val file: File) extends Logging { - file.createNewFile() - private val channel = new RandomAccessFile(file, "rw").getChannel() - private var flock: java.nio.channels.FileLock = null - - /** - * Lock the file or throw an exception if the lock is already held - */ - def lock() { - this synchronized { - trace("Acquiring lock on " + file.getAbsolutePath) - flock = channel.lock() - } + file.createNewFile() // create the file if it doesn't exist + private val channel = new RandomAccessFile(file, "rw").getChannel() + private var flock: java.nio.channels.FileLock = null + + /** + * Lock the file or throw an exception if the lock is already held + */ + def lock() { + this synchronized { + trace("Acquiring lock on " + file.getAbsolutePath) + flock = channel.lock() } - - /** - * Try to lock the file and return true if the locking succeeds - */ - def tryLock(): Boolean = { - this synchronized { - trace("Acquiring lock on " + file.getAbsolutePath) - try { - // weirdly this method will return null if the lock is held by another - // process, but will throw an exception if the lock is held by this process - // so we have to handle both cases - flock = channel.tryLock() - flock != null - } catch { - case e: OverlappingFileLockException => false - } + } + + /** + * Try to lock the file and return true if the locking succeeds + */ + def tryLock(): Boolean = { + this synchronized { + trace("Acquiring lock on " + file.getAbsolutePath) + try { + // weirdly this method will return null if the lock is held by another + // process, but will throw an exception if the lock is held by this process + // so we have to handle both cases + flock = channel.tryLock() + flock != null + } catch { + case e: OverlappingFileLockException => false } } - - /** - * Unlock the lock if it is held - */ - def unlock() { - this synchronized { - trace("Releasing lock on " + file.getAbsolutePath) - if(flock != null) - flock.release() - } + } + + /** + * Unlock the lock if it is held + */ + def unlock() { + this synchronized { + trace("Releasing lock on " + file.getAbsolutePath) + if(flock != null) + flock.release() } - - /** - * Destroy this lock, closing the associated FileChannel - */ - def destroy() = { - this synchronized { - unlock() - channel.close() - } + } + + /** + * Destroy this lock, closing the associated FileChannel + */ + def destroy() = { + this synchronized { + unlock() + channel.close() } + } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/IteratorTemplate.scala b/core/src/main/scala/kafka/utils/IteratorTemplate.scala index 301f9346aff8e..fd952f3ec0f04 100644 --- a/core/src/main/scala/kafka/utils/IteratorTemplate.scala +++ b/core/src/main/scala/kafka/utils/IteratorTemplate.scala @@ -32,16 +32,21 @@ object FAILED extends State abstract class IteratorTemplate[T] extends Iterator[T] with java.util.Iterator[T] { private var state: State = NOT_READY - private var nextItem: Option[T] = None + private var nextItem = null.asInstanceOf[T] def next(): T = { if(!hasNext()) throw new NoSuchElementException() state = NOT_READY - nextItem match { - case Some(item) => item - case None => throw new IllegalStateException("Expected item but none found.") - } + if(nextItem == null) + throw new IllegalStateException("Expected item but none found.") + nextItem + } + + def peek(): T = { + if(!hasNext()) + throw new NoSuchElementException() + nextItem } def hasNext(): Boolean = { @@ -58,7 +63,7 @@ abstract class IteratorTemplate[T] extends Iterator[T] with java.util.Iterator[T def maybeComputeNext(): Boolean = { state = FAILED - nextItem = Some(makeNext()) + nextItem = makeNext() if(state == DONE) { false } else { diff --git a/core/src/main/scala/kafka/utils/Json.scala b/core/src/main/scala/kafka/utils/Json.scala index 03fb06fd8dba2..d1102844748f2 100644 --- a/core/src/main/scala/kafka/utils/Json.scala +++ b/core/src/main/scala/kafka/utils/Json.scala @@ -17,6 +17,7 @@ package kafka.utils import kafka.common._ +import scala.collection._ import util.parsing.json.JSON /** @@ -27,6 +28,9 @@ object Json extends Logging { JSON.globalNumberParser = myConversionFunc val lock = new Object + /** + * Parse a JSON string into an object + */ def parseFull(input: String): Option[Any] = { lock synchronized { try { @@ -37,4 +41,31 @@ object Json extends Logging { } } } + + /** + * Encode an object into a JSON string. This method accepts any type T where + * T => null | Boolean | String | Number | Map[String, T] | Array[T] | Iterable[T] + * Any other type will result in an exception. + * + * This method does not properly handle non-ascii characters. + */ + def encode(obj: Any): String = { + obj match { + case null => "null" + case b: Boolean => b.toString + case s: String => "\"" + s + "\"" + case n: Number => n.toString + case m: Map[_, _] => + "{" + + m.map(elem => + elem match { + case t: Tuple2[_,_] => encode(t._1) + ":" + encode(t._2) + case _ => throw new IllegalArgumentException("Invalid map element (" + elem + ") in " + obj) + }).mkString(",") + + "}" + case a: Array[_] => encode(a.toSeq) + case i: Iterable[_] => "[" + i.map(encode).mkString(",") + "]" + case other: AnyRef => throw new IllegalArgumentException("Unknown arguement of type " + other.getClass + ": " + other) + } + } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/KafkaScheduler.scala b/core/src/main/scala/kafka/utils/KafkaScheduler.scala index 5a222b816a14a..8e3750506c180 100644 --- a/core/src/main/scala/kafka/utils/KafkaScheduler.scala +++ b/core/src/main/scala/kafka/utils/KafkaScheduler.scala @@ -22,51 +22,97 @@ import atomic._ import collection.mutable.HashMap /** - * A scheduler for running jobs in the background + * A scheduler for running jobs + * + * This interface controls a job scheduler that allows scheduling either repeating background jobs + * that execute periodically or delayed one-time actions that are scheduled in the future. */ -class KafkaScheduler(val numThreads: Int) extends Logging { - private var executor:ScheduledThreadPoolExecutor = null - private val daemonThreadFactory = new ThreadFactory() { - def newThread(runnable: Runnable): Thread = Utils.newThread(runnable, true) - } - private val nonDaemonThreadFactory = new ThreadFactory() { - def newThread(runnable: Runnable): Thread = Utils.newThread(runnable, false) - } - private val threadNamesAndIds = new HashMap[String, AtomicInteger]() +trait Scheduler { + + /** + * Initialize this scheduler so it is ready to accept scheduling of tasks + */ + def startup() + + /** + * Shutdown this scheduler. When this method is complete no more executions of background tasks will occur. + * This includes tasks scheduled with a delayed execution. + */ + def shutdown() + + /** + * Schedule a task + * @param name The name of this task + * @param delay The amount of time to wait before the first execution + * @param period The period with which to execute the task. If < 0 the task will execute only once. + * @param unit The unit for the preceding times. + */ + def schedule(name: String, fun: ()=>Unit, delay: Long = 0, period: Long = -1, unit: TimeUnit = TimeUnit.MILLISECONDS) +} - def startup() = { - executor = new ScheduledThreadPoolExecutor(numThreads) - executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false) - executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false) +/** + * A scheduler based on java.util.concurrent.ScheduledThreadPoolExecutor + * + * It has a pool of kafka-scheduler- threads that do the actual work. + * + * @param threads The number of threads in the thread pool + * @param threadNamePrefix The name to use for scheduler threads. This prefix will have a number appended to it. + * @param daemon If true the scheduler threads will be "daemon" threads and will not block jvm shutdown. + */ +@threadsafe +class KafkaScheduler(val threads: Int, + val threadNamePrefix: String = "kafka-scheduler-", + daemon: Boolean = true) extends Scheduler with Logging { + @volatile private var executor: ScheduledThreadPoolExecutor = null + private val schedulerThreadId = new AtomicInteger(0) + + override def startup() { + debug("Initializing task scheduler.") + this synchronized { + if(executor != null) + throw new IllegalStateException("This scheduler has already been started!") + executor = new ScheduledThreadPoolExecutor(threads) + executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false) + executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false) + executor.setThreadFactory(new ThreadFactory() { + def newThread(runnable: Runnable): Thread = + Utils.newThread(threadNamePrefix + schedulerThreadId.getAndIncrement(), runnable, daemon) + }) + } } - - def hasShutdown: Boolean = executor.isShutdown - - private def ensureExecutorHasStarted = { - if(executor == null) - throw new IllegalStateException("Kafka scheduler has not been started") + + override def shutdown() { + debug("Shutting down task scheduler.") + ensureStarted + executor.shutdown() + executor.awaitTermination(1, TimeUnit.DAYS) + this.executor = null } - def scheduleWithRate(fun: () => Unit, name: String, delayMs: Long, periodMs: Long, isDaemon: Boolean = true) = { - ensureExecutorHasStarted - if(isDaemon) - executor.setThreadFactory(daemonThreadFactory) + def schedule(name: String, fun: ()=>Unit, delay: Long, period: Long, unit: TimeUnit) = { + debug("Scheduling task %s with initial delay %d ms and period %d ms." + .format(name, TimeUnit.MILLISECONDS.convert(delay, unit), TimeUnit.MILLISECONDS.convert(period, unit))) + ensureStarted + val runnable = new Runnable { + def run() = { + try { + trace("Begining execution of scheduled task '%s'.".format(name)) + fun() + } catch { + case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t) + } finally { + trace("Completed execution of scheduled task '%s'.".format(name)) + } + } + } + if(period >= 0) + executor.scheduleAtFixedRate(runnable, delay, period, unit) else - executor.setThreadFactory(nonDaemonThreadFactory) - val threadId = threadNamesAndIds.getOrElseUpdate(name, new AtomicInteger(0)) - executor.scheduleAtFixedRate(Utils.loggedRunnable(fun, name + threadId.incrementAndGet()), delayMs, periodMs, - TimeUnit.MILLISECONDS) - } - - def shutdownNow() { - ensureExecutorHasStarted - executor.shutdownNow() - info("Forcing shutdown of Kafka scheduler") + executor.schedule(runnable, delay, unit) } - - def shutdown() { - ensureExecutorHasStarted - executor.shutdown() - info("Shutdown Kafka scheduler") + + private def ensureStarted = { + if(executor == null) + throw new IllegalStateException("Kafka scheduler has not been started") } } diff --git a/core/src/main/scala/kafka/utils/Logging.scala b/core/src/main/scala/kafka/utils/Logging.scala index d9f010bfe7a9c..2890e7f51ddcb 100644 --- a/core/src/main/scala/kafka/utils/Logging.scala +++ b/core/src/main/scala/kafka/utils/Logging.scala @@ -23,12 +23,13 @@ trait Logging { val loggerName = this.getClass.getName lazy val logger = Logger.getLogger(loggerName) - protected var logIdent = "" + protected var logIdent: String = null // Force initialization to register Log4jControllerMBean private val log4jController = Log4jController - private def msgWithLogIdent(msg: String) = logIdent + msg + private def msgWithLogIdent(msg: String) = + if(logIdent == null) msg else logIdent + msg def trace(msg: => String): Unit = { if (logger.isTraceEnabled()) diff --git a/core/src/main/scala/kafka/producer/async/AsyncProducerStats.scala b/core/src/main/scala/kafka/utils/Os.scala similarity index 85% rename from core/src/main/scala/kafka/producer/async/AsyncProducerStats.scala rename to core/src/main/scala/kafka/utils/Os.scala index 487329ad228e9..6574f08d9182c 100644 --- a/core/src/main/scala/kafka/producer/async/AsyncProducerStats.scala +++ b/core/src/main/scala/kafka/utils/Os.scala @@ -13,4 +13,11 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - */ \ No newline at end of file + */ + +package kafka.utils + +object Os { + val name = System.getProperty("os.name").toLowerCase + val isWindows = name.startsWith("windows") +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/Throttler.scala b/core/src/main/scala/kafka/utils/Throttler.scala index 2bfa102a7d44e..c6c3c75ee8408 100644 --- a/core/src/main/scala/kafka/utils/Throttler.scala +++ b/core/src/main/scala/kafka/utils/Throttler.scala @@ -17,38 +17,29 @@ package kafka.utils; +import java.util.Random import scala.math._ -object Throttler extends Logging { - val DefaultCheckIntervalMs = 100L -} - /** * A class to measure and throttle the rate of some process. The throttler takes a desired rate-per-second * (the units of the process don't matter, it could be bytes or a count of some other thing), and will sleep for - * an appropraite amount of time when maybeThrottle() is called to attain the desired rate. + * an appropriate amount of time when maybeThrottle() is called to attain the desired rate. * * @param desiredRatePerSec: The rate we want to hit in units/sec * @param checkIntervalMs: The interval at which to check our rate * @param throttleDown: Does throttling increase or decrease our rate? * @param time: The time implementation to use */ -@nonthreadsafe +@threadsafe class Throttler(val desiredRatePerSec: Double, - val checkIntervalMs: Long, - val throttleDown: Boolean, - val time: Time) { + val checkIntervalMs: Long = 100L, + val throttleDown: Boolean = true, + val time: Time = SystemTime) extends Logging { private val lock = new Object private var periodStartNs: Long = time.nanoseconds private var observedSoFar: Double = 0.0 - def this(desiredRatePerSec: Double, throttleDown: Boolean) = - this(desiredRatePerSec, Throttler.DefaultCheckIntervalMs, throttleDown, SystemTime) - - def this(desiredRatePerSec: Double) = - this(desiredRatePerSec, Throttler.DefaultCheckIntervalMs, true, SystemTime) - def maybeThrottle(observed: Double) { lock synchronized { observedSoFar += observed @@ -61,12 +52,11 @@ class Throttler(val desiredRatePerSec: Double, val needAdjustment = !(throttleDown ^ (rateInSecs > desiredRatePerSec)) if(needAdjustment) { // solve for the amount of time to sleep to make us hit the desired rate - val desiredRateMs = desiredRatePerSec / Time.MsPerSec.asInstanceOf[Double] + val desiredRateMs = desiredRatePerSec / Time.MsPerSec.toDouble val elapsedMs = elapsedNs / Time.NsPerMs val sleepTime = round(observedSoFar / desiredRateMs - elapsedMs) if(sleepTime > 0) { - Throttler.debug("Natural rate is " + rateInSecs + " per second but desired rate is " + desiredRatePerSec + - ", sleeping for " + sleepTime + " ms to compensate.") + trace("Natural rate is %f per second but desired rate is %f, sleeping for %d ms to compensate.".format(rateInSecs, desiredRatePerSec, sleepTime)) time.sleep(sleepTime) } } @@ -77,3 +67,26 @@ class Throttler(val desiredRatePerSec: Double, } } + +object Throttler { + + def main(args: Array[String]) { + val rand = new Random() + val throttler = new Throttler(100000, 100, true, SystemTime) + val interval = 30000 + var start = System.currentTimeMillis + var total = 0 + while(true) { + val value = rand.nextInt(1000) + Thread.sleep(1) + throttler.maybeThrottle(value) + total += value + val now = System.currentTimeMillis + if(now - start >= interval) { + println(total / (interval/1000.0)) + start = now + total = 0 + } + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index e0a5a27c72abf..a89b0463685e6 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -21,14 +21,15 @@ import java.io._ import java.nio._ import charset.Charset import java.nio.channels._ +import java.util.concurrent.locks.Lock import java.lang.management._ -import java.util.zip.CRC32 import javax.management._ import scala.collection._ import mutable.ListBuffer import scala.collection.mutable import java.util.Properties import kafka.common.KafkaException +import kafka.common.KafkaStorageException /** @@ -53,26 +54,6 @@ object Utils extends Logging { new Runnable() { def run() = fun() } - - /** - * Wrap the given function in a java.lang.Runnable that logs any errors encountered - * @param fun A function - * @return A Runnable that just executes the function - */ - def loggedRunnable(fun: () => Unit, name: String): Runnable = - new Runnable() { - def run() = { - Thread.currentThread().setName(name) - try { - fun() - } - catch { - case t: Throwable => - // log any error and the stack trace - error("error in loggedRunnable", t) - } - } - } /** * Create a daemon thread @@ -181,7 +162,7 @@ object Utils extends Logging { * @param log The log method to use for logging. E.g. logger.warn * @param action The action to execute */ - def swallow(log: (Object, Throwable) => Unit, action: => Unit) = { + def swallow(log: (Object, Throwable) => Unit, action: => Unit) { try { action } catch { @@ -243,18 +224,18 @@ object Utils extends Logging { * @param file The root file at which to begin deleting */ def rm(file: File) { - if(file == null) { - return - } else if(file.isDirectory) { - val files = file.listFiles() - if(files != null) { - for(f <- files) - rm(f) + if(file == null) { + return + } else if(file.isDirectory) { + val files = file.listFiles() + if(files != null) { + for(f <- files) + rm(f) + } + file.delete() + } else { + file.delete() } - file.delete() - } else { - file.delete() - } } /** @@ -349,7 +330,7 @@ object Utils extends Logging { * @return The CRC32 */ def crc32(bytes: Array[Byte], offset: Int, size: Int): Long = { - val crc = new CRC32() + val crc = new Crc32() crc.update(bytes, offset, size) crc.getValue() } @@ -464,65 +445,6 @@ object Utils extends Logging { */ def nullOrEmpty(s: String): Boolean = s == null || s.equals("") - /** - * Merge JSON fields of the format "key" : value/object/array. - */ - def mergeJsonFields(objects: Seq[String]): String = { - val builder = new StringBuilder - builder.append("{ ") - builder.append(objects.sorted.map(_.trim).mkString(", ")) - builder.append(" }") - builder.toString - } - - /** - * Format a Map[String, String] as JSON object. - */ - def mapToJsonFields(jsonDataMap: Map[String, String], valueInQuotes: Boolean): Seq[String] = { - val jsonFields: mutable.ListBuffer[String] = ListBuffer() - val builder = new StringBuilder - for ((key, value) <- jsonDataMap.toList.sorted) { - builder.append("\"" + key + "\":") - if (valueInQuotes) - builder.append("\"" + value + "\"") - else - builder.append(value) - jsonFields += builder.toString - builder.clear() - } - jsonFields - } - - /** - * Format a Map[String, String] as JSON object. - */ - def mapToJson(jsonDataMap: Map[String, String], valueInQuotes: Boolean): String = { - mergeJsonFields(mapToJsonFields(jsonDataMap, valueInQuotes)) - } - - /** - * Format a Seq[String] as JSON array. - */ - def seqToJson(jsonData: Seq[String], valueInQuotes: Boolean): String = { - val builder = new StringBuilder - builder.append("[ ") - if (valueInQuotes) - builder.append(jsonData.map("\"" + _ + "\"").mkString(", ")) - else - builder.append(jsonData.mkString(", ")) - builder.append(" ]") - builder.toString - } - - /** - * Format a Map[String, Seq[Int]] as JSON - */ - - def mapWithSeqValuesToJson(jsonDataMap: Map[String, Seq[Int]]): String = { - mergeJsonFields(mapToJsonFields(jsonDataMap.map(e => (e._1 -> seqToJson(e._2.map(_.toString), valueInQuotes = false))), - valueInQuotes = false)) - } - /** * Create a circular (looping) iterator over a collection. * @param coll An iterable over the underlying collection. @@ -554,4 +476,69 @@ object Utils extends Logging { * This is different from java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!). */ def abs(n: Int) = n & 0x7fffffff -} \ No newline at end of file + + /** + * Replace the given string suffix with the new suffix. If the string doesn't end with the given suffix throw an exception. + */ + def replaceSuffix(s: String, oldSuffix: String, newSuffix: String): String = { + if(!s.endsWith(oldSuffix)) + throw new IllegalArgumentException("Expected string to end with '%s' but string is '%s'".format(oldSuffix, s)) + s.substring(0, s.length - oldSuffix.length) + newSuffix + } + + /** + * Create a file with the given path + * @param path The path to create + * @throw KafkaStorageException If the file create fails + * @return The created file + */ + def createFile(path: String): File = { + val f = new File(path) + val created = f.createNewFile() + if(!created) + throw new KafkaStorageException("Failed to create file %s.".format(path)) + f + } + + /** + * Turn a properties map into a string + */ + def asString(props: Properties): String = { + val writer = new StringWriter() + props.store(writer, "") + writer.toString + } + + /** + * Read some properties with the given default values + */ + def readProps(s: String, defaults: Properties): Properties = { + val reader = new StringReader(s) + val props = new Properties(defaults) + props.load(reader) + props + } + + /** + * Read a big-endian integer from a byte array + */ + def readInt(bytes: Array[Byte], offset: Int): Int = { + ((bytes(offset) & 0xFF) << 24) | + ((bytes(offset + 1) & 0xFF) << 16) | + ((bytes(offset + 2) & 0xFF) << 8) | + (bytes(offset + 3) & 0xFF) + } + + /** + * Execute the given function inside the lock + */ + def inLock[T](lock: Lock)(fun: => T): T = { + lock.lock() + try { + return fun + } finally { + lock.unlock() + } + } + +} diff --git a/core/src/main/scala/kafka/utils/VerifiableProperties.scala b/core/src/main/scala/kafka/utils/VerifiableProperties.scala index d694ba98522a0..b070bb4df117d 100644 --- a/core/src/main/scala/kafka/utils/VerifiableProperties.scala +++ b/core/src/main/scala/kafka/utils/VerifiableProperties.scala @@ -18,6 +18,7 @@ package kafka.utils import java.util.Properties +import java.util.Collections import scala.collection._ class VerifiableProperties(val props: Properties) extends Logging { @@ -38,10 +39,7 @@ class VerifiableProperties(val props: Properties) extends Logging { /** * Read a required integer property value or throw an exception if no such property is found */ - def getInt(name: String): Int = { - require(containsKey(name), "Missing required property '" + name + "'") - return getInt(name, -1) - } + def getInt(name: String): Int = getString(name).toInt def getIntInRange(name: String, range: (Int, Int)): Int = { require(containsKey(name), "Missing required property '" + name + "'") @@ -92,10 +90,7 @@ class VerifiableProperties(val props: Properties) extends Logging { /** * Read a required long property value or throw an exception if no such property is found */ - def getLong(name: String): Long = { - require(containsKey(name), "Missing required property '" + name + "'") - return getLong(name, -1) - } + def getLong(name: String): Long = getString(name).toLong /** * Read an long from the properties instance @@ -124,6 +119,26 @@ class VerifiableProperties(val props: Properties) extends Logging { require(v >= range._1 && v <= range._2, name + " has value " + v + " which is not in the range " + range + ".") v } + + /** + * Get a required argument as a double + * @param name The property name + * @return the value + * @throw IllegalArgumentException If the given property is not present + */ + def getDouble(name: String): Double = getString(name).toDouble + + /** + * Get an optional argument as a double + * @param name The property name + * @default The default value for the property if not present + */ + def getDouble(name: String, default: Double): Double = { + if(containsKey(name)) + getDouble(name) + else + default + } /** * Read a boolean value from the properties instance @@ -140,6 +155,8 @@ class VerifiableProperties(val props: Properties) extends Logging { v.toBoolean } } + + def getBoolean(name: String) = getString(name).toBoolean /** * Get a string property, or, if no such property is defined, return the given default value @@ -162,7 +179,7 @@ class VerifiableProperties(val props: Properties) extends Logging { /** * Get a Map[String, String] from a property list in the form k1:v2, k2:v2, ... */ - def getMap(name: String, valid: String => Boolean): Map[String, String] = { + def getMap(name: String, valid: String => Boolean = s => true): Map[String, String] = { try { val m = Utils.parseCsvMap(getString(name, "")) m.foreach { @@ -178,10 +195,12 @@ class VerifiableProperties(val props: Properties) extends Logging { def verify() { info("Verifying properties") - val specifiedProperties = props.propertyNames() - while (specifiedProperties.hasMoreElements) { - val key = specifiedProperties.nextElement().asInstanceOf[String] - if (!referenceSet.contains(key)) + val propNames = { + import JavaConversions._ + Collections.list(props.propertyNames).map(_.toString).sorted + } + for(key <- propNames) { + if (!referenceSet.contains(key) && !key.startsWith("external")) warn("Property %s is not valid".format(key)) else info("Property %s is overridden to %s".format(key, props.getProperty(key))) @@ -189,4 +208,5 @@ class VerifiableProperties(val props: Properties) extends Logging { } override def toString(): String = props.toString + } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 6eede1bde352a..b71b96deb6162 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -32,14 +32,18 @@ import kafka.common.{KafkaException, NoEpochForPartitionException} import kafka.controller.ReassignedPartitionsContext import kafka.controller.PartitionAndReplica import kafka.controller.KafkaController -import scala.Some +import scala.{collection, Some} import kafka.controller.LeaderIsrAndControllerEpoch import kafka.common.TopicAndPartition +import kafka.utils.Utils.inLock +import scala.collection object ZkUtils extends Logging { val ConsumersPath = "/consumers" val BrokerIdsPath = "/brokers/ids" val BrokerTopicsPath = "/brokers/topics" + val TopicConfigPath = "/config/topics" + val TopicConfigChangesPath = "/config/changes" val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" val ReassignPartitionsPath = "/admin/reassign_partitions" @@ -53,6 +57,9 @@ object ZkUtils extends Logging { getTopicPath(topic) + "/partitions" } + def getTopicConfigPath(topic: String): String = + TopicConfigPath + "/" + topic + def getController(zkClient: ZkClient): Int= { readDataMaybeNull(zkClient, ControllerPath)._1 match { case Some(controller) => KafkaController.parseControllerId(controller) @@ -60,17 +67,14 @@ object ZkUtils extends Logging { } } - def getTopicPartitionPath(topic: String, partitionId: Int): String ={ + def getTopicPartitionPath(topic: String, partitionId: Int): String = getTopicPartitionsPath(topic) + "/" + partitionId - } - def getTopicPartitionLeaderAndIsrPath(topic: String, partitionId: Int): String ={ + def getTopicPartitionLeaderAndIsrPath(topic: String, partitionId: Int): String = getTopicPartitionPath(topic, partitionId) + "/" + "state" - } - def getSortedBrokerList(zkClient: ZkClient): Seq[Int] ={ + def getSortedBrokerList(zkClient: ZkClient): Seq[Int] = ZkUtils.getChildren(zkClient, BrokerIdsPath).map(_.toInt).sorted - } def getAllBrokersInCluster(zkClient: ZkClient): Seq[Broker] = { val brokerIds = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerIdsPath).sorted @@ -91,6 +95,11 @@ object ZkUtils extends Logging { def getLeaderAndIsrForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderAndIsr] = { getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr) } + + def setupCommonPaths(zkClient: ZkClient) { + for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath)) + makeSurePersistentPathExists(zkClient, path) + } def parseLeaderAndIsr(leaderAndIsrStr: String, topic: String, partition: Int, stat: Stat) : Option[LeaderIsrAndControllerEpoch] = { @@ -176,20 +185,11 @@ object ZkUtils extends Logging { } } - def isPartitionOnBroker(zkClient: ZkClient, topic: String, partition: Int, brokerId: Int): Boolean = { - val replicas = getReplicasForPartition(zkClient, topic, partition) - debug("The list of replicas for partition [%s,%d] is %s".format(topic, partition, replicas)) - replicas.contains(brokerId.toString) - } - - def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, timeout: Int, jmxPort: Int) { + def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, rack: 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, "timestamp" -> timestamp), - valueInQuotes = false)) - val expectedBroker = new Broker(id, host, port) + val timestamp = SystemTime.milliseconds.toString + val brokerInfo = Json.encode(Map("version" -> 2, "host" -> host, "port" -> port, "rack" -> rack, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val expectedBroker = new Broker(id, host, port, rack) try { createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, @@ -211,18 +211,17 @@ object ZkUtils extends Logging { topicDirs.consumerOwnerDir + "/" + partition } + def leaderAndIsrZkData(leaderAndIsr: LeaderAndIsr, controllerEpoch: Int): String = { - val isrInfo = Utils.seqToJson(leaderAndIsr.isr.map(_.toString), valueInQuotes = false) - Utils.mapToJson(Map("version" -> 1.toString, "leader" -> leaderAndIsr.leader.toString, "leader_epoch" -> leaderAndIsr.leaderEpoch.toString, - "controller_epoch" -> controllerEpoch.toString, "isr" -> isrInfo), valueInQuotes = false) + Json.encode(Map("version" -> 1, "leader" -> leaderAndIsr.leader, "leader_epoch" -> leaderAndIsr.leaderEpoch, + "controller_epoch" -> controllerEpoch, "isr" -> leaderAndIsr.isr)) } /** * Get JSON partition to replica map from zookeeper. */ - def replicaAssignmentZkdata(map: Map[String, Seq[Int]]): String = { - val jsonReplicaAssignmentMap = Utils.mapWithSeqValuesToJson(map) - Utils.mapToJson(Map("version" -> 1.toString, "partitions" -> jsonReplicaAssignmentMap), valueInQuotes = false) + def replicaAssignmentZkData(map: Map[String, Seq[Int]], maxRackReplication: Int): String = { + Json.encode(Map("version" -> 2, "partitions" -> map, "max-rack-replication" -> maxRackReplication)) } /** @@ -495,8 +494,6 @@ object ZkUtils extends Logging { client.exists(path) } - def getLastPart(path : String) : String = path.substring(path.lastIndexOf('/') + 1) - def getCluster(zkClient: ZkClient) : Cluster = { val cluster = new Cluster val nodes = getChildrenParentMayNotExist(zkClient, BrokerIdsPath) @@ -507,6 +504,18 @@ object ZkUtils extends Logging { cluster } + /* Provide a Cluster object filtered with the given broker list. Usefull when dealing with a sub-cluster */ + def getFilteredCluster(zkClient: ZkClient, brokerList: Seq[Int]) : Cluster = { + val cluster = new Cluster + for (brokerId <- brokerList) { + ZkUtils.readDataMaybeNull(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 match { + case Some(brokerInfo) => cluster.add(Broker.createBroker(brokerId, brokerInfo)) + case None => /* do nothing */ + } + } + cluster + } + def getPartitionLeaderAndIsrForTopics(zkClient: ZkClient, topicAndPartitions: Set[TopicAndPartition]) : mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = { val ret = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch] @@ -519,6 +528,28 @@ object ZkUtils extends Logging { ret } + def getMaxRackReplicationForTopics(zkClient: ZkClient, topics: Seq[String]): mutable.Map[String, Int] = { + val ret = new mutable.HashMap[String, Int]() + topics.foreach { topic => + val jsonPartitionMapOpt = readDataMaybeNull(zkClient, getTopicPath(topic))._1 + jsonPartitionMapOpt match { + case Some(jsonPartitionMap) => + Json.parseFull(jsonPartitionMap) match { + case Some(m) => m.asInstanceOf[Map[String, Any]].get("max-rack-replication") match { + case Some(repl) => + val maxRackReplication = repl.asInstanceOf[Int] + ret.put(topic, maxRackReplication) + case None => + ret.put(topic, -1) + } + case None => + } + case None => + } + } + ret + } + def getReplicaAssignmentForTopics(zkClient: ZkClient, topics: Seq[String]): mutable.Map[TopicAndPartition, Seq[Int]] = { val ret = new mutable.HashMap[TopicAndPartition, Seq[Int]] topics.foreach { topic => @@ -566,17 +597,6 @@ object ZkUtils extends Logging { ret } - def getReplicaAssignmentFromPartitionAssignment(topicPartitionAssignment: mutable.Map[String, collection.Map[Int, Seq[Int]]]): - mutable.Map[(String, Int), Seq[Int]] = { - val ret = new mutable.HashMap[(String, Int), Seq[Int]] - for((topic, partitionAssignment) <- topicPartitionAssignment){ - for((partition, replicaAssignment) <- partitionAssignment){ - ret.put((topic, partition), replicaAssignment) - } - } - ret - } - def getPartitionsForTopics(zkClient: ZkClient, topics: Seq[String]): mutable.Map[String, Seq[Int]] = { getPartitionAssignmentForTopics(zkClient, topics).map { topicAndPartitionMap => val topic = topicAndPartitionMap._1 @@ -586,19 +606,6 @@ object ZkUtils extends Logging { } } - def getPartitionsAssignedToBroker(zkClient: ZkClient, topics: Seq[String], brokerId: Int): Seq[(String, Int)] = { - val topicsAndPartitions = getPartitionAssignmentForTopics(zkClient, topics) - topicsAndPartitions.map { topicAndPartitionMap => - val topic = topicAndPartitionMap._1 - val partitionMap = topicAndPartitionMap._2 - val relevantPartitionsMap = partitionMap.filter( m => m._2.contains(brokerId) ) - val relevantPartitions = relevantPartitionsMap.map(_._1) - for(relevantPartition <- relevantPartitions) yield { - (topic, relevantPartition) - } - }.flatten[(String, Int)].toSeq - } - def getPartitionsBeingReassigned(zkClient: ZkClient): Map[TopicAndPartition, ReassignedPartitionsContext] = { // read the partitions and their new replica list val jsonPartitionMapOpt = readDataMaybeNull(zkClient, ReassignPartitionsPath)._1 @@ -648,16 +655,8 @@ object ZkUtils extends Logging { } def getPartitionReassignmentZkData(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]): String = { - var jsonPartitionsData: mutable.ListBuffer[String] = ListBuffer[String]() - for (p <- partitionsToBeReassigned) { - val jsonReplicasData = Utils.seqToJson(p._2.map(_.toString), valueInQuotes = false) - val jsonTopicData = Utils.mapToJsonFields(Map("topic" -> p._1.topic), valueInQuotes = true) - val jsonPartitionData = Utils.mapToJsonFields(Map("partition" -> p._1.partition.toString, "replicas" -> jsonReplicasData), - valueInQuotes = false) - jsonPartitionsData += Utils.mergeJsonFields(jsonTopicData ++ jsonPartitionData) - } - Utils.mapToJson(Map("version" -> 1.toString, "partitions" -> Utils.seqToJson(jsonPartitionsData.toSeq, valueInQuotes = false)), - valueInQuotes = false) + Json.encode(Map("version" -> 1, "partitions" -> partitionsToBeReassigned.map(e => Map("topic" -> e._1.topic, "partition" -> e._1.partition, + "replicas" -> e._2)))) } def updatePartitionReassignmentData(zkClient: ZkClient, partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]) { @@ -675,22 +674,11 @@ object ZkUtils extends Logging { case nne: ZkNoNodeException => ZkUtils.createPersistentPath(zkClient, zkPath, jsonData) debug("Created path %s with %s for partition reassignment".format(zkPath, jsonData)) - case e2: Throwable => throw new AdministrationException(e2.toString) + case e2: Throwable => throw new AdminOperationException(e2.toString) } } } - def getAllReplicasOnBroker(zkClient: ZkClient, topics: Seq[String], brokerIds: Seq[Int]): Set[PartitionAndReplica] = { - Set.empty[PartitionAndReplica] ++ brokerIds.map { brokerId => - // read all the partitions and their assigned replicas into a map organized by - // { replica id -> partition 1, partition 2... - val partitionsAssignedToThisBroker = getPartitionsAssignedToBroker(zkClient, topics, brokerId) - if(partitionsAssignedToThisBroker.size == 0) - info("No state transitions triggered since no partitions are assigned to brokers %s".format(brokerIds.mkString(","))) - partitionsAssignedToThisBroker.map(p => new PartitionAndReplica(p._1, p._2, brokerId)) - }.flatten - } - def getPartitionsUndergoingPreferredReplicaElection(zkClient: ZkClient): Set[TopicAndPartition] = { // read the partitions and their new replica list val jsonPartitionListOpt = readDataMaybeNull(zkClient, PreferredReplicaLeaderElectionPath)._1 @@ -774,8 +762,7 @@ class LeaderExistsOrChangedListener(topic: String, def handleDataChange(dataPath: String, data: Object) { val t = dataPath.split("/").takeRight(3).head val p = dataPath.split("/").takeRight(2).head.toInt - leaderLock.lock() - try { + inLock(leaderLock) { if(t == topic && p == partition){ if(oldLeaderOpt == None){ trace("In leader existence listener on partition [%s, %d], leader has been created".format(topic, partition)) @@ -790,18 +777,12 @@ class LeaderExistsOrChangedListener(topic: String, } } } - finally { - leaderLock.unlock() - } } @throws(classOf[Exception]) def handleDataDeleted(dataPath: String) { - leaderLock.lock() - try { + inLock(leaderLock) { leaderExistsOrChanged.signal() - }finally { - leaderLock.unlock() } } } diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala index 78e054877bd2b..8fcd068b24868 100644 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -32,15 +32,14 @@ object StressTestLog { def main(args: Array[String]) { val dir = TestUtils.tempDir() - val log = new Log(dir, - maxLogFileSize = 64*1024*1024, - maxMessageSize = Int.MaxValue, - flushInterval = Int.MaxValue, - rollIntervalMs = Long.MaxValue, - needsRecovery = false, - maxIndexSize = 1024*1024, - time = SystemTime, - brokerId = 0) + val time = new MockTime + val log = new Log(dir = dir, + config = LogConfig(segmentSize = 64*1024*1024, + maxMessageSize = Int.MaxValue, + maxIndexSize = 1024*1024), + recoveryPoint = 0L, + scheduler = time.scheduler, + time = time) val writer = new WriterThread(log) writer.start() val reader = new ReaderThread(log) @@ -80,8 +79,8 @@ object StressTestLog { class WriterThread(val log: Log) extends WorkerThread { @volatile var offset = 0 override def work() { - val offsets = log.append(TestUtils.singleMessageSet(offset.toString.getBytes)) - require(offsets._1 == offset && offsets._2 == offset) + val logAppendInfo = log.append(TestUtils.singleMessageSet(offset.toString.getBytes)) + require(logAppendInfo.firstOffset == offset && logAppendInfo.lastOffset == offset) offset += 1 if(offset % 1000 == 0) Thread.sleep(500) diff --git a/core/src/test/scala/other/kafka/TestCrcPerformance.scala b/core/src/test/scala/other/kafka/TestCrcPerformance.scala new file mode 100644 index 0000000000000..e5d3df18db307 --- /dev/null +++ b/core/src/test/scala/other/kafka/TestCrcPerformance.scala @@ -0,0 +1,31 @@ +package kafka.log + +import java.util.Random +import kafka.message._ +import kafka.utils.{TestUtils, Utils} + +object TestCrcPerformance { + + def main(args: Array[String]): Unit = { + if(args.length < 2) + Utils.croak("USAGE: java " + getClass().getName() + " num_messages message_size") + val numMessages = args(0).toInt + val messageSize = args(1).toInt + //val numMessages = 100000000 + //val messageSize = 32 + + val dir = TestUtils.tempDir() + val content = new Array[Byte](messageSize) + new Random(1).nextBytes(content) + + // create message test + val start = System.nanoTime + for(i <- 0 until numMessages) { + new Message(content) + } + val ellapsed = System.nanoTime - start + println("%d messages created in %.2f seconds + (%.2f ns per message).".format(numMessages, ellapsed/(1000.0*1000.0*1000.0), + ellapsed / numMessages.toDouble)) + + } +} diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index 36d52e75f05c7..eeb8c8856200c 100644 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -20,9 +20,16 @@ package kafka import java.io._ import java.nio._ import java.nio.channels._ +import java.util.Random +import kafka.log._ +import kafka.utils._ +import kafka.message._ import scala.math._ import joptsimple._ +/** + * This test does linear writes using either a kafka log or a file and measures throughput and latency. + */ object TestLinearWriteSpeed { def main(args: Array[String]): Unit = { @@ -32,7 +39,7 @@ object TestLinearWriteSpeed { .describedAs("path") .ofType(classOf[java.lang.String]) .defaultsTo(System.getProperty("java.io.tmpdir")) - val bytesOpt = parser.accepts("bytes", "REQUIRED: The number of bytes to write.") + val bytesOpt = parser.accepts("bytes", "REQUIRED: The total number of bytes to write.") .withRequiredArg .describedAs("num_bytes") .ofType(classOf[java.lang.Long]) @@ -40,7 +47,12 @@ object TestLinearWriteSpeed { .withRequiredArg .describedAs("num_bytes") .ofType(classOf[java.lang.Integer]) - val filesOpt = parser.accepts("files", "REQUIRED: The number of files.") + val messageSizeOpt = parser.accepts("message-size", "REQUIRED: The size of each message in the message set.") + .withRequiredArg + .describedAs("num_bytes") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1024) + val filesOpt = parser.accepts("files", "REQUIRED: The number of logs or files.") .withRequiredArg .describedAs("num_files") .ofType(classOf[java.lang.Integer]) @@ -55,7 +67,19 @@ object TestLinearWriteSpeed { .describedAs("mb") .ofType(classOf[java.lang.Integer]) .defaultsTo(Integer.MAX_VALUE) - val mmapOpt = parser.accepts("mmap", "Mmap file.") + val flushIntervalOpt = parser.accepts("flush-interval", "The number of messages between flushes") + .withRequiredArg() + .describedAs("message_count") + .ofType(classOf[java.lang.Long]) + .defaultsTo(Long.MaxValue) + val compressionCodecOpt = parser.accepts("compression", "The compression codec to use") + .withRequiredArg + .describedAs("codec") + .ofType(classOf[java.lang.String]) + .defaultsTo(NoCompressionCodec.name) + val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.") + val channelOpt = parser.accepts("channel", "Do writes to file channesl.") + val logOpt = parser.accepts("log", "Do writes to kafka logs.") val options = parser.parse(args : _*) @@ -68,26 +92,35 @@ object TestLinearWriteSpeed { } var bytesToWrite = options.valueOf(bytesOpt).longValue - val mmap = options.has(mmapOpt) val bufferSize = options.valueOf(sizeOpt).intValue val numFiles = options.valueOf(filesOpt).intValue val reportingInterval = options.valueOf(reportingIntervalOpt).longValue val dir = options.valueOf(dirOpt) val maxThroughputBytes = options.valueOf(maxThroughputOpt).intValue * 1024L * 1024L val buffer = ByteBuffer.allocate(bufferSize) - while(buffer.hasRemaining) - buffer.put(123.asInstanceOf[Byte]) + val messageSize = options.valueOf(messageSizeOpt).intValue + val flushInterval = options.valueOf(flushIntervalOpt).longValue + val compressionCodec = CompressionCodec.getCompressionCodec(options.valueOf(compressionCodecOpt)) + val rand = new Random + rand.nextBytes(buffer.array) + val numMessages = bufferSize / (messageSize + MessageSet.LogOverhead) + val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec, messages = (0 until numMessages).map(x => new Message(new Array[Byte](messageSize))): _*) val writables = new Array[Writable](numFiles) + val scheduler = new KafkaScheduler(1) + scheduler.startup() for(i <- 0 until numFiles) { - val file = new File(dir, "kafka-test-" + i + ".dat") - file.deleteOnExit() - val raf = new RandomAccessFile(file, "rw") - raf.setLength(bytesToWrite / numFiles) - if(mmap) - writables(i) = new MmapWritable(raf.getChannel().map(FileChannel.MapMode.READ_WRITE, 0, raf.length())) - else - writables(i) = new ChannelWritable(raf.getChannel()) + if(options.has(mmapOpt)) { + writables(i) = new MmapWritable(new File(dir, "kafka-test-" + i + ".dat"), bytesToWrite / numFiles, buffer) + } else if(options.has(channelOpt)) { + writables(i) = new ChannelWritable(new File(dir, "kafka-test-" + i + ".dat"), buffer) + } else if(options.has(logOpt)) { + val segmentSize = rand.nextInt(512)*1024*1024 + 64*1024*1024 // vary size to avoid herd effect + writables(i) = new LogWritable(new File(dir, "kafka-test-" + i), new LogConfig(segmentSize=segmentSize, flushInterval = flushInterval), scheduler, messageSet) + } else { + System.err.println("Must specify what to write to with one of --log, --channel, or --mmap") + System.exit(1) + } } bytesToWrite = (bytesToWrite / numFiles) * numFiles @@ -101,15 +134,14 @@ object TestLinearWriteSpeed { var totalWritten = 0L var lastReport = beginTest while(totalWritten + bufferSize < bytesToWrite) { - buffer.rewind() val start = System.nanoTime - writables((count % numFiles).toInt.abs).write(buffer) + val writeSize = writables((count % numFiles).toInt.abs).write() val ellapsed = System.nanoTime - start maxLatency = max(ellapsed, maxLatency) totalLatency += ellapsed - written += bufferSize + written += writeSize count += 1 - totalWritten += bufferSize + totalWritten += writeSize if((start - lastReport)/(1000.0*1000.0) > reportingInterval.doubleValue) { val ellapsedSecs = (start - lastReport) / (1000.0*1000.0*1000.0) val mb = written / (1024.0*1024.0) @@ -118,7 +150,7 @@ object TestLinearWriteSpeed { written = 0 maxLatency = 0L totalLatency = 0L - } else if(written > maxThroughputBytes) { + } else if(written > maxThroughputBytes * (reportingInterval / 1000.0)) { // if we have written enough, just sit out this reporting interval val lastReportMs = lastReport / (1000*1000) val now = System.nanoTime / (1000*1000) @@ -129,21 +161,53 @@ object TestLinearWriteSpeed { } val elapsedSecs = (System.nanoTime - beginTest) / (1000.0*1000.0*1000.0) println(bytesToWrite / (1024.0 * 1024.0 * elapsedSecs) + " MB per sec") + scheduler.shutdown() } trait Writable { - def write(buffer: ByteBuffer) + def write(): Int + def close() } - class MmapWritable(val buffer: ByteBuffer) extends Writable { - def write(b: ByteBuffer) { - buffer.put(b) + class MmapWritable(val file: File, size: Long, val content: ByteBuffer) extends Writable { + file.deleteOnExit() + val raf = new RandomAccessFile(file, "rw") + raf.setLength(size) + val buffer = raf.getChannel().map(FileChannel.MapMode.READ_WRITE, 0, raf.length()) + def write(): Int = { + buffer.put(content) + content.rewind() + content.limit + } + def close() { + raf.close() } } - class ChannelWritable(val channel: FileChannel) extends Writable { - def write(b: ByteBuffer) { - channel.write(b) + class ChannelWritable(val file: File, val content: ByteBuffer) extends Writable { + file.deleteOnExit() + val raf = new RandomAccessFile(file, "rw") + val channel = raf.getChannel + def write(): Int = { + channel.write(content) + content.rewind() + content.limit + } + def close() { + raf.close() + } + } + + class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: ByteBufferMessageSet) extends Writable { + Utils.rm(dir) + val log = new Log(dir, config, 0L, scheduler, SystemTime) + def write(): Int = { + log.append(messages, true) + messages.sizeInBytes + } + def close() { + log.close() + Utils.rm(log.dir) } } diff --git a/core/src/test/scala/other/kafka/TestLogCleaning.scala b/core/src/test/scala/other/kafka/TestLogCleaning.scala new file mode 100644 index 0000000000000..22b16e54980e3 --- /dev/null +++ b/core/src/test/scala/other/kafka/TestLogCleaning.scala @@ -0,0 +1,314 @@ +/** + * 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 + +import joptsimple.OptionParser +import java.util.Properties +import java.util.Random +import java.io._ +import scala.io.Source +import scala.io.BufferedSource +import kafka.producer._ +import kafka.consumer._ +import kafka.serializer._ +import kafka.utils._ +import kafka.log.FileMessageSet +import kafka.log.Log + +/** + * This is a torture test that runs against an existing broker. Here is how it works: + * + * It produces a series of specially formatted messages to one or more partitions. Each message it produces + * it logs out to a text file. The messages have a limited set of keys, so there is duplication in the key space. + * + * The broker will clean its log as the test runs. + * + * When the specified number of messages have been produced we create a consumer and consume all the messages in the topic + * and write that out to another text file. + * + * Using a stable unix sort we sort both the producer log of what was sent and the consumer log of what was retrieved by the message key. + * Then we compare the final message in both logs for each key. If this final message is not the same for all keys we + * print an error and exit with exit code 1, otherwise we print the size reduction and exit with exit code 0. + */ +object TestLogCleaning { + + def main(args: Array[String]) { + val parser = new OptionParser + val numMessagesOpt = parser.accepts("messages", "The number of messages to send or consume.") + .withRequiredArg + .describedAs("count") + .ofType(classOf[java.lang.Long]) + .defaultsTo(Long.MaxValue) + val numDupsOpt = parser.accepts("duplicates", "The number of duplicates for each key.") + .withRequiredArg + .describedAs("count") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(5) + val brokerOpt = parser.accepts("broker", "Url to connect to.") + .withRequiredArg + .describedAs("url") + .ofType(classOf[String]) + val topicsOpt = parser.accepts("topics", "The number of topics to test.") + .withRequiredArg + .describedAs("count") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) + val percentDeletesOpt = parser.accepts("percent-deletes", "The percentage of updates that are deletes.") + .withRequiredArg + .describedAs("percent") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(0) + val zkConnectOpt = parser.accepts("zk", "Zk url.") + .withRequiredArg + .describedAs("url") + .ofType(classOf[String]) + val sleepSecsOpt = parser.accepts("sleep", "Time to sleep between production and consumption.") + .withRequiredArg + .describedAs("ms") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(0) + val dumpOpt = parser.accepts("dump", "Dump the message contents of a topic partition that contains test data from this test to standard out.") + .withRequiredArg + .describedAs("directory") + .ofType(classOf[String]) + + val options = parser.parse(args:_*) + + if(options.has(dumpOpt)) { + dumpLog(new File(options.valueOf(dumpOpt))) + System.exit(0) + } + + if(!options.has(brokerOpt) || !options.has(zkConnectOpt) || !options.has(numMessagesOpt)) { + parser.printHelpOn(System.err) + System.exit(1) + } + + // parse options + val messages = options.valueOf(numMessagesOpt).longValue + val percentDeletes = options.valueOf(percentDeletesOpt).intValue + val dups = options.valueOf(numDupsOpt).intValue + val brokerUrl = options.valueOf(brokerOpt) + val topicCount = options.valueOf(topicsOpt).intValue + val zkUrl = options.valueOf(zkConnectOpt) + val sleepSecs = options.valueOf(sleepSecsOpt).intValue + + val testId = new Random().nextInt(Int.MaxValue) + val topics = (0 until topicCount).map("log-cleaner-test-" + testId + "-" + _).toArray + + println("Producing %d messages...".format(messages)) + val producedDataFile = produceMessages(brokerUrl, topics, messages, dups, percentDeletes) + println("Sleeping for %d seconds...".format(sleepSecs)) + Thread.sleep(sleepSecs * 1000) + println("Consuming messages...") + val consumedDataFile = consumeMessages(zkUrl, topics) + + val producedLines = lineCount(producedDataFile) + val consumedLines = lineCount(consumedDataFile) + val reduction = 1.0 - consumedLines.toDouble/producedLines.toDouble + println("%d rows of data produced, %d rows of data consumed (%.1f%% reduction).".format(producedLines, consumedLines, 100 * reduction)) + + println("Deduplicating and validating output files...") + validateOutput(producedDataFile, consumedDataFile) + producedDataFile.delete() + consumedDataFile.delete() + } + + def dumpLog(dir: File) { + require(dir.exists, "Non-existant directory: " + dir.getAbsolutePath) + for(file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) { + val ms = new FileMessageSet(new File(dir, file)) + for(entry <- ms) { + val key = Utils.readString(entry.message.key) + val content = + if(entry.message.isNull) + null + else + Utils.readString(entry.message.payload) + println("offset = %s, key = %s, content = %s".format(entry.offset, key, content)) + } + } + } + + def lineCount(file: File): Int = io.Source.fromFile(file).getLines.size + + def validateOutput(producedDataFile: File, consumedDataFile: File) { + val producedReader = externalSort(producedDataFile) + val consumedReader = externalSort(consumedDataFile) + val produced = valuesIterator(producedReader) + val consumed = valuesIterator(consumedReader) + val producedDedupedFile = new File(producedDataFile.getAbsolutePath + ".deduped") + val producedDeduped = new BufferedWriter(new FileWriter(producedDedupedFile), 1024*1024) + val consumedDedupedFile = new File(consumedDataFile.getAbsolutePath + ".deduped") + val consumedDeduped = new BufferedWriter(new FileWriter(consumedDedupedFile), 1024*1024) + var total = 0 + var mismatched = 0 + while(produced.hasNext && consumed.hasNext) { + val p = produced.next() + producedDeduped.write(p.toString) + producedDeduped.newLine() + val c = consumed.next() + consumedDeduped.write(c.toString) + consumedDeduped.newLine() + if(p != c) + mismatched += 1 + total += 1 + } + producedDeduped.close() + consumedDeduped.close() + require(!produced.hasNext, "Additional values produced not found in consumer log.") + require(!consumed.hasNext, "Additional values consumed not found in producer log.") + println("Validated " + total + " values, " + mismatched + " mismatches.") + require(mismatched == 0, "Non-zero number of row mismatches.") + // if all the checks worked out we can delete the deduped files + producedDedupedFile.delete() + consumedDedupedFile.delete() + } + + def valuesIterator(reader: BufferedReader) = { + new IteratorTemplate[TestRecord] { + def makeNext(): TestRecord = { + var next = readNext(reader) + while(next != null && next.delete) + next = readNext(reader) + if(next == null) + allDone() + else + next + } + } + } + + def readNext(reader: BufferedReader): TestRecord = { + var line = reader.readLine() + if(line == null) + return null + var curr = new TestRecord(line) + while(true) { + line = peekLine(reader) + if(line == null) + return curr + val next = new TestRecord(line) + if(next == null || next.topicAndKey != curr.topicAndKey) + return curr + curr = next + reader.readLine() + } + null + } + + def peekLine(reader: BufferedReader) = { + reader.mark(4096) + val line = reader.readLine + reader.reset() + line + } + + def externalSort(file: File): BufferedReader = { + val builder = new ProcessBuilder("sort", "--key=1,2", "--stable", "--buffer-size=20%", "--temporary-directory=" + System.getProperty("java.io.tmpdir"), file.getAbsolutePath) + val process = builder.start() + new Thread() { + override def run() { + val exitCode = process.waitFor() + if(exitCode != 0) { + System.err.println("Process exited abnormally.") + while(process.getErrorStream.available > 0) { + System.err.write(process.getErrorStream().read()) + } + } + } + }.start() + new BufferedReader(new InputStreamReader(process.getInputStream()), 10*1024*1024) + } + + def produceMessages(brokerUrl: String, + topics: Array[String], + messages: Long, + dups: Int, + percentDeletes: Int): File = { + val producerProps = new Properties + producerProps.setProperty("producer.type", "async") + producerProps.setProperty("broker.list", brokerUrl) + producerProps.setProperty("serializer.class", classOf[StringEncoder].getName) + producerProps.setProperty("key.serializer.class", classOf[StringEncoder].getName) + producerProps.setProperty("queue.enqueue.timeout.ms", "-1") + producerProps.setProperty("batch.size", 1000.toString) + val producer = new Producer[String, String](new ProducerConfig(producerProps)) + val rand = new Random(1) + val keyCount = (messages / dups).toInt + val producedFile = File.createTempFile("kafka-log-cleaner-produced-", ".txt") + println("Logging produce requests to " + producedFile.getAbsolutePath) + val producedWriter = new BufferedWriter(new FileWriter(producedFile), 1024*1024) + for(i <- 0L until (messages * topics.length)) { + val topic = topics((i % topics.length).toInt) + val key = rand.nextInt(keyCount) + val delete = i % 100 < percentDeletes + val msg = + if(delete) + new KeyedMessage[String, String](topic = topic, key = key.toString, message = null) + else + new KeyedMessage[String, String](topic = topic, key = key.toString, message = i.toString) + producer.send(msg) + producedWriter.write(TestRecord(topic, key, i, delete).toString) + producedWriter.newLine() + } + producedWriter.close() + producer.close() + producedFile + } + + def makeConsumer(zkUrl: String, topics: Array[String]): ZookeeperConsumerConnector = { + val consumerProps = new Properties + consumerProps.setProperty("group.id", "log-cleaner-test-" + new Random().nextInt(Int.MaxValue)) + consumerProps.setProperty("zk.connect", zkUrl) + consumerProps.setProperty("consumer.timeout.ms", (10*1000).toString) + new ZookeeperConsumerConnector(new ConsumerConfig(consumerProps)) + } + + def consumeMessages(zkUrl: String, topics: Array[String]): File = { + val connector = makeConsumer(zkUrl, topics) + val streams = connector.createMessageStreams(topics.map(topic => (topic, 1)).toMap, new StringDecoder, new StringDecoder) + val consumedFile = File.createTempFile("kafka-log-cleaner-consumed-", ".txt") + println("Logging consumed messages to " + consumedFile.getAbsolutePath) + val consumedWriter = new BufferedWriter(new FileWriter(consumedFile)) + for(topic <- topics) { + val stream = streams(topic).head + try { + for(item <- stream) { + val delete = item.message == null + val value = if(delete) -1L else item.message.toLong + consumedWriter.write(TestRecord(topic, item.key.toInt, value, delete).toString) + consumedWriter.newLine() + } + } catch { + case e: ConsumerTimeoutException => + } + } + consumedWriter.close() + connector.shutdown() + consumedFile + } + +} + +case class TestRecord(val topic: String, val key: Int, val value: Long, val delete: Boolean) { + def this(pieces: Array[String]) = this(pieces(0), pieces(1).toInt, pieces(2).toLong, pieces(3) == "d") + def this(line: String) = this(line.split("\t")) + override def toString() = topic + "\t" + key + "\t" + value + "\t" + (if(delete) "d" else "u") + def topicAndKey = topic + key +} \ No newline at end of file diff --git a/core/src/test/scala/other/kafka/TestLogPerformance.scala b/core/src/test/scala/other/kafka/TestLogPerformance.scala deleted file mode 100644 index 9f3bb40728fa6..0000000000000 --- a/core/src/test/scala/other/kafka/TestLogPerformance.scala +++ /dev/null @@ -1,56 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.log - -import kafka.message._ -import kafka.utils.{SystemTime, TestUtils, Utils} -import kafka.server.KafkaConfig - -object TestLogPerformance { - - def main(args: Array[String]): Unit = { - if(args.length < 4) - Utils.croak("USAGE: java " + getClass().getName() + " num_messages message_size batch_size compression_codec") - val numMessages = args(0).toInt - val messageSize = args(1).toInt - val batchSize = args(2).toInt - val compressionCodec = CompressionCodec.getCompressionCodec(args(3).toInt) - val props = TestUtils.createBrokerConfig(0, -1) - val config = new KafkaConfig(props) - val dir = TestUtils.tempDir() - val log = new Log(dir, 50*1024*1024, config.messageMaxBytes, 5000000, config.logRollHours*60*60*1000L, needsRecovery = false, time = SystemTime) - val bytes = new Array[Byte](messageSize) - new java.util.Random().nextBytes(bytes) - val message = new Message(bytes) - val messages = new Array[Message](batchSize) - for(i <- 0 until batchSize) - messages(i) = message - val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec, messages = messages: _*) - val numBatches = numMessages / batchSize - val start = System.currentTimeMillis() - for(i <- 0 until numBatches) - log.append(messageSet) - log.close() - val elapsed = (System.currentTimeMillis() - start) / 1000.0 - val writtenBytes = MessageSet.entrySize(message) * numMessages - println("message size = " + MessageSet.entrySize(message)) - println("MB/sec: " + writtenBytes / elapsed / (1024.0 * 1024.0)) - Utils.rm(dir) - } - -} diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 24362890fe446..e4d739a3d7f1c 100644 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -61,13 +61,13 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { 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)) + brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.config.port, s.config.rackId)) // create topics with 1 partition, 2 replicas, one on each broker - CreateTopicCommand.createTopic(zkClient, topic1, 1, 2, "0:1") - CreateTopicCommand.createTopic(zkClient, topic2, 1, 2, "1:2") - CreateTopicCommand.createTopic(zkClient, topic3, 1, 4, "2:3:0:1") - CreateTopicCommand.createTopic(zkClient, topic4, 1, 4, "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,0,1))) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic4, Map(0->Seq(0,3))) // wait until leader is elected @@ -100,26 +100,26 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testTopicDoesNotExist { try { - AddPartitionsCommand.addPartitions(zkClient, "Blah", 1) + AdminUtils.addPartitions(zkClient, "Blah", 1) fail("Topic should not exist") } catch { - case e: AdministrationException => //this is good + case e: AdminOperationException => //this is good case e2: Throwable => throw e2 } } def testWrongReplicaCount { try { - AddPartitionsCommand.addPartitions(zkClient, topic1, 2, "0:1:2") + AdminUtils.addPartitions(zkClient, topic1, 2, "0:1,0:1:2") fail("Add partitions should fail") } catch { - case e: AdministrationException => //this is good + case e: AdminOperationException => //this is good case e2: Throwable => throw e2 } } def testIncrementPartitions { - AddPartitionsCommand.addPartitions(zkClient, topic1, 2) + AdminUtils.addPartitions(zkClient, topic1, 3) // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1, 500) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2, 500) @@ -144,7 +144,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testManualAssignmentOfReplicas { - AddPartitionsCommand.addPartitions(zkClient, topic2, 2, "0:1,2:3") + AdminUtils.addPartitions(zkClient, topic2, 3, "1:2,0:1,2:3") // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 1, 500) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 2, 500) @@ -170,7 +170,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testReplicaPlacement { - AddPartitionsCommand.addPartitions(zkClient, topic3, 6) + AdminUtils.addPartitions(zkClient, topic3, 7) // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 1, 500) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 2, 500) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index a480881d7f351..d8c7e864e4062 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -19,10 +19,15 @@ package kafka.admin import junit.framework.Assert._ import org.junit.Test import org.scalatest.junit.JUnit3Suite +import java.util.Properties +import kafka.utils._ +import kafka.log._ import kafka.zk.ZooKeeperTestHarness -import kafka.server.KafkaConfig import kafka.utils.{Logging, ZkUtils, TestUtils} -import kafka.common.{TopicExistsException, ErrorMapping, TopicAndPartition} +import kafka.cluster.{Broker, Cluster} +import kafka.common.{TopicExistsException, TopicAndPartition} +import kafka.server.{KafkaServer, KafkaConfig} +import java.io.File class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { @@ -32,28 +37,17 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val brokerList = List(0, 1, 2, 3, 4) // test 0 replication factor - try { - AdminUtils.assignReplicasToBrokers(brokerList, 10, 0) - fail("shouldn't allow replication factor 0") - } - catch { - case e: AdministrationException => // this is good - case e2: Throwable => throw e2 + intercept[AdminOperationException] { + AdminUtils.assignReplicasToBrokers(brokerList, ZkUtils.getFilteredCluster(zkClient, brokerList), 10, 0) } // test wrong replication factor - try { - AdminUtils.assignReplicasToBrokers(brokerList, 10, 6) - fail("shouldn't allow replication factor larger than # of brokers") - } - catch { - case e: AdministrationException => // this is good - case e2: Throwable => throw e2 + intercept[AdminOperationException] { + AdminUtils.assignReplicasToBrokers(brokerList, ZkUtils.getFilteredCluster(zkClient, brokerList), 10, 6) } // correct assignment - { - val expectedAssignment = Map( + val expectedAssignment = Map( 0 -> List(0, 1, 2), 1 -> List(1, 2, 3), 2 -> List(2, 3, 4), @@ -63,10 +57,70 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { 6 -> List(1, 3, 4), 7 -> List(2, 4, 0), 8 -> List(3, 0, 1), - 9 -> List(4, 1, 2) - ) + 9 -> List(4, 1, 2)) + + val actualAssignment = AdminUtils.assignReplicasToBrokers(brokerList, ZkUtils.getFilteredCluster(zkClient, brokerList), 10, 3, 0) + val e = (expectedAssignment.toList == actualAssignment.toList) + assertTrue(expectedAssignment.toList == actualAssignment.toList) + } + + @Test + def testRackReplicaAssignment() { + val brokerList = List(0, 1, 2, 3, 4) + + // test not enough distinct rack-ids + intercept[AdminOperationException] { + AdminUtils.assignReplicasToBrokers(brokerList, new Cluster(List( + Broker(0, "localhost", 0, 0), + Broker(1, "localhost", 0, 1), + Broker(2, "localhost", 0, 0), + Broker(3, "localhost", 0, 1), + Broker(4, "localhost", 0, 0))), 10, 3, maxReplicaPerRack = 1) + } - val actualAssignment = AdminUtils.assignReplicasToBrokers(brokerList, 10, 3, 0) + { // correct assignment with max-rack-replication = 1 + val expectedAssignment = Map( + 0 -> List(0, 1, 2), + 1 -> List(1, 2, 3), + 2 -> List(2, 3, 4), + 3 -> List(3, 4, 2), + 4 -> List(4, 0, 2), + 5 -> List(0, 2, 4), + 6 -> List(1, 3, 2), + 7 -> List(2, 4, 0), + 8 -> List(3, 1, 2), + 9 -> List(4, 2, 3)) + + val filteredCluster = new Cluster(List( + Broker(0, "localhost", 0, 0), + Broker(1, "localhost", 0, 1), + Broker(2, "localhost", 0, 2), + Broker(3, "localhost", 0, 0), + Broker(4, "localhost", 0, 1))) + val actualAssignment = AdminUtils.assignReplicasToBrokers(brokerList, filteredCluster, 10, 3, 0, maxReplicaPerRack = 1) + val e = (expectedAssignment.toList == actualAssignment.toList) + assertTrue(expectedAssignment.toList == actualAssignment.toList) + } + { // correct assignment with max-rack-replication = 2 + val expectedAssignment = Map( + 0 -> List(0, 1, 2), + 1 -> List(1, 2, 3), + 2 -> List(2, 3, 4), + 3 -> List(3, 4, 0), + 4 -> List(4, 0, 1), + 5 -> List(0, 2, 3), + 6 -> List(1, 3, 4), + 7 -> List(2, 4, 0), + 8 -> List(3, 0, 1), + 9 -> List(4, 1, 2)) + + val filteredCluster = new Cluster(List( + Broker(0, "localhost", 0, 0), + Broker(1, "localhost", 0, 1), + Broker(2, "localhost", 0, 2), + Broker(3, "localhost", 0, 0), + Broker(4, "localhost", 0, 1))) + val actualAssignment = AdminUtils.assignReplicasToBrokers(brokerList, filteredCluster, 10, 3, 0, maxReplicaPerRack = 2) val e = (expectedAssignment.toList == actualAssignment.toList) assertTrue(expectedAssignment.toList == actualAssignment.toList) } @@ -74,54 +128,25 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { @Test def testManualReplicaAssignment() { - val brokerList = Set(0, 1, 2, 3, 4) - - // duplicated brokers - try { - val replicationAssignmentStr = "0,0,1:1,2,3" - CreateTopicCommand.getManualReplicaAssignment(replicationAssignmentStr, brokerList) - fail("replication assginment shouldn't have duplicated brokers") - } - catch { - case e: AdministrationException => // this is good - case e2: Throwable => throw e2 - } + val brokers = List(0, 1, 2, 3, 4) + TestUtils.createBrokersInZk(zkClient, brokers) - // non-exist brokers - try { - val replicationAssignmentStr = "0,1,2:1,2,7" - CreateTopicCommand.getManualReplicaAssignment(replicationAssignmentStr, brokerList) - fail("replication assginment shouldn't contain non-exist brokers") - } - catch { - case e: AdministrationException => // this is good - case e2: Throwable => throw e2 + // duplicate brokers + intercept[IllegalArgumentException] { + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,0))) } // inconsistent replication factor - try { - val replicationAssignmentStr = "0,1,2:1,2" - CreateTopicCommand.getManualReplicaAssignment(replicationAssignmentStr, brokerList) - fail("all partitions should have the same replication factor") - } - catch { - case e: AdministrationException => // this is good - case e2: Throwable => throw e2 + intercept[IllegalArgumentException] { + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,1), 1->Seq(0))) } // good assignment - { - val replicationAssignmentStr = "0:1:2,1:2:3" - val expectedReplicationAssignment = Map( - 0 -> List(0, 1, 2), - 1 -> List(1, 2, 3) - ) - val actualReplicationAssignment = CreateTopicCommand.getManualReplicaAssignment(replicationAssignmentStr, brokerList) - assertEquals(expectedReplicationAssignment.size, actualReplicationAssignment.size) - for( (part, replicas) <- expectedReplicationAssignment ) { - assertEquals(replicas, actualReplicationAssignment(part)) - } - } + val assignment = Map(0 -> List(0, 1, 2), + 1 -> List(1, 2, 3)) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", assignment) + val found = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq("test")) + assertEquals(assignment, found("test")) } @Test @@ -157,7 +182,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.createOrUpdateTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient) + 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 @@ -165,15 +190,18 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { for(i <- 0 until actualReplicaList.size) assertEquals(expectedReplicaAssignment.get(i).get, actualReplicaList(i)) - try { - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient) - fail("shouldn't be able to create a topic already exists") - } catch { - case e: TopicExistsException => // this is good - case e2: Throwable => throw e2 + intercept[TopicExistsException] { + // shouldn't be able to create a topic that already exists + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) } } + private def getBrokersWithPartitionDir(servers: Iterable[KafkaServer], topic: String, partitionId: Int): Set[Int] = { + servers.filter(server => new File(server.config.logDirs.head, topic + "-" + partitionId).exists) + .map(_.config.brokerId) + .toSet + } + @Test def testPartitionReassignmentWithLeaderInNewReplicas() { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) @@ -181,7 +209,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.createOrUpdateTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 val newReplicas = Seq(0, 2, 3) val partitionToBeReassigned = 0 @@ -195,7 +223,11 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; }, 1000) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) + // in sync replicas should not have any replica that is not in the new assigned replicas + checkForPhantomInSyncReplicas(topic, partitionToBeReassigned, assignedReplicas) assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas) + ensureNoUnderReplicatedPartitions(topic, partitionToBeReassigned, assignedReplicas, servers) + assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) servers.foreach(_.shutdown()) } @@ -206,7 +238,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.createOrUpdateTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 val newReplicas = Seq(1, 2, 3) val partitionToBeReassigned = 0 @@ -221,7 +253,9 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { }, 1000) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas) - // leader should be 2 + checkForPhantomInSyncReplicas(topic, partitionToBeReassigned, assignedReplicas) + ensureNoUnderReplicatedPartitions(topic, partitionToBeReassigned, assignedReplicas, servers) + assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) servers.foreach(_.shutdown()) } @@ -232,7 +266,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.createOrUpdateTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 val newReplicas = Seq(2, 3) val partitionToBeReassigned = 0 @@ -247,7 +281,9 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { }, 2000) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 2, 3", newReplicas, assignedReplicas) - // leader should be 2 + checkForPhantomInSyncReplicas(topic, partitionToBeReassigned, assignedReplicas) + ensureNoUnderReplicatedPartitions(topic, partitionToBeReassigned, assignedReplicas, servers) + assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) servers.foreach(_.shutdown()) } @@ -264,7 +300,6 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) val reassignedPartitions = ZkUtils.getPartitionsBeingReassigned(zkClient) assertFalse("Partition should not be reassigned", reassignedPartitions.contains(topicAndPartition)) - // leader should be 2 servers.foreach(_.shutdown()) } @@ -273,7 +308,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1)) val topic = "test" // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // put the partition in the reassigned path as well // reassign partition 0 val newReplicas = Seq(0, 1) @@ -286,6 +321,10 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { TestUtils.waitUntilTrue(checkIfReassignPartitionPathExists, 1000) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 0, 1", newReplicas, assignedReplicas) + checkForPhantomInSyncReplicas(topic, partitionToBeReassigned, assignedReplicas) + // ensure that there are no under replicated partitions + ensureNoUnderReplicatedPartitions(topic, partitionToBeReassigned, assignedReplicas, servers) + assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) servers.foreach(_.shutdown()) } @@ -312,7 +351,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { // create brokers val serverConfigs = TestUtils.createBrokerConfigs(3).map(new KafkaConfig(_)) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient) + 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 @@ -333,8 +372,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.createOrUpdateTopicPartitionAssignmentPathInZK(topic, expectedReplicaAssignment, zkClient) - + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) TestUtils.waitUntilMetadataIsPropagated(servers, topic, partition, 1000) val controllerId = ZkUtils.getController(zkClient) @@ -370,6 +408,70 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { } } + /** + * This test creates a topic with a few config overrides and checks that the configs are applied to the new topic + * then changes the config and checks that the new values take effect. + */ + @Test + def testTopicConfigChange() { + val partitions = 3 + val topic = "my-topic" + val server = TestUtils.createServer(new KafkaConfig(TestUtils.createBrokerConfig(0))) + + def makeConfig(messageSize: Int, retentionMs: Long) = { + var props = new Properties() + props.setProperty(LogConfig.MaxMessageBytesProp, messageSize.toString) + props.setProperty(LogConfig.RententionMsProp, retentionMs.toString) + props + } + + def checkConfig(messageSize: Int, retentionMs: Long) { + TestUtils.retry(10000) { + for(part <- 0 until partitions) { + val logOpt = server.logManager.getLog(TopicAndPartition(topic, part)) + assertTrue(logOpt.isDefined) + assertEquals(retentionMs, logOpt.get.config.retentionMs) + assertEquals(messageSize, logOpt.get.config.maxMessageSize) + } + } + } + + try { + // create a topic with a few config overrides and check that they are applied + val maxMessageSize = 1024 + val retentionMs = 1000*1000 + AdminUtils.createTopic(server.zkClient, topic, partitions, 1, topicConfig = makeConfig(maxMessageSize, retentionMs)) + checkConfig(maxMessageSize, retentionMs) + + // now double the config values for the topic and check that it is applied + AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs)) + checkConfig(2*maxMessageSize, 2 * retentionMs) + } finally { + server.shutdown() + server.config.logDirs.map(Utils.rm(_)) + } + } + + private def checkForPhantomInSyncReplicas(topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int]) { + val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionToBeReassigned) + // in sync replicas should not have any replica that is not in the new assigned replicas + val phantomInSyncReplicas = inSyncReplicas.toSet -- assignedReplicas.toSet + assertTrue("All in sync replicas %s must be in the assigned replica list %s".format(inSyncReplicas, assignedReplicas), + phantomInSyncReplicas.size == 0) + } + + private def ensureNoUnderReplicatedPartitions(topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int], + servers: Seq[KafkaServer]) { + val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionToBeReassigned) + assertFalse("Reassigned partition [%s,%d] is underreplicated".format(topic, partitionToBeReassigned), + inSyncReplicas.size < assignedReplicas.size) + val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitionToBeReassigned) + assertTrue("Reassigned partition [%s,%d] is unavailable".format(topic, partitionToBeReassigned), leader.isDefined) + val leaderBroker = servers.filter(s => s.config.brokerId == leader.get).head + assertTrue("Reassigned partition [%s,%d] is underreplicated as reported by the leader %d".format(topic, partitionToBeReassigned, leader.get), + leaderBroker.replicaManager.underReplicatedPartitionCount() == 0) + } + private def checkIfReassignPartitionPathExists(): Boolean = { ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath) } diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index f43ac8f0f4ff3..b411487302924 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -24,7 +24,7 @@ import java.nio.ByteBuffer import kafka.message.{Message, ByteBufferMessageSet} import kafka.cluster.Broker import collection.mutable._ -import kafka.common.{TopicAndPartition, ErrorMapping} +import kafka.common.{TopicAndPartition, ErrorMapping, OffsetMetadataAndError} import kafka.controller.LeaderIsrAndControllerEpoch @@ -35,10 +35,10 @@ object SerializationTestUtils{ private val isr1 = List(0, 1, 2) private val leader2 = 0 private val isr2 = List(0, 2, 3) - private val partitionDataFetchResponse0 = new FetchResponsePartitionData(new ByteBufferMessageSet(new Message("first message".getBytes))) - private val partitionDataFetchResponse1 = new FetchResponsePartitionData(new ByteBufferMessageSet(new Message("second message".getBytes))) - private val partitionDataFetchResponse2 = new FetchResponsePartitionData(new ByteBufferMessageSet(new Message("third message".getBytes))) - private val partitionDataFetchResponse3 = new FetchResponsePartitionData(new ByteBufferMessageSet(new Message("fourth message".getBytes))) + private val partitionDataFetchResponse0 = new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) + private val partitionDataFetchResponse1 = new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("second message".getBytes))) + private val partitionDataFetchResponse2 = new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("third message".getBytes))) + private val partitionDataFetchResponse3 = new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("fourth message".getBytes))) private val partitionDataFetchResponseMap = Map((0, partitionDataFetchResponse0), (1, partitionDataFetchResponse1), (2, partitionDataFetchResponse2), (3, partitionDataFetchResponse3)) private val topicDataFetchResponse = { @@ -75,7 +75,7 @@ object SerializationTestUtils{ TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100) ) - private val brokers = List(new Broker(0, "localhost", 1011), new Broker(1, "localhost", 1012), new Broker(2, "localhost", 1013)) + private val brokers = List(new Broker(0, "localhost", 1011, 0), new Broker(1, "localhost", 1012, 0), new Broker(2, "localhost", 1013, 0)) private val partitionMetaData0 = new PartitionMetadata(0, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 0) private val partitionMetaData1 = new PartitionMetadata(1, Some(brokers.head), replicas = brokers, isr = brokers.tail, errorCode = 1) private val partitionMetaData2 = new PartitionMetadata(2, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 2) @@ -145,6 +145,36 @@ object SerializationTestUtils{ def createTestTopicMetadataResponse: TopicMetadataResponse = { new TopicMetadataResponse(Seq(topicmetaData1, topicmetaData2), 1) } + + def createTestOffsetCommitRequest: OffsetCommitRequest = { + new OffsetCommitRequest("group 1", collection.immutable.Map( + TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(offset=42L, metadata="some metadata"), + TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(offset=100L, metadata=OffsetMetadataAndError.NoMetadata) + )) + } + + def createTestOffsetCommitResponse: OffsetCommitResponse = { + new OffsetCommitResponse(collection.immutable.Map( + TopicAndPartition(topic1, 0) -> ErrorMapping.NoError, + TopicAndPartition(topic1, 1) -> ErrorMapping.UnknownTopicOrPartitionCode + )) + } + + def createTestOffsetFetchRequest: OffsetFetchRequest = { + new OffsetFetchRequest("group 1", Seq( + TopicAndPartition(topic1, 0), + TopicAndPartition(topic1, 1) + )) + } + + def createTestOffsetFetchResponse: OffsetFetchResponse = { + new OffsetFetchResponse(collection.immutable.Map( + TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(42L, "some metadata", ErrorMapping.NoError), + TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetMetadataAndError.NoMetadata, + ErrorMapping.UnknownTopicOrPartitionCode) + )) + } + } class RequestResponseSerializationTest extends JUnitSuite { @@ -159,6 +189,10 @@ class RequestResponseSerializationTest extends JUnitSuite { private val offsetResponse = SerializationTestUtils.createTestOffsetResponse private val topicMetadataRequest = SerializationTestUtils.createTestTopicMetadataRequest private val topicMetadataResponse = SerializationTestUtils.createTestTopicMetadataResponse + private val offsetCommitRequest = SerializationTestUtils.createTestOffsetCommitRequest + private val offsetCommitResponse = SerializationTestUtils.createTestOffsetCommitResponse + private val offsetFetchRequest = SerializationTestUtils.createTestOffsetFetchRequest + private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse @Test @@ -239,5 +273,34 @@ class RequestResponseSerializationTest extends JUnitSuite { val deserializedTopicMetadataResponse = TopicMetadataResponse.readFrom(buffer) assertEquals("The original and deserialzed topicMetadataResponse should be the same", topicMetadataResponse, deserializedTopicMetadataResponse) + + buffer = ByteBuffer.allocate(offsetCommitRequest.sizeInBytes) + offsetCommitRequest.writeTo(buffer) + buffer.rewind() + val deserializedOffsetCommitRequest = OffsetCommitRequest.readFrom(buffer) + assertEquals("The original and deserialzed offsetCommitRequest should be the same", offsetCommitRequest, + deserializedOffsetCommitRequest) + + buffer = ByteBuffer.allocate(offsetCommitResponse.sizeInBytes) + offsetCommitResponse.writeTo(buffer) + buffer.rewind() + val deserializedOffsetCommitResponse = OffsetCommitResponse.readFrom(buffer) + assertEquals("The original and deserialzed offsetCommitResponse should be the same", offsetCommitResponse, + deserializedOffsetCommitResponse) + + buffer = ByteBuffer.allocate(offsetFetchRequest.sizeInBytes) + offsetFetchRequest.writeTo(buffer) + buffer.rewind() + val deserializedOffsetFetchRequest = OffsetFetchRequest.readFrom(buffer) + assertEquals("The original and deserialzed offsetFetchRequest should be the same", offsetFetchRequest, + deserializedOffsetFetchRequest) + + buffer = ByteBuffer.allocate(offsetFetchResponse.sizeInBytes) + offsetFetchResponse.writeTo(buffer) + buffer.rewind() + val deserializedOffsetFetchResponse = OffsetFetchResponse.readFrom(buffer) + assertEquals("The original and deserialzed offsetFetchResponse should be the same", offsetFetchResponse, + deserializedOffsetFetchResponse) + } } diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index 1ee34b95d5fe2..57b9179cef6e6 100644 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -20,6 +20,7 @@ package kafka.consumer import java.util.concurrent._ import java.util.concurrent.atomic._ +import java.util.Properties import scala.collection._ import junit.framework.Assert._ @@ -27,7 +28,7 @@ import kafka.message._ import kafka.server._ import kafka.utils.TestUtils._ import kafka.utils._ -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import org.junit.Test import kafka.serializer._ import kafka.cluster.{Broker, Cluster} @@ -47,7 +48,7 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { val group = "group1" val consumer0 = "consumer0" val consumedOffset = 5 - val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) + val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port, c.rackId))) val queue = new LinkedBlockingQueue[FetchedDataChunk] val topicInfos = configs.map(c => new PartitionTopicInfo(topic, 0, @@ -60,7 +61,7 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { override def setUp() { super.setUp - CreateTopicCommand.createTopic(zkClient, topic, 1, 1, configs.head.brokerId.toString) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(configs.head.brokerId)), new Properties) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) } @@ -87,4 +88,40 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { val unconsumed = messageSet.filter(_.offset >= consumedOffset).map(m => Utils.readString(m.message.payload)) assertEquals(unconsumed, receivedMessages) } + + @Test + def testConsumerIteratorDecodingFailure() { + val messageStrings = (0 until 10).map(_.toString).toList + val messages = messageStrings.map(s => new Message(s.getBytes)) + val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new AtomicLong(0), messages:_*) + + topicInfos(0).enqueue(messageSet) + assertEquals(1, queue.size) + + val iter = new ConsumerIterator[String, String](queue, + ConsumerConfig.ConsumerTimeoutMs, + new FailDecoder(), + new FailDecoder(), + clientId = "") + + val receivedMessages = (0 until 5).map{ i => + assertTrue(iter.hasNext) + val message = iter.next + assertEquals(message.offset, i + consumedOffset) + + try { + message.message // should fail + } + catch { + case e: UnsupportedOperationException => // this is ok + case e2: Throwable => fail("Unexpected exception when iterating the message set. " + e2.getMessage) + } + } + } + + class FailDecoder(props: VerifiableProperties = null) extends Decoder[String] { + def fromBytes(bytes: Array[Byte]): String = { + throw new UnsupportedOperationException("This decoder does not work at all..") + } + } } diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index 40a2bf7a9277e..cf2724bb68d39 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -29,16 +29,13 @@ class TopicFilterTest extends JUnitSuite { def testWhitelists() { val topicFilter1 = new Whitelist("white1,white2") - assertFalse(topicFilter1.requiresTopicEventWatcher) assertTrue(topicFilter1.isTopicAllowed("white2")) assertFalse(topicFilter1.isTopicAllowed("black1")) val topicFilter2 = new Whitelist(".+") - assertTrue(topicFilter2.requiresTopicEventWatcher) assertTrue(topicFilter2.isTopicAllowed("alltopics")) val topicFilter3 = new Whitelist("white_listed-topic.+") - assertTrue(topicFilter3.requiresTopicEventWatcher) assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1")) assertFalse(topicFilter3.isTopicAllowed("black1")) } @@ -46,6 +43,5 @@ class TopicFilterTest extends JUnitSuite { @Test def testBlacklists() { val topicFilter1 = new Blacklist("black1") - assertTrue(topicFilter1.requiresTopicEventWatcher) } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 268d14e8e22cf..8fe7259c9abd2 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -25,7 +25,7 @@ import scala.collection._ import org.scalatest.junit.JUnit3Suite import kafka.message._ import kafka.serializer._ -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import org.I0Itec.zkclient.ZkClient import kafka.utils._ import kafka.producer.{ProducerConfig, KeyedMessage, Producer} @@ -310,7 +310,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkClient = new ZkClient(zookeeperConnect, 6000, 30000, ZKStringSerializer) // create topic topic1 with 1 partition on broker 0 - CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0") + AdminUtils.createTopic(zkClient, topic, 1, 1) // send some messages to each broker val sentMessages1 = sendMessages(configs.head, nMessages, "batch1", NoCompressionCodec, 1) @@ -406,10 +406,12 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar } def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = { - import scala.collection.JavaConversions._ val children = zkClient.getChildren(path) Collections.sort(children) - val childrenAsSeq : Seq[java.lang.String] = (children: mutable.Buffer[String]).toSeq + val childrenAsSeq : Seq[java.lang.String] = { + import JavaConversions._ + children.toSeq + } childrenAsSeq.map(partition => (partition, zkClient.readData(path + "/" + partition).asInstanceOf[String])) } diff --git a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala index 2317760ef977a..e5703bc16b2ce 100644 --- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala +++ b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala @@ -32,8 +32,7 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L val topic = "test_topic" val group = "default_group" val testConsumer = "consumer" - val BrokerPort = 9892 - val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0, BrokerPort))) + val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0))) val NumMessages = 10 val LargeOffset = 10000 val SmallOffset = -1 diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index c5cddeac2da9b..a5b9d72957cbe 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -30,7 +30,7 @@ import kafka.serializer._ import kafka.producer.{KeyedMessage, Producer} import kafka.utils.TestUtils._ import kafka.utils.TestUtils -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { @@ -40,7 +40,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { yield new KafkaConfig(props) val messages = new mutable.HashMap[Int, Seq[Array[Byte]]] val topic = "topic" - val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) + val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port, c.rackId))) val shutdown = ZookeeperConsumerConnector.shutdownCommand val queue = new LinkedBlockingQueue[FetchedDataChunk] val topicInfos = configs.map(c => new PartitionTopicInfo(topic, @@ -55,7 +55,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { override def setUp() { super.setUp - CreateTopicCommand.createTopic(zkClient, topic, 1, 1, configs.head.brokerId.toString) + 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() @@ -88,7 +88,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { new StringEncoder()) val ms = 0.until(messagesPerNode).map(x => (conf.brokerId * 5 + x).toString.getBytes).toArray messages += conf.brokerId -> ms - producer.send(ms.map(m => KeyedMessage[String, Array[Byte]](topic, topic, m)):_*) + producer.send(ms.map(m => new KeyedMessage[String, Array[Byte]](topic, topic, m)):_*) producer.close() count += ms.size } diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index f764151b2a548..5f331d22bc99b 100644 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -27,7 +27,7 @@ import org.I0Itec.zkclient.ZkClient import kafka.zk.ZooKeeperTestHarness import org.scalatest.junit.JUnit3Suite import scala.collection._ -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionException, OffsetOutOfRangeException} import kafka.utils.{TestUtils, Utils} @@ -42,19 +42,6 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val configs = List(config) val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) - override def setUp() { - super.setUp - // temporarily set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.FATAL) - } - - override def tearDown() { - // restore set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.ERROR) - - super.tearDown - } - def testFetchRequestCanProperlySerialize() { val request = new FetchRequestBuilder() .clientId("test-client") @@ -299,7 +286,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with def testConsumerEmptyTopic() { val newTopic = "new-topic" - CreateTopicCommand.createTopic(zkClient, newTopic, 1, 1, config.brokerId.toString) + AdminUtils.createTopic(zkClient, newTopic, 1, 1) TestUtils.waitUntilMetadataIsPropagated(servers, newTopic, 0, 1000) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, newTopic, 0, 500) val fetchResponse = consumer.fetch(new FetchRequestBuilder().addFetch(newTopic, 0, 0, 10000).build()) @@ -326,10 +313,10 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with } // wait until the messages are published - TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog("test1", 0).get.logEndOffset == 2 }, 1000) - TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog("test2", 0).get.logEndOffset == 2 }, 1000) - TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog("test3", 0).get.logEndOffset == 2 }, 1000) - TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog("test4", 0).get.logEndOffset == 2 }, 1000) + TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog(TopicAndPartition("test1", 0)).get.logEndOffset == 2 }, 1000) + TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog(TopicAndPartition("test2", 0)).get.logEndOffset == 2 }, 1000) + TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog(TopicAndPartition("test3", 0)).get.logEndOffset == 2 }, 1000) + TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog(TopicAndPartition("test4", 0)).get.logEndOffset == 2 }, 1000) val replicaId = servers.head.config.brokerId val hwWaitMs = config.replicaHighWatermarkCheckpointIntervalMs @@ -353,7 +340,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with */ def createSimpleTopicsAndAwaitLeader(zkClient: ZkClient, topics: Seq[String], brokerId: Int) { for( topic <- topics ) { - CreateTopicCommand.createTopic(zkClient, topic, 1, 1, brokerId.toString) + AdminUtils.createTopic(zkClient, topic, 1, 1) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) } } diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala index 26e9bd6e4e1f2..b585f0ec0b1c4 100644 --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -19,7 +19,6 @@ package kafka.server import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness -import kafka.admin.CreateTopicCommand import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} @@ -27,6 +26,7 @@ import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, Control import kafka.cluster.Broker import kafka.common.ErrorMapping import kafka.api._ +import kafka.admin.AdminUtils class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { val brokerId1 = 0 @@ -79,11 +79,10 @@ class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { val topic4 = "new-topic4" // create topics with 1 partition, 2 replicas, one on each broker - CreateTopicCommand.createTopic(zkClient, topic1, 1, 2, "0:1") - CreateTopicCommand.createTopic(zkClient, topic2, 1, 2, "1:2") - CreateTopicCommand.createTopic(zkClient, topic3, 1, 2, "2:3") - CreateTopicCommand.createTopic(zkClient, topic4, 1, 2, "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/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index edf855528b27f..7650ba5ef8b2f 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -19,7 +19,7 @@ package kafka.integration import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import java.nio.ByteBuffer import junit.framework.Assert._ import kafka.cluster.Broker @@ -34,7 +34,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { val props = createBrokerConfigs(1) val configs = props.map(p => new KafkaConfig(p)) private var server1: KafkaServer = null - val brokers = configs.map(c => new Broker(c.brokerId,c.hostName,c.port)) + val brokers = configs.map(c => new Broker(c.brokerId,c.hostName,c.port, c.rackId)) override def setUp() { super.setUp() @@ -49,7 +49,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { def testTopicMetadataRequest { // create topic val topic = "test" - CreateTopicCommand.createTopic(zkClient, topic, 1) + AdminUtils.createTopic(zkClient, topic, 1, 1) // create a topic metadata request val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0) @@ -65,7 +65,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { def testBasicTopicMetadata { // create topic val topic = "test" - CreateTopicCommand.createTopic(zkClient, topic, 1) + AdminUtils.createTopic(zkClient, topic, 1, 1) TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0, 1000) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", @@ -84,8 +84,8 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { // create topic val topic1 = "testGetAllTopicMetadata1" val topic2 = "testGetAllTopicMetadata2" - CreateTopicCommand.createTopic(zkClient, topic1, 1) - CreateTopicCommand.createTopic(zkClient, topic2, 1) + AdminUtils.createTopic(zkClient, topic1, 1, 1) + AdminUtils.createTopic(zkClient, topic2, 1, 1) // wait for leader to be elected for both topics TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic1, 0, 1000) @@ -133,4 +133,4 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(1, partitionMetadata.head.replicas.size) assertTrue(partitionMetadata.head.leader.isDefined) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala index e8e454fa782b3..43af649f32976 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala @@ -21,7 +21,7 @@ import junit.framework.Assert._ import kafka.integration.KafkaServerTestHarness import kafka.server._ import org.scalatest.junit.JUnit3Suite -import scala.collection.JavaConversions._ +import scala.collection.JavaConversions import org.apache.log4j.{Level, Logger} import kafka.message._ import kafka.serializer._ @@ -84,7 +84,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar for (partition <- 0 until numParts) { val ms = 0.until(messagesPerNode).map(x => header + conf.brokerId + "-" + partition + "-" + x) messages ++= ms - import scala.collection.JavaConversions._ + import JavaConversions._ javaProducer.send(ms.map(new KeyedMessage[Int, String](topic, partition, _)): java.util.List[KeyedMessage[Int, String]]) } javaProducer.close diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala new file mode 100644 index 0000000000000..51cd94b316764 --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -0,0 +1,250 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +import junit.framework.Assert._ +import org.scalatest.junit.JUnitSuite +import org.junit.{After, Test} +import java.nio._ +import java.io.File +import scala.collection._ +import kafka.common._ +import kafka.utils._ +import kafka.message._ + +/** + * Unit tests for the log cleaning logic + */ +class CleanerTest extends JUnitSuite { + + val dir = TestUtils.tempDir() + val logConfig = LogConfig(segmentSize=1024, maxIndexSize=1024, dedupe=true) + val time = new MockTime() + val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time) + + @After + def teardown() { + Utils.rm(dir) + } + + /** + * Test simple log cleaning + */ + @Test + def testCleanSegments() { + val cleaner = makeCleaner(Int.MaxValue) + val log = makeLog(config = logConfig.copy(segmentSize = 1024)) + + // append messages to the log until we have four segments + while(log.numberOfSegments < 4) + log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) + val keysFound = keysInLog(log) + assertEquals((0L until log.logEndOffset), keysFound) + + // pretend we have the following keys + val keys = immutable.ListSet(1, 3, 5, 7, 9) + val map = new FakeOffsetMap(Int.MaxValue) + keys.foreach(k => map.put(key(k), Long.MaxValue)) + + // clean the log + cleaner.cleanSegments(log, log.logSegments.take(3).toSeq, map, 0L) + val shouldRemain = keysInLog(log).filter(!keys.contains(_)) + assertEquals(shouldRemain, keysInLog(log)) + } + + @Test + def testCleaningWithDeletes() { + val cleaner = makeCleaner(Int.MaxValue) + val log = makeLog(config = logConfig.copy(segmentSize = 1024)) + + // append messages with the keys 0 through N + while(log.numberOfSegments < 2) + log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) + + // delete all even keys between 0 and N + val leo = log.logEndOffset + for(key <- 0 until leo.toInt by 2) + log.append(deleteMessage(key)) + + // append some new unique keys to pad out to a new active segment + while(log.numberOfSegments < 4) + log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) + + cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0)) + val keys = keysInLog(log).toSet + assertTrue("None of the keys we deleted should still exist.", + (0 until leo.toInt by 2).forall(!keys.contains(_))) + } + + /* extract all the keys from a log */ + def keysInLog(log: Log): Iterable[Int] = + log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).map(m => Utils.readString(m.message.key).toInt)) + + def abortCheckDone(topicAndPartition: TopicAndPartition) { + throw new LogCleaningAbortedException() + } + + /** + * Test that abortion during cleaning throws a LogCleaningAbortedException + */ + @Test + def testCleanSegmentsWithAbort() { + val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) + val log = makeLog(config = logConfig.copy(segmentSize = 1024)) + + // append messages to the log until we have four segments + while(log.numberOfSegments < 4) + log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) + + val keys = keysInLog(log) + val map = new FakeOffsetMap(Int.MaxValue) + keys.foreach(k => map.put(key(k), Long.MaxValue)) + intercept[LogCleaningAbortedException] { + cleaner.cleanSegments(log, log.logSegments.take(3).toSeq, map, 0L) + } + } + + /** + * Validate the logic for grouping log segments together for cleaning + */ + @Test + def testSegmentGrouping() { + val cleaner = makeCleaner(Int.MaxValue) + val log = makeLog(config = logConfig.copy(segmentSize = 300, indexInterval = 1)) + + // append some messages to the log + var i = 0 + while(log.numberOfSegments < 10) { + log.append(TestUtils.singleMessageSet("hello".getBytes)) + i += 1 + } + + // grouping by very large values should result in a single group with all the segments in it + var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue) + assertEquals(1, groups.size) + assertEquals(log.numberOfSegments, groups(0).size) + checkSegmentOrder(groups) + + // grouping by very small values should result in all groups having one entry + groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue) + assertEquals(log.numberOfSegments, groups.size) + assertTrue("All groups should be singletons.", groups.forall(_.size == 1)) + checkSegmentOrder(groups) + groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = 1) + assertEquals(log.numberOfSegments, groups.size) + assertTrue("All groups should be singletons.", groups.forall(_.size == 1)) + checkSegmentOrder(groups) + + val groupSize = 3 + + // check grouping by log size + val logSize = log.logSegments.take(groupSize).map(_.size).sum.toInt + 1 + groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue) + checkSegmentOrder(groups) + assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize)) + + // check grouping by index size + val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes()).sum + 1 + groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize) + checkSegmentOrder(groups) + assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize)) + } + + private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]) { + val offsets = groups.flatMap(_.map(_.baseOffset)) + assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets) + } + + /** + * Test building an offset map off the log + */ + @Test + def testBuildOffsetMap() { + val map = new FakeOffsetMap(1000) + val log = makeLog() + val cleaner = makeCleaner(Int.MaxValue) + val start = 0 + val end = 500 + val offsets = writeToLog(log, (start until end) zip (start until end)) + def checkRange(map: FakeOffsetMap, start: Int, end: Int) { + val endOffset = cleaner.buildOffsetMap(log, start, end, map) + 1 + assertEquals("Last offset should be the end offset.", end, endOffset) + assertEquals("Should have the expected number of messages in the map.", end-start, map.size) + for(i <- start until end) + assertEquals("Should find all the keys", i.toLong, map.get(key(i))) + assertEquals("Should not find a value too small", -1L, map.get(key(start-1))) + assertEquals("Should not find a value too large", -1L, map.get(key(end))) + } + val segments = log.logSegments.toSeq + checkRange(map, 0, segments(1).baseOffset.toInt) + checkRange(map, segments(1).baseOffset.toInt, segments(3).baseOffset.toInt) + checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt) + } + + def makeLog(dir: File = dir, config: LogConfig = logConfig) = + new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time) + + def noOpCheckDone(topicAndPartition: TopicAndPartition) { /* do nothing */ } + + def makeCleaner(capacity: Int, checkDone: (TopicAndPartition) => Unit = noOpCheckDone) = + new Cleaner(id = 0, + offsetMap = new FakeOffsetMap(capacity), + ioBufferSize = 64*1024, + maxIoBufferSize = 64*1024, + dupBufferLoadFactor = 0.75, + throttler = throttler, + time = time, + checkDone = checkDone ) + + def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = { + for((key, value) <- seq) + yield log.append(message(key, value)).firstOffset + } + + def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes) + + def message(key: Int, value: Int) = + new ByteBufferMessageSet(new Message(key=key.toString.getBytes, bytes=value.toString.getBytes)) + + def deleteMessage(key: Int) = + new ByteBufferMessageSet(new Message(key=key.toString.getBytes, bytes=null)) + +} + +class FakeOffsetMap(val slots: Int) extends OffsetMap { + val map = new java.util.HashMap[String, Long]() + + private def keyFor(key: ByteBuffer) = + new String(Utils.readBytes(key.duplicate), "UTF-8") + + def put(key: ByteBuffer, offset: Long): Unit = + map.put(keyFor(key), offset) + + def get(key: ByteBuffer): Long = { + val k = keyFor(key) + if(map.containsKey(k)) + map.get(k) + else + -1L + } + + def clear() = map.clear() + + def size: Int = map.size + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index d0044cf49c996..cec1caecc5150 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -35,13 +35,21 @@ class FileMessageSetTest extends BaseMessageSetTestCases { set } + /** + * Test that the cached size variable matches the actual file size as we append messages + */ @Test def testFileSize() { assertEquals(messageSet.channel.size, messageSet.sizeInBytes) - messageSet.append(singleMessageSet("abcd".getBytes())) - assertEquals(messageSet.channel.size, messageSet.sizeInBytes) + for(i <- 0 until 20) { + messageSet.append(singleMessageSet("abcd".getBytes)) + assertEquals(messageSet.channel.size, messageSet.sizeInBytes) + } } + /** + * Test that adding invalid bytes to the end of the log doesn't break iteration + */ @Test def testIterationOverPartialAndTruncation() { testPartialWrite(0, messageSet) @@ -62,6 +70,9 @@ class FileMessageSetTest extends BaseMessageSetTestCases { checkEquals(messages.iterator, messageSet.map(m => m.message).iterator) } + /** + * Iterating over the file does file reads but shouldn't change the position of the underlying FileChannel. + */ @Test def testIterationDoesntChangePosition() { val position = messageSet.channel.position @@ -69,39 +80,71 @@ class FileMessageSetTest extends BaseMessageSetTestCases { assertEquals(position, messageSet.channel.position) } + /** + * Test a simple append and read. + */ @Test def testRead() { - val read = messageSet.read(0, messageSet.sizeInBytes) + var read = messageSet.read(0, messageSet.sizeInBytes) checkEquals(messageSet.iterator, read.iterator) val items = read.iterator.toList val sec = items.tail.head - val read2 = messageSet.read(MessageSet.entrySize(sec.message), messageSet.sizeInBytes) - checkEquals(items.tail.iterator, read2.iterator) + read = messageSet.read(position = MessageSet.entrySize(sec.message), size = messageSet.sizeInBytes) + assertEquals("Try a read starting from the second message", items.tail, read.toList) + read = messageSet.read(MessageSet.entrySize(sec.message), MessageSet.entrySize(sec.message)) + assertEquals("Try a read of a single message starting from the second message", List(items.tail.head), read.toList) } + /** + * Test the MessageSet.searchFor API. + */ @Test def testSearch() { // append a new message with a high offset val lastMessage = new Message("test".getBytes) messageSet.append(new ByteBufferMessageSet(NoCompressionCodec, new AtomicLong(50), lastMessage)) - var physicalOffset = 0 + var position = 0 assertEquals("Should be able to find the first message by its offset", - OffsetPosition(0L, physicalOffset), + OffsetPosition(0L, position), messageSet.searchFor(0, 0)) - physicalOffset += MessageSet.entrySize(messageSet.head.message) + position += MessageSet.entrySize(messageSet.head.message) assertEquals("Should be able to find second message when starting from 0", - OffsetPosition(1L, physicalOffset), + OffsetPosition(1L, position), messageSet.searchFor(1, 0)) assertEquals("Should be able to find second message starting from its offset", - OffsetPosition(1L, physicalOffset), - messageSet.searchFor(1, physicalOffset)) - physicalOffset += MessageSet.entrySize(messageSet.tail.head.message) - assertEquals("Should be able to find third message from a non-existant offset", - OffsetPosition(50L, physicalOffset), - messageSet.searchFor(3, physicalOffset)) - assertEquals("Should be able to find third message by correct offset", - OffsetPosition(50L, physicalOffset), - messageSet.searchFor(50, physicalOffset)) + OffsetPosition(1L, position), + messageSet.searchFor(1, position)) + position += MessageSet.entrySize(messageSet.tail.head.message) + MessageSet.entrySize(messageSet.tail.tail.head.message) + assertEquals("Should be able to find fourth message from a non-existant offset", + OffsetPosition(50L, position), + messageSet.searchFor(3, position)) + assertEquals("Should be able to find fourth message by correct offset", + OffsetPosition(50L, position), + messageSet.searchFor(50, position)) + } + + /** + * Test that the message set iterator obeys start and end slicing + */ + @Test + def testIteratorWithLimits() { + val message = messageSet.toList(1) + val start = messageSet.searchFor(1, 0).position + val size = message.message.size + val slice = messageSet.read(start, size) + assertEquals(List(message), slice.toList) + } + + /** + * Test the truncateTo method lops off messages and appropriately updates the size + */ + @Test + def testTruncate() { + val message = messageSet.toList(0) + val end = messageSet.searchFor(1, 0).position + messageSet.truncateTo(end) + assertEquals(List(message), messageSet.toList) + assertEquals(MessageSet.entrySize(message.message), messageSet.sizeInBytes) } } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala new file mode 100644 index 0000000000000..1de3ef0435d76 --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +import java.io.File +import scala.collection._ +import org.junit._ +import kafka.common.TopicAndPartition +import kafka.utils._ +import kafka.message._ +import org.scalatest.junit.JUnitSuite +import junit.framework.Assert._ + +/** + * This is an integration test that tests the fully integrated log cleaner + */ +class LogCleanerIntegrationTest extends JUnitSuite { + + val time = new MockTime() + val segmentSize = 100 + val deleteDelay = 1000 + val logName = "log" + val logDir = TestUtils.tempDir() + var counter = 0 + val topics = Array(TopicAndPartition("log", 0), TopicAndPartition("log", 1), TopicAndPartition("log", 2)) + + @Test + def cleanerTest() { + val cleaner = makeCleaner(parts = 3) + val log = cleaner.logs.get(topics(0)) + + val appends = writeDups(numKeys = 100, numDups = 3, log) + val startSize = log.size + cleaner.startup() + + val lastCleaned = log.activeSegment.baseOffset + // wait until we clean up to base_offset of active segment - minDirtyMessages + cleaner.awaitCleaned("log", 0, lastCleaned) + + val read = readFromLog(log) + assertEquals("Contents of the map shouldn't change.", appends.toMap, read.toMap) + assertTrue(startSize > log.size) + + // write some more stuff and validate again + val appends2 = appends ++ writeDups(numKeys = 100, numDups = 3, log) + val lastCleaned2 = log.activeSegment.baseOffset + cleaner.awaitCleaned("log", 0, lastCleaned2) + val read2 = readFromLog(log) + assertEquals("Contents of the map shouldn't change.", appends2.toMap, read2.toMap) + + cleaner.shutdown() + } + + def readFromLog(log: Log): Iterable[(Int, Int)] = { + for(segment <- log.logSegments; message <- segment.log) yield { + val key = Utils.readString(message.message.key).toInt + val value = Utils.readString(message.message.payload).toInt + key -> value + } + } + + def writeDups(numKeys: Int, numDups: Int, log: Log): Seq[(Int, Int)] = { + for(dup <- 0 until numDups; key <- 0 until numKeys) yield { + val count = counter + log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true) + counter += 1 + (key, count) + } + } + + @After + def teardown() { + Utils.rm(logDir) + } + + /* create a cleaner instance and logs with the given parameters */ + def makeCleaner(parts: Int, + minDirtyMessages: Int = 0, + numThreads: Int = 1, + defaultPolicy: String = "dedupe", + policyOverrides: Map[String, String] = Map()): LogCleaner = { + + // create partitions and add them to the pool + val logs = new Pool[TopicAndPartition, Log]() + for(i <- 0 until parts) { + val dir = new File(logDir, "log-" + i) + dir.mkdirs() + val log = new Log(dir = dir, + LogConfig(segmentSize = segmentSize, maxIndexSize = 100*1024, fileDeleteDelayMs = deleteDelay, dedupe = true), + recoveryPoint = 0L, + scheduler = time.scheduler, + time = time) + logs.put(TopicAndPartition("log", i), log) + } + + new LogCleaner(CleanerConfig(numThreads = numThreads), + logDirs = Array(logDir), + logs = logs, + time = time) + } + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index ce893bfe84293..b4bee33191ebc 100644 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -24,34 +24,37 @@ import org.scalatest.junit.JUnit3Suite import kafka.server.KafkaConfig import kafka.common._ import kafka.utils._ +import kafka.server.OffsetCheckpoint class LogManagerTest extends JUnit3Suite { val time: MockTime = new MockTime() val maxRollInterval = 100 - val maxLogAgeHours = 10 + val maxLogAgeMs = 10*60*60*1000 + val logConfig = LogConfig(segmentSize = 1024, maxIndexSize = 4096, retentionMs = maxLogAgeMs) var logDir: File = null var logManager: LogManager = null - var config: KafkaConfig = null val name = "kafka" val veryLargeLogFlushInterval = 10000000L - val scheduler = new KafkaScheduler(2) + val cleanerConfig = CleanerConfig(enableCleaner = false) override def setUp() { super.setUp() - config = new KafkaConfig(TestUtils.createBrokerConfig(0, -1)) { - override val logSegmentBytes = 1024 - override val logFlushIntervalMessages = 10000 - override val logRetentionHours = maxLogAgeHours - } - scheduler.startup - logManager = new LogManager(config, scheduler, time) + logDir = TestUtils.tempDir() + logManager = new LogManager(logDirs = Array(logDir), + topicConfigs = Map(), + defaultConfig = logConfig, + cleanerConfig = cleanerConfig, + flushCheckMs = 1000L, + flushCheckpointMs = 100000L, + retentionCheckMs = 1000L, + scheduler = time.scheduler, + time = time) logManager.startup logDir = logManager.logDirs(0) } override def tearDown() { - scheduler.shutdown() if(logManager != null) logManager.shutdown() Utils.rm(logDir) @@ -59,41 +62,48 @@ class LogManagerTest extends JUnit3Suite { super.tearDown() } + /** + * Test that getOrCreateLog on a non-existent log creates a new log and that we can append to the new log. + */ @Test def testCreateLog() { - val log = logManager.getOrCreateLog(name, 0) - val logFile = new File(config.logDirs(0), name + "-0") + val log = logManager.createLog(TopicAndPartition(name, 0), logConfig) + val logFile = new File(logDir, name + "-0") assertTrue(logFile.exists) log.append(TestUtils.singleMessageSet("test".getBytes())) } + /** + * Test that get on a non-existent returns None and no log is created. + */ @Test - def testGetLog() { - val log = logManager.getLog(name, 0) - val logFile = new File(config.logDirs(0), name + "-0") + def testGetNonExistentLog() { + val log = logManager.getLog(TopicAndPartition(name, 0)) + assertEquals("No log should be found.", None, log) + val logFile = new File(logDir, name + "-0") assertTrue(!logFile.exists) } + /** + * Test time-based log cleanup. First append messages, then set the time into the future and run cleanup. + */ @Test def testCleanupExpiredSegments() { - val log = logManager.getOrCreateLog(name, 0) + val log = logManager.createLog(TopicAndPartition(name, 0), logConfig) var offset = 0L - for(i <- 0 until 1000) { + for(i <- 0 until 200) { var set = TestUtils.singleMessageSet("test".getBytes()) - val (start, end) = log.append(set) - offset = end + val info = log.append(set) + offset = info.lastOffset } - log.flush - assertTrue("There should be more than one segment now.", log.numberOfSegments > 1) - - // update the last modified time of all log segments - val logSegments = log.segments.view - logSegments.foreach(_.messageSet.file.setLastModified(time.currentMs)) - - time.currentMs += maxLogAgeHours*60*60*1000 + 1 - logManager.cleanupLogs() - assertEquals("Now there should only be only one segment.", 1, log.numberOfSegments) + + log.logSegments.foreach(_.log.file.setLastModified(time.milliseconds)) + + time.sleep(maxLogAgeMs + 1) + assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments) + time.sleep(log.config.fileDeleteDelayMs + 1) + assertEquals("Files should have been deleted", log.numberOfSegments * 2, log.dir.list.length) assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).sizeInBytes) try { @@ -106,42 +116,37 @@ class LogManagerTest extends JUnit3Suite { log.append(TestUtils.singleMessageSet("test".getBytes())) } + /** + * Test size-based cleanup. Append messages, then run cleanup and check that segments are deleted. + */ @Test def testCleanupSegmentsToMaintainSize() { val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes - val retentionHours = 1 - val retentionMs = 1000 * 60 * 60 * retentionHours - val props = TestUtils.createBrokerConfig(0, -1) logManager.shutdown() - config = new KafkaConfig(props) { - override val logSegmentBytes = (10 * (setSize - 1)) // each segment will be 10 messages - override val logRetentionBytes = (5 * 10 * setSize + 10).asInstanceOf[Long] - override val logRetentionHours = retentionHours - override val logFlushIntervalMessages = 100 - override val logRollHours = maxRollInterval - } - logManager = new LogManager(config, scheduler, time) + + val config = logConfig.copy(segmentSize = 10 * (setSize - 1), retentionSize = 5L * 10L * setSize + 10L) + logManager = new LogManager(Array(logDir), Map(), config, cleanerConfig, 1000L, 100000L, 1000L, time.scheduler, time) logManager.startup // create a log - val log = logManager.getOrCreateLog(name, 0) + val log = logManager.createLog(TopicAndPartition(name, 0), config) var offset = 0L // add a bunch of messages that should be larger than the retentionSize - for(i <- 0 until 1000) { + val numMessages = 200 + for(i <- 0 until numMessages) { val set = TestUtils.singleMessageSet("test".getBytes()) - val (start, end) = log.append(set) - offset = start + val info = log.append(set) + offset = info.firstOffset } - // flush to make sure it's written to disk - log.flush - // should be exactly 100 full segments + 1 new empty one - assertEquals("There should be example 100 segments.", 100, log.numberOfSegments) + assertEquals("Check we have the expected number of segments.", numMessages * setSize / config.segmentSize, log.numberOfSegments) // this cleanup shouldn't find any expired segments but should delete some to reduce size - logManager.cleanupLogs() + time.sleep(logManager.InitialTaskDelayMs) assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments) + time.sleep(log.config.fileDeleteDelayMs + 1) + assertEquals("Files should have been deleted", log.numberOfSegments * 2, log.dir.list.length) assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).sizeInBytes) try { log.read(0, 1024) @@ -153,55 +158,75 @@ class LogManagerTest extends JUnit3Suite { log.append(TestUtils.singleMessageSet("test".getBytes())) } + /** + * Test that flush is invoked by the background scheduler thread. + */ @Test def testTimeBasedFlush() { - val props = TestUtils.createBrokerConfig(0, -1) logManager.shutdown() - config = new KafkaConfig(props) { - override val logSegmentBytes = 1024 *1024 *1024 - override val logFlushSchedulerIntervalMs = 50 - override val logFlushIntervalMessages = Int.MaxValue - override val logRollHours = maxRollInterval - override val logFlushIntervalMsPerTopicMap = Map("timebasedflush" -> 100) - } - logManager = new LogManager(config, scheduler, time) + val config = logConfig.copy(flushMs = 1000) + logManager = new LogManager(Array(logDir), Map(), config, cleanerConfig, 1000L, 10000L, 1000L, time.scheduler, time) logManager.startup - val log = logManager.getOrCreateLog(name, 0) + val log = logManager.createLog(TopicAndPartition(name, 0), config) + val lastFlush = log.lastFlushTime for(i <- 0 until 200) { var set = TestUtils.singleMessageSet("test".getBytes()) log.append(set) } - val ellapsed = System.currentTimeMillis - log.getLastFlushedTime - assertTrue("The last flush time has to be within defaultflushInterval of current time (was %d)".format(ellapsed), - ellapsed < 2*config.logFlushSchedulerIntervalMs) + time.sleep(logManager.InitialTaskDelayMs) + assertTrue("Time based flush should have been triggered triggered", lastFlush != log.lastFlushTime) } + /** + * Test that new logs that are created are assigned to the least loaded log directory + */ @Test def testLeastLoadedAssignment() { // create a log manager with multiple data directories - val props = TestUtils.createBrokerConfig(0, -1) - val dirs = Seq(TestUtils.tempDir().getAbsolutePath, - TestUtils.tempDir().getAbsolutePath, - TestUtils.tempDir().getAbsolutePath) - props.put("log.dirs", dirs.mkString(",")) + val dirs = Array(TestUtils.tempDir(), + TestUtils.tempDir(), + TestUtils.tempDir()) logManager.shutdown() - logManager = new LogManager(new KafkaConfig(props), scheduler, time) + logManager = new LogManager(dirs, Map(), logConfig, cleanerConfig, 1000L, 10000L, 1000L, time.scheduler, time) // verify that logs are always assigned to the least loaded partition for(partition <- 0 until 20) { - logManager.getOrCreateLog("test", partition) + logManager.createLog(TopicAndPartition("test", partition), logConfig) assertEquals("We should have created the right number of logs", partition + 1, logManager.allLogs.size) val counts = logManager.allLogs.groupBy(_.dir.getParent).values.map(_.size) assertTrue("Load should balance evenly", counts.max <= counts.min + 1) } } + /** + * Test that it is not possible to open two log managers using the same data directory + */ def testTwoLogManagersUsingSameDirFails() { try { - new LogManager(logManager.config, scheduler, time) + new LogManager(Array(logDir), Map(), logConfig, cleanerConfig, 1000L, 10000L, 1000L, time.scheduler, time) fail("Should not be able to create a second log manager instance with the same data directory") } catch { case e: KafkaException => // this is good } } + + /** + * Test that recovery points are correctly written out to disk + */ + def testCheckpointRecoveryPoints() { + val topicA = TopicAndPartition("test-a", 1) + val topicB = TopicAndPartition("test-b", 1) + val logA = this.logManager.createLog(topicA, logConfig) + val logB = this.logManager.createLog(topicB, logConfig) + for(i <- 0 until 50) + logA.append(TestUtils.singleMessageSet("test".getBytes())) + for(i <- 0 until 100) + logB.append(TestUtils.singleMessageSet("test".getBytes())) + logA.flush() + logB.flush() + logManager.checkpointRecoveryPointOffsets() + val checkpoints = new OffsetCheckpoint(new File(logDir, logManager.RecoveryPointCheckpointFile)).read() + assertEquals("Recovery point should equal checkpoint", checkpoints(topicA), logA.recoveryPoint) + assertEquals("Recovery point should equal checkpoint", checkpoints(topicB), logB.recoveryPoint) + } } diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 2879907a3b9fa..6b7603728ae52 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -18,6 +18,9 @@ import junit.framework.Assert._ import java.util.concurrent.atomic._ +import java.io.File +import java.io.RandomAccessFile +import java.util.Random import org.junit.{Test, After} import org.scalatest.junit.JUnit3Suite import kafka.utils.TestUtils @@ -29,6 +32,7 @@ class LogSegmentTest extends JUnit3Suite { val segments = mutable.ArrayBuffer[LogSegment]() + /* create a segment with the given base offset */ def createSegment(offset: Long): LogSegment = { val msFile = TestUtils.tempFile() val ms = new FileMessageSet(msFile) @@ -40,6 +44,7 @@ class LogSegmentTest extends JUnit3Suite { seg } + /* create a ByteBufferMessageSet for the given messages starting from the given offset */ def messages(offset: Long, messages: String*): ByteBufferMessageSet = { new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, offsetCounter = new AtomicLong(offset), @@ -50,17 +55,24 @@ class LogSegmentTest extends JUnit3Suite { def teardown() { for(seg <- segments) { seg.index.delete() - seg.messageSet.delete() + seg.log.delete() } } + /** + * A read on an empty log segment should return null + */ @Test def testReadOnEmptySegment() { val seg = createSegment(40) val read = seg.read(startOffset = 40, maxSize = 300, maxOffset = None) - assertEquals(0, read.size) + assertNull("Read beyond the last offset in the segment should be null", read) } + /** + * Reading from before the first offset in the segment should return messages + * beginning with the first message in the segment + */ @Test def testReadBeforeFirstOffset() { val seg = createSegment(40) @@ -70,24 +82,40 @@ class LogSegmentTest extends JUnit3Suite { assertEquals(ms.toList, read.toList) } + /** + * If we set the startOffset and maxOffset for the read to be the same value + * we should get only the first message in the log + */ @Test - def testReadSingleMessage() { - val seg = createSegment(40) - val ms = messages(50, "hello", "there") - seg.append(50, ms) - val read = seg.read(startOffset = 41, maxSize = 200, maxOffset = Some(50)) - assertEquals(new Message("hello".getBytes), read.head.message) + def testMaxOffset() { + val baseOffset = 50 + val seg = createSegment(baseOffset) + val ms = messages(baseOffset, "hello", "there", "beautiful") + seg.append(baseOffset, ms) + def validate(offset: Long) = + assertEquals(ms.filter(_.offset == offset).toList, + seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).toList) + validate(50) + validate(51) + validate(52) } + /** + * If we read from an offset beyond the last offset in the segment we should get null + */ @Test def testReadAfterLast() { val seg = createSegment(40) val ms = messages(50, "hello", "there") seg.append(50, ms) val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None) - assertEquals(0, read.size) + assertNull("Read beyond the last offset in the segment should give null", null) } + /** + * If we read from an offset which doesn't exist we should get a message set beginning + * with the least offset greater than the given startOffset. + */ @Test def testReadFromGap() { val seg = createSegment(40) @@ -99,6 +127,10 @@ class LogSegmentTest extends JUnit3Suite { assertEquals(ms2.toList, read.toList) } + /** + * In a loop append two messages then truncate off the second of those messages and check that we can read + * the first but not the second message. + */ @Test def testTruncate() { val seg = createSegment(40) @@ -109,26 +141,33 @@ class LogSegmentTest extends JUnit3Suite { val ms2 = messages(offset+1, "hello") seg.append(offset+1, ms2) // check that we can read back both messages - val read = seg.read(offset, 10000, None) + val read = seg.read(offset, None, 10000) assertEquals(List(ms1.head, ms2.head), read.toList) // now truncate off the last message seg.truncateTo(offset + 1) - val read2 = seg.read(offset, 10000, None) + val read2 = seg.read(offset, None, 10000) assertEquals(1, read2.size) assertEquals(ms1.head, read2.head) offset += 1 } } + /** + * Test truncating the whole segment, and check that we can reappend with the original offset. + */ @Test def testTruncateFull() { // test the case where we fully truncate the log val seg = createSegment(40) seg.append(40, messages(40, "hello", "there")) seg.truncateTo(0) + assertNull("Segment should be empty.", seg.read(0, None, 1024)) seg.append(40, messages(40, "hello", "there")) } + /** + * Test that offsets are assigned sequentially and that the nextOffset variable is incremented + */ @Test def testNextOffsetCalculation() { val seg = createSegment(40) @@ -137,4 +176,55 @@ class LogSegmentTest extends JUnit3Suite { assertEquals(53, seg.nextOffset()) } + /** + * Test that we can change the file suffixes for the log and index files + */ + @Test + def testChangeFileSuffixes() { + val seg = createSegment(40) + val logFile = seg.log.file + val indexFile = seg.index.file + seg.changeFileSuffixes("", ".deleted") + assertEquals(logFile.getAbsolutePath + ".deleted", seg.log.file.getAbsolutePath) + assertEquals(indexFile.getAbsolutePath + ".deleted", seg.index.file.getAbsolutePath) + assertTrue(seg.log.file.exists) + assertTrue(seg.index.file.exists) + } + + /** + * Create a segment with some data and an index. Then corrupt the index, + * and recover the segment, the entries should all be readable. + */ + @Test + def testRecoveryFixesCorruptIndex() { + val seg = createSegment(0) + for(i <- 0 until 100) + seg.append(i, messages(i, i.toString)) + val indexFile = seg.index.file + TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt) + seg.recover(64*1024) + for(i <- 0 until 100) + assertEquals(i, seg.read(i, Some(i+1), 1024).head.offset) + } + + /** + * Randomly corrupt a log a number of times and attempt recovery. + */ + @Test + def testRecoveryWithCorruptMessage() { + val messagesAppended = 20 + for(iteration <- 0 until 10) { + val seg = createSegment(0) + for(i <- 0 until messagesAppended) + seg.append(i, messages(i, i.toString)) + val offsetToBeginCorruption = TestUtils.random.nextInt(messagesAppended) + // start corrupting somewhere in the middle of the chosen record all the way to the end + val position = seg.log.searchFor(offsetToBeginCorruption, 0).position + TestUtils.random.nextInt(15) + TestUtils.writeNonsenseToFile(seg.log.file, position, seg.log.file.length.toInt - position) + seg.recover(64*1024) + assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.map(_.offset).toList) + seg.delete() + } + } + } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index df906953fb241..1da1393983d4b 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -5,7 +5,7 @@ * 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 @@ -30,10 +30,11 @@ import scala.Some import kafka.server.KafkaConfig class LogTest extends JUnitSuite { - + var logDir: File = null - val time = new MockTime + val time = new MockTime(0) var config: KafkaConfig = null + val logConfig = LogConfig() @Before def setUp() { @@ -46,7 +47,7 @@ class LogTest extends JUnitSuite { def tearDown() { Utils.rm(logDir) } - + def createEmptyLogs(dir: File, offsets: Int*) { for(offset <- offsets) { Log.logFilename(dir, offset).createNewFile() @@ -54,49 +55,52 @@ class LogTest extends JUnitSuite { } } - /** Test that the size and time based log segment rollout works. */ + /** + * Tests for time based log roll. This test appends messages then changes the time + * using the mock clock to force the log to roll and checks the number of segments. + */ @Test def testTimeBasedLogRoll() { val set = TestUtils.singleMessageSet("test".getBytes()) - val rollMs = 1 * 60 * 60L - val time: MockTime = new MockTime() // create a log - val log = new Log(logDir, 1000, config.messageMaxBytes, 1000, rollMs, needsRecovery = false, time = time) - time.currentMs += rollMs + 1 - - // segment age is less than its limit - log.append(set) - assertEquals("There should be exactly one segment.", 1, log.numberOfSegments) - + val log = new Log(logDir, + logConfig.copy(segmentMs = 1 * 60 * 60L), + recoveryPoint = 0L, + scheduler = time.scheduler, + time = time) + assertEquals("Log begins with a single empty segment.", 1, log.numberOfSegments) + time.sleep(log.config.segmentMs + 1) log.append(set) - assertEquals("There should be exactly one segment.", 1, log.numberOfSegments) + assertEquals("Log doesn't roll if doing so creates an empty segment.", 1, log.numberOfSegments) - // segment expires in age - time.currentMs += rollMs + 1 log.append(set) - assertEquals("There should be exactly 2 segments.", 2, log.numberOfSegments) + assertEquals("Log rolls on this append since time has expired.", 2, log.numberOfSegments) - time.currentMs += rollMs + 1 - val blank = Array[Message]() - log.append(new ByteBufferMessageSet(new Message("blah".getBytes))) - assertEquals("There should be exactly 3 segments.", 3, log.numberOfSegments) + for(numSegments <- 3 until 5) { + time.sleep(log.config.segmentMs + 1) + log.append(set) + assertEquals("Changing time beyond rollMs and appending should create a new segment.", numSegments, log.numberOfSegments) + } - time.currentMs += rollMs + 1 - // the last segment expired in age, but was blank. So new segment should not be generated + val numSegments = log.numberOfSegments + time.sleep(log.config.segmentMs + 1) log.append(new ByteBufferMessageSet()) - assertEquals("There should be exactly 3 segments.", 3, log.numberOfSegments) + assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments) } + /** + * Test that appending more than the maximum segment size rolls the log + */ @Test def testSizeBasedLogRoll() { - val set = TestUtils.singleMessageSet("test".getBytes()) + val set = TestUtils.singleMessageSet("test".getBytes) val setSize = set.sizeInBytes val msgPerSeg = 10 - val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages + val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages // create a log - val log = new Log(logDir, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time) + val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), recoveryPoint = 0L, time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) // segments expire in size @@ -106,32 +110,84 @@ class LogTest extends JUnitSuite { assertEquals("There should be exactly 2 segments.", 2, log.numberOfSegments) } + /** + * Test that we can open and append to an empty log + */ @Test def testLoadEmptyLog() { createEmptyLogs(logDir, 0) - new Log(logDir, 1024, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) + val log = new Log(logDir, logConfig, recoveryPoint = 0L, time.scheduler, time = time) + log.append(TestUtils.singleMessageSet("test".getBytes)) } + /** + * This test case appends a bunch of messages and checks that we can read them all back using sequential offsets. + */ @Test - def testAppendAndRead() { - val log = new Log(logDir, 1024, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) - val message = new Message(Integer.toString(42).getBytes()) - for(i <- 0 until 10) - log.append(new ByteBufferMessageSet(NoCompressionCodec, message)) - log.flush() - val messages = log.read(0, 1024) - var current = 0 - for(curr <- messages) { - assertEquals("Read message should equal written", message, curr.message) - current += 1 + def testAppendAndReadWithSequentialOffsets() { + val log = new Log(logDir, logConfig.copy(segmentSize = 71), recoveryPoint = 0L, time.scheduler, time = time) + val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray + + for(i <- 0 until messages.length) + log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = messages(i))) + for(i <- 0 until messages.length) { + val read = log.read(i, 100, Some(i+1)).head + assertEquals("Offset read should match order appended.", i, read.offset) + assertEquals("Message should match appended.", messages(i), read.message) } - assertEquals(10, current) + assertEquals("Reading beyond the last message returns nothing.", 0, log.read(messages.length, 100, None).size) } - + + /** + * This test appends a bunch of messages with non-sequential offsets and checks that we can read the correct message + * from any offset less than the logEndOffset including offsets not appended. + */ + @Test + def testAppendAndReadWithNonSequentialOffsets() { + val log = new Log(logDir, logConfig.copy(segmentSize = 71), recoveryPoint = 0L, time.scheduler, time = time) + val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray + val messages = messageIds.map(id => new Message(id.toString.getBytes)) + + // now test the case that we give the offsets and use non-sequential offsets + for(i <- 0 until messages.length) + log.append(new ByteBufferMessageSet(NoCompressionCodec, new AtomicLong(messageIds(i)), messages = messages(i)), assignOffsets = false) + for(i <- 50 until messageIds.max) { + val idx = messageIds.indexWhere(_ >= i) + val read = log.read(i, 100, None).head + assertEquals("Offset read should match message id.", messageIds(idx), read.offset) + assertEquals("Message should match appended.", messages(idx), read.message) + } + } + + /** + * This test covers an odd case where we have a gap in the offsets that falls at the end of a log segment. + * Specifically we create a log where the last message in the first segment has offset 0. If we + * then read offset 1, we should expect this read to come from the second segment, even though the + * first segment has the greatest lower bound on the offset. + */ + @Test + def testReadAtLogGap() { + val log = new Log(logDir, logConfig.copy(segmentSize = 300), recoveryPoint = 0L, time.scheduler, time = time) + + // keep appending until we have two segments with only a single message in the second segment + while(log.numberOfSegments == 1) + log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes))) + + // now manually truncate off all but one message from the first segment to create a gap in the messages + log.logSegments.head.truncateTo(1) + + assertEquals("A read should now return the last message in the log", log.logEndOffset-1, log.read(1, 200, None).head.offset) + } + + /** + * Test reading at the boundary of the log, specifically + * - reading from the logEndOffset should give an empty message set + * - reading beyond the log end offset should throw an OffsetOutOfRangeException + */ @Test def testReadOutOfRange() { createEmptyLogs(logDir, 1024) - val log = new Log(logDir, 1024, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) + val log = new Log(logDir, logConfig.copy(segmentSize = 1024), recoveryPoint = 0L, time.scheduler, time = time) assertEquals("Reading just beyond end of log should produce 0 byte read.", 0, log.read(1024, 1000).sizeInBytes) try { log.read(0, 1024) @@ -143,18 +199,21 @@ class LogTest extends JUnitSuite { log.read(1025, 1000) fail("Expected exception on invalid read.") } catch { - case e: OffsetOutOfRangeException => "This is good." + case e: OffsetOutOfRangeException => // This is good. } } - /** Test that writing and reading beyond the log size boundary works */ + /** + * Test that covers reads and writes on a multisegment log. This test appends a bunch of messages + * and then reads them all back and checks that the message read and offset matches what was appended. + */ @Test def testLogRolls() { /* create a multipart log with 100 messages */ - val log = new Log(logDir, 100, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) + val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) val numMessages = 100 val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes)) - val offsets = messageSets.map(log.append(_)._1) + messageSets.foreach(log.append(_)) log.flush /* do successive reads to ensure all our messages are there */ @@ -167,90 +226,77 @@ class LogTest extends JUnitSuite { } val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)) assertEquals("Should be no more messages", 0, lastRead.size) + + // check that rolling the log forced a flushed the log--the flush is asyn so retry in case of failure + TestUtils.retry(1000L){ + assertTrue("Log role should have forced flush", log.recoveryPoint >= log.activeSegment.baseOffset) + } } - - /** Test the case where we have compressed batches of messages */ + + /** + * Test reads at offsets that fall within compressed message set boundaries. + */ @Test def testCompressedMessages() { /* this log should roll after every messageset */ - val log = new Log(logDir, 10, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) - + val log = new Log(logDir, logConfig.copy(segmentSize = 10), recoveryPoint = 0L, time.scheduler, time = time) + /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */ log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes))) log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("alpha".getBytes), new Message("beta".getBytes))) - + def read(offset: Int) = ByteBufferMessageSet.decompress(log.read(offset, 4096).head.message) - + /* we should always get the first message in the compressed set when reading any offset in the set */ assertEquals("Read at offset 0 should produce 0", 0, read(0).head.offset) assertEquals("Read at offset 1 should produce 0", 0, read(1).head.offset) assertEquals("Read at offset 2 should produce 2", 2, read(2).head.offset) assertEquals("Read at offset 3 should produce 2", 2, read(3).head.offset) } - - @Test - def testFindSegment() { - assertEquals("Search in empty segments list should find nothing", None, Log.findRange(makeRanges(), 45)) - assertEquals("Search in segment list just outside the range of the last segment should find last segment", - 9, Log.findRange(makeRanges(5, 9, 12), 12).get.start) - assertEquals("Search in segment list far outside the range of the last segment should find last segment", - 9, Log.findRange(makeRanges(5, 9, 12), 100).get.start) - assertEquals("Search in segment list far outside the range of the last segment should find last segment", - None, Log.findRange(makeRanges(5, 9, 12), -1)) - assertContains(makeRanges(5, 9, 12), 11) - assertContains(makeRanges(5), 4) - assertContains(makeRanges(5,8), 5) - assertContains(makeRanges(5,8), 6) - } - - @Test - def testEdgeLogRollsStartingAtZero() { - // first test a log segment starting at 0 - val log = new Log(logDir, 100, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) - val curOffset = log.logEndOffset - assertEquals(curOffset, 0) - - // time goes by; the log file is deleted - log.markDeletedWhile(_ => true) - - // we now have a new log; the starting offset of the new log should remain 0 - assertEquals(curOffset, log.logEndOffset) - log.delete() - } - + + /** + * Test garbage collecting old segments + */ @Test - def testEdgeLogRollsStartingAtNonZero() { - // second test an empty log segment starting at non-zero - val log = new Log(logDir, 100, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) - val numMessages = 1 - for(i <- 0 until numMessages) - log.append(TestUtils.singleMessageSet(i.toString.getBytes)) - val curOffset = log.logEndOffset - - // time goes by; the log file is deleted - log.markDeletedWhile(_ => true) - - // we now have a new log - assertEquals(curOffset, log.logEndOffset) - - // time goes by; the log file (which is empty) is deleted again - val deletedSegments = log.markDeletedWhile(_ => true) - - // we shouldn't delete the last empty log segment. - assertTrue("We shouldn't delete the last empty log segment", deletedSegments.size == 0) - - // we now have a new log - assertEquals(curOffset, log.logEndOffset) + def testThatGarbageCollectingSegmentsDoesntChangeOffset() { + for(messagesToAppend <- List(0, 1, 25)) { + logDir.mkdirs() + // first test a log segment starting at 0 + val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) + for(i <- 0 until messagesToAppend) + log.append(TestUtils.singleMessageSet(i.toString.getBytes)) + + var currOffset = log.logEndOffset + assertEquals(currOffset, messagesToAppend) + + // time goes by; the log file is deleted + log.deleteOldSegments(_ => true) + + assertEquals("Deleting segments shouldn't have changed the logEndOffset", currOffset, log.logEndOffset) + assertEquals("We should still have one segment left", 1, log.numberOfSegments) + assertEquals("Further collection shouldn't delete anything", 0, log.deleteOldSegments(_ => true)) + assertEquals("Still no change in the logEndOffset", currOffset, log.logEndOffset) + assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append", + currOffset, + log.append(TestUtils.singleMessageSet("hello".toString.getBytes)).firstOffset) + + // cleanup the log + log.delete() + } } + /** + * We have a max size limit on message appends, check that it is properly enforced by appending a message larger than the + * setting and checking that an exception is thrown. + */ @Test def testMessageSizeCheck() { - val first = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes()), new Message("bethe".getBytes())) - val second = new ByteBufferMessageSet(NoCompressionCodec, new Message("change".getBytes())) + val first = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes)) + val second = new ByteBufferMessageSet(NoCompressionCodec, new Message("change".getBytes)) // append messages to log val maxMessageSize = second.sizeInBytes - 1 - val log = new Log(logDir, 100, maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) + val log = new Log(logDir, logConfig.copy(maxMessageSize = maxMessageSize), recoveryPoint = 0L, time.scheduler, time = time) // should be able to append the small message log.append(first) @@ -259,56 +305,88 @@ class LogTest extends JUnitSuite { log.append(second) fail("Second message set should throw MessageSizeTooLargeException.") } catch { - case e:MessageSizeTooLargeException => // this is good + case e: MessageSizeTooLargeException => // this is good } } - + + /** + * Append a bunch of messages to a log and then re-open it both with and without recovery and check that the log re-initializes correctly. + */ @Test def testLogRecoversToCorrectOffset() { val numMessages = 100 val messageSize = 100 val segmentSize = 7 * messageSize val indexInterval = 3 * messageSize - var log = new Log(logDir, segmentSize, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, indexIntervalBytes = indexInterval, maxIndexSize = 4096) + val config = logConfig.copy(segmentSize = segmentSize, indexInterval = indexInterval, maxIndexSize = 4096) + var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) for(i <- 0 until numMessages) log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(messageSize))) assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset) - val lastIndexOffset = log.segments.view.last.index.lastOffset - val numIndexEntries = log.segments.view.last.index.entries + val lastIndexOffset = log.activeSegment.index.lastOffset + val numIndexEntries = log.activeSegment.index.entries + val lastOffset = log.logEndOffset log.close() - - // test non-recovery case - log = new Log(logDir, segmentSize, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, indexIntervalBytes = indexInterval, maxIndexSize = 4096) + + log = new Log(logDir, config, recoveryPoint = lastOffset, time.scheduler, time) assertEquals("Should have %d messages when log is reopened w/o recovery".format(numMessages), numMessages, log.logEndOffset) - assertEquals("Should have same last index offset as before.", lastIndexOffset, log.segments.view.last.index.lastOffset) - assertEquals("Should have same number of index entries as before.", numIndexEntries, log.segments.view.last.index.entries) + assertEquals("Should have same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset) + assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries) log.close() - - // test - log = new Log(logDir, segmentSize, config.messageMaxBytes, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, indexIntervalBytes = indexInterval, maxIndexSize = 4096) + + // test recovery case + log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) assertEquals("Should have %d messages when log is reopened with recovery".format(numMessages), numMessages, log.logEndOffset) - assertEquals("Should have same last index offset as before.", lastIndexOffset, log.segments.view.last.index.lastOffset) - assertEquals("Should have same number of index entries as before.", numIndexEntries, log.segments.view.last.index.entries) + assertEquals("Should have same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset) + assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries) + log.close() + } + + /** + * Test that if we manually delete an index segment it is rebuilt when the log is re-opened + */ + @Test + def testIndexRebuild() { + // publish the messages and close the log + val numMessages = 200 + val config = logConfig.copy(segmentSize = 200, indexInterval = 1) + var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) + for(i <- 0 until numMessages) + log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10))) + val indexFiles = log.logSegments.map(_.index.file) + log.close() + + // delete all the index files + indexFiles.foreach(_.delete()) + + // reopen the log + log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) + assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset) + for(i <- 0 until numMessages) + assertEquals(i, log.read(i, 100, None).head.offset) log.close() } + /** + * Test the Log truncate operations + */ @Test def testTruncateTo() { val set = TestUtils.singleMessageSet("test".getBytes()) val setSize = set.sizeInBytes val msgPerSeg = 10 - val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages + val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages // create a log - val log = new Log(logDir, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time) + val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), recoveryPoint = 0L, scheduler = time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) for (i<- 1 to msgPerSeg) log.append(set) - + assertEquals("There should be exactly 1 segments.", 1, log.numberOfSegments) assertEquals("Log end offset should be equal to number of messages", msgPerSeg, log.logEndOffset) - + val lastOffset = log.logEndOffset val size = log.size log.truncateTo(log.logEndOffset) // keep the entire log @@ -326,10 +404,10 @@ class LogTest extends JUnitSuite { for (i<- 1 to msgPerSeg) log.append(set) - + assertEquals("Should be back to original offset", log.logEndOffset, lastOffset) assertEquals("Should be back to original size", log.size, size) - log.truncateAndStartWithNewOffset(log.logEndOffset - (msgPerSeg - 1)) + log.truncateFullyAndStartAt(log.logEndOffset - (msgPerSeg - 1)) assertEquals("Should change offset", log.logEndOffset, lastOffset - (msgPerSeg - 1)) assertEquals("Should change log size", log.size, 0) @@ -343,13 +421,17 @@ class LogTest extends JUnitSuite { assertEquals("Should change log size", log.size, 0) } + /** + * Verify that when we truncate a log the index of the last segment is resized to the max index size to allow more appends + */ @Test def testIndexResizingAtTruncation() { val set = TestUtils.singleMessageSet("test".getBytes()) val setSize = set.sizeInBytes val msgPerSeg = 10 - val logFileSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages - val log = new Log(logDir, logFileSize, config.messageMaxBytes, 1000, 10000, needsRecovery = false, time = time) + val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages + val config = logConfig.copy(segmentSize = segmentSize) + val log = new Log(logDir, config, recoveryPoint = 0L, scheduler = time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) for (i<- 1 to msgPerSeg) log.append(set) @@ -357,41 +439,15 @@ class LogTest extends JUnitSuite { for (i<- 1 to msgPerSeg) log.append(set) assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments) - assertEquals("The index of the first segment should be trim to empty", 0, log.segments.view(0).index.maxEntries) + assertEquals("The index of the first segment should be trimmed to empty", 0, log.logSegments.toList(0).index.maxEntries) log.truncateTo(0) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) - assertEquals("The index of segment 1 should be resized to maxIndexSize", log.maxIndexSize/8, log.segments.view(0).index.maxEntries) + assertEquals("The index of segment 1 should be resized to maxIndexSize", log.config.maxIndexSize/8, log.logSegments.toList(0).index.maxEntries) for (i<- 1 to msgPerSeg) log.append(set) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) } - - @Test - def testAppendWithoutOffsetAssignment() { - for(codec <- List(NoCompressionCodec, DefaultCompressionCodec)) { - logDir.mkdir() - var log = new Log(logDir, - maxLogFileSize = 64*1024, - maxMessageSize = config.messageMaxBytes, - maxIndexSize = 1000, - indexIntervalBytes = 10000, - needsRecovery = true) - val messages = List("one", "two", "three", "four", "five", "six") - val ms = new ByteBufferMessageSet(compressionCodec = codec, - offsetCounter = new AtomicLong(0), - messages = messages.map(s => new Message(s.getBytes)):_*) - val firstOffset = ms.toList.head.offset - val lastOffset = ms.toList.last.offset - val (first, last) = log.append(ms, assignOffsets = false) - assertEquals(last + 1, log.logEndOffset) - assertEquals(firstOffset, first) - assertEquals(lastOffset, last) - assertTrue(log.read(0, 64*1024).size > 0) - log.delete() - } - } - /** * When we open a log any index segments without an associated log segment should be deleted. */ @@ -399,70 +455,195 @@ class LogTest extends JUnitSuite { def testBogusIndexSegmentsAreRemoved() { val bogusIndex1 = Log.indexFilename(logDir, 0) val bogusIndex2 = Log.indexFilename(logDir, 5) - + val set = TestUtils.singleMessageSet("test".getBytes()) - val log = new Log(logDir, - maxLogFileSize = set.sizeInBytes * 5, - maxMessageSize = config.messageMaxBytes, - maxIndexSize = 1000, - indexIntervalBytes = 1, - needsRecovery = false) - + val log = new Log(logDir, + logConfig.copy(segmentSize = set.sizeInBytes * 5, + maxIndexSize = 1000, + indexInterval = 1), + recoveryPoint = 0L, + time.scheduler, + time) + assertTrue("The first index file should have been replaced with a larger file", bogusIndex1.length > 0) assertFalse("The second index file should have been deleted.", bogusIndex2.exists) - + // check that we can append to the log for(i <- 0 until 10) log.append(set) - + log.delete() } + /** + * Verify that truncation works correctly after re-opening the log + */ @Test def testReopenThenTruncate() { val set = TestUtils.singleMessageSet("test".getBytes()) + val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, + maxIndexSize = 1000, + indexInterval = 10000) // create a log - var log = new Log(logDir, - maxLogFileSize = set.sizeInBytes * 5, - maxMessageSize = config.messageMaxBytes, - maxIndexSize = 1000, - indexIntervalBytes = 10000, - needsRecovery = true) - + var log = new Log(logDir, + config, + recoveryPoint = 0L, + time.scheduler, + time) + // add enough messages to roll over several segments then close and re-open and attempt to truncate for(i <- 0 until 100) log.append(set) log.close() - log = new Log(logDir, - maxLogFileSize = set.sizeInBytes * 5, - maxMessageSize = config.messageMaxBytes, - maxIndexSize = 1000, - indexIntervalBytes = 10000, - needsRecovery = true) + log = new Log(logDir, + config, + recoveryPoint = 0L, + time.scheduler, + time) log.truncateTo(3) assertEquals("All but one segment should be deleted.", 1, log.numberOfSegments) assertEquals("Log end offset should be 3.", 3, log.logEndOffset) } - - def assertContains(ranges: Array[Range], offset: Long) = { - Log.findRange(ranges, offset) match { - case Some(range) => - assertTrue(range + " does not contain " + offset, range.contains(offset)) - case None => fail("No range found, but expected to find " + offset) + + /** + * Test that deleted files are deleted after the appropriate time. + */ + @Test + def testAsyncDelete() { + val set = TestUtils.singleMessageSet("test".getBytes()) + val asyncDeleteMs = 1000 + val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, + fileDeleteDelayMs = asyncDeleteMs, + maxIndexSize = 1000, + indexInterval = 10000) + val log = new Log(logDir, + config, + recoveryPoint = 0L, + time.scheduler, + time) + + // append some messages to create some segments + for(i <- 0 until 100) + log.append(set) + + // files should be renamed + val segments = log.logSegments.toArray + val oldFiles = segments.map(_.log.file) ++ segments.map(_.index.file) + log.deleteOldSegments((s) => true) + + assertEquals("Only one segment should remain.", 1, log.numberOfSegments) + assertTrue("All log and index files should end in .deleted", segments.forall(_.log.file.getName.endsWith(Log.DeletedFileSuffix)) && + segments.forall(_.index.file.getName.endsWith(Log.DeletedFileSuffix))) + assertTrue("The .deleted files should still be there.", segments.forall(_.log.file.exists) && + segments.forall(_.index.file.exists)) + assertTrue("The original file should be gone.", oldFiles.forall(!_.exists)) + + // when enough time passes the files should be deleted + val deletedFiles = segments.map(_.log.file) ++ segments.map(_.index.file) + time.sleep(asyncDeleteMs + 1) + assertTrue("Files should all be gone.", deletedFiles.forall(!_.exists)) + } + + /** + * Any files ending in .deleted should be removed when the log is re-opened. + */ + @Test + def testOpenDeletesObsoleteFiles() { + val set = TestUtils.singleMessageSet("test".getBytes()) + val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, maxIndexSize = 1000) + var log = new Log(logDir, + config, + recoveryPoint = 0L, + time.scheduler, + time) + + // append some messages to create some segments + for(i <- 0 until 100) + log.append(set) + + log.deleteOldSegments((s) => true) + log.close() + + log = new Log(logDir, + config, + recoveryPoint = 0L, + time.scheduler, + time) + assertEquals("The deleted segments should be gone.", 1, log.numberOfSegments) + } + + @Test + def testAppendMessageWithNullPayload() { + var log = new Log(logDir, + LogConfig(), + recoveryPoint = 0L, + time.scheduler, + time) + log.append(new ByteBufferMessageSet(new Message(bytes = null))) + val ms = log.read(0, 4096, None) + assertEquals(0, ms.head.offset) + assertTrue("Message payload should be null.", ms.head.message.isNull) + } + + @Test + def testCorruptLog() { + // append some messages to create some segments + val config = logConfig.copy(indexInterval = 1, maxMessageSize = 64*1024, segmentSize = 1000) + val set = TestUtils.singleMessageSet("test".getBytes()) + val recoveryPoint = 50L + for(iteration <- 0 until 50) { + // create a log and write some messages to it + logDir.mkdirs() + var log = new Log(logDir, + config, + recoveryPoint = 0L, + time.scheduler, + time) + val numMessages = 50 + TestUtils.random.nextInt(50) + for(i <- 0 until numMessages) + log.append(set) + val messages = log.logSegments.flatMap(_.log.iterator.toList) + log.close() + + // corrupt index and log by appending random bytes + TestUtils.appendNonsenseToFile(log.activeSegment.index.file, TestUtils.random.nextInt(1024) + 1) + TestUtils.appendNonsenseToFile(log.activeSegment.log.file, TestUtils.random.nextInt(1024) + 1) + + // attempt recovery + log = new Log(logDir, config, recoveryPoint, time.scheduler, time) + assertEquals(numMessages, log.logEndOffset) + assertEquals("Messages in the log after recovery should be the same.", messages, log.logSegments.flatMap(_.log.iterator.toList)) + Utils.rm(logDir) } } - class SimpleRange(val start: Long, val size: Long) extends Range + @Test + def testCleanShutdownFile() { + // append some messages to create some segments + val config = logConfig.copy(indexInterval = 1, maxMessageSize = 64*1024, segmentSize = 1000) + val set = TestUtils.singleMessageSet("test".getBytes()) + val parentLogDir = logDir.getParentFile + assertTrue("Data directory %s must exist", parentLogDir.isDirectory) + val cleanShutdownFile = new File(parentLogDir, Log.CleanShutdownFile) + cleanShutdownFile.createNewFile() + assertTrue(".kafka_cleanshutdown must exist", cleanShutdownFile.exists()) + var recoveryPoint = 0L + // create a log and write some messages to it + var log = new Log(logDir, + config, + recoveryPoint = 0L, + time.scheduler, + time) + for(i <- 0 until 100) + log.append(set) + log.close() - def makeRanges(breaks: Int*): Array[Range] = { - val list = new ArrayList[Range] - var prior = 0 - for(brk <- breaks) { - list.add(new SimpleRange(prior, brk - prior)) - prior = brk - } - list.toArray(new Array[Range](list.size)) + // check if recovery was attempted. Even if the recovery point is 0L, recovery should not be attempted as the + // clean shutdown file exists. + recoveryPoint = log.logEndOffset + log = new Log(logDir, config, 0L, time.scheduler, time) + assertEquals(recoveryPoint, log.logEndOffset) + cleanShutdownFile.delete() } - } diff --git a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala new file mode 100644 index 0000000000000..12ce39e665afd --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +import java.nio._ +import org.junit._ +import org.scalatest.junit.JUnitSuite +import junit.framework.Assert._ + +class OffsetMapTest extends JUnitSuite { + + @Test + def testBasicValidation() { + validateMap(10) + validateMap(100) + validateMap(1000) + validateMap(5000) + } + + @Test + def testClear() { + val map = new SkimpyOffsetMap(4000) + for(i <- 0 until 10) + map.put(key(i), i) + for(i <- 0 until 10) + assertEquals(i.toLong, map.get(key(i))) + map.clear() + for(i <- 0 until 10) + assertEquals(map.get(key(i)), -1L) + } + + def key(key: Int) = ByteBuffer.wrap(key.toString.getBytes) + + def validateMap(items: Int, loadFactor: Double = 0.5): SkimpyOffsetMap = { + val map = new SkimpyOffsetMap((items/loadFactor * 24).toInt) + for(i <- 0 until items) + map.put(key(i), i) + var misses = 0 + for(i <- 0 until items) + assertEquals(map.get(key(i)), i.toLong) + map + } + +} + +object OffsetMapTest { + def main(args: Array[String]) { + if(args.length != 2) { + System.err.println("USAGE: java OffsetMapTest size load") + System.exit(1) + } + val test = new OffsetMapTest() + val size = args(0).toInt + val load = args(1).toDouble + val start = System.nanoTime + val map = test.validateMap(size, load) + val ellapsedMs = (System.nanoTime - start) / 1000.0 / 1000.0 + println(map.size + " entries in map of size " + map.slots + " in " + ellapsedMs + " ms") + println("Collision rate: %.1f%%".format(100*map.collisionRate)) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/log/SegmentListTest.scala b/core/src/test/scala/unit/kafka/log/SegmentListTest.scala deleted file mode 100644 index bf91ff1751091..0000000000000 --- a/core/src/test/scala/unit/kafka/log/SegmentListTest.scala +++ /dev/null @@ -1,113 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.log - -import junit.framework.Assert._ -import org.junit.Test -import org.scalatest.junit.JUnitSuite -import kafka.common.KafkaException - -class SegmentListTest extends JUnitSuite { - - @Test - def testAppend() { - val list = List(1, 2, 3, 4) - val sl = new SegmentList(list) - val view = sl.view - assertEquals(list, view.iterator.toList) - sl.append(5) - assertEquals("Appending to both should result in lists that are still equals", - list ::: List(5), sl.view.iterator.toList) - assertEquals("But the prior view should still equal the original list", list, view.iterator.toList) - } - - @Test - def testTrunc() { - { - val hd = List(1,2,3) - val tail = List(4,5,6) - val sl = new SegmentList(hd ::: tail) - val view = sl.view - assertEquals(hd ::: tail, view.iterator.toList) - val deleted = sl.trunc(3) - assertEquals(tail, sl.view.iterator.toList) - assertEquals(hd, deleted.iterator.toList) - assertEquals("View should remain consistent", hd ::: tail, view.iterator.toList) - } - { - val hd = List(1,2,3,4,5) - val tail = List(6) - val sl = new SegmentList(hd ::: tail) - val view = sl.view - assertEquals(hd ::: tail, view.iterator.toList) - try { - sl.trunc(-1) - fail("Attempt to truncate with illegal index should fail") - } catch { - case e: KafkaException => // this is ok - } - val deleted = sl.truncLast(4) - assertEquals(hd, sl.view.iterator.toList) - assertEquals(tail, deleted.iterator.toList) - assertEquals("View should remain consistent", hd ::: tail, view.iterator.toList) - } - { - val sl = new SegmentList(List(1, 2)) - sl.trunc(3) - assertEquals(0, sl.view.length) - } - } - - @Test - def testTruncLast() { - { - val hd = List(1,2,3) - val tail = List(4,5,6) - val sl = new SegmentList(hd ::: tail) - val view = sl.view - assertEquals(hd ::: tail, view.iterator.toList) - val deleted = sl.truncLast(2) - assertEquals(hd, sl.view.iterator.toList) - assertEquals(tail, deleted.iterator.toList) - assertEquals("View should remain consistent", hd ::: tail, view.iterator.toList) - } - { - val hd = List(1,2,3,4,5) - val tail = List(6) - val sl = new SegmentList(hd ::: tail) - val view = sl.view - assertEquals(hd ::: tail, view.iterator.toList) - try { - sl.truncLast(6) - fail("Attempt to truncate with illegal index should fail") - } catch { - case e: KafkaException => // this is ok - } - try { - sl.truncLast(-1) - fail("Attempt to truncate with illegal index should fail") - } catch { - case e: KafkaException => // this is ok - } - val deleted = sl.truncLast(4) - assertEquals(hd, sl.view.iterator.toList) - assertEquals(tail, deleted.iterator.toList) - assertEquals("View should remain consistent", hd ::: tail, view.iterator.toList) - } - } -} diff --git a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala index ef74ba8f213b3..6db245c956d21 100644 --- a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala +++ b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala @@ -26,7 +26,7 @@ import org.junit.Test trait BaseMessageSetTestCases extends JUnitSuite { - val messages = Array(new Message("abcd".getBytes()), new Message("efgh".getBytes())) + val messages = Array(new Message("abcd".getBytes), new Message("efgh".getBytes), new Message("ijkl".getBytes)) def createMessageSet(messages: Seq[Message]): MessageSet diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index accb7b2406102..4837585d03535 100644 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -17,8 +17,8 @@ package kafka.message -import java.util._ import java.nio._ +import java.util.HashMap import scala.collection._ import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite @@ -38,16 +38,21 @@ class MessageTest extends JUnitSuite { @Before def setUp(): Unit = { val keys = Array(null, "key".getBytes, "".getBytes) - val vals = Array("value".getBytes, "".getBytes) + val vals = Array("value".getBytes, "".getBytes, null) val codecs = Array(NoCompressionCodec, GZIPCompressionCodec) for(k <- keys; v <- vals; codec <- codecs) messages += new MessageTestVal(k, v, codec, new Message(v, k, codec)) } @Test - def testFieldValues = { + def testFieldValues { for(v <- messages) { - TestUtils.checkEquals(ByteBuffer.wrap(v.payload), v.message.payload) + if(v.payload == null) { + assertTrue(v.message.isNull) + assertEquals("Payload should be null", null, v.message.payload) + } else { + TestUtils.checkEquals(ByteBuffer.wrap(v.payload), v.message.payload) + } assertEquals(Message.CurrentMagicValue, v.message.magic) if(v.message.hasKey) TestUtils.checkEquals(ByteBuffer.wrap(v.key), v.message.key) @@ -69,7 +74,7 @@ class MessageTest extends JUnitSuite { } @Test - def testEquality() = { + def testEquality() { for(v <- messages) { assertFalse("Should not equal null", v.message.equals(null)) assertFalse("Should not equal a random string", v.message.equals("asdf")) @@ -80,7 +85,7 @@ class MessageTest extends JUnitSuite { } @Test - def testIsHashable() = { + def testIsHashable() { // this is silly, but why not val m = new HashMap[Message, Message]() for(v <- messages) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 94b5a2a36b2b8..4ff6f55914909 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -29,6 +29,7 @@ import java.nio.ByteBuffer import kafka.common.TopicAndPartition import kafka.message.ByteBufferMessageSet import java.nio.channels.SelectionKey +import kafka.utils.TestUtils class SocketServerTest extends JUnitSuite { @@ -107,7 +108,7 @@ class SocketServerTest extends JUnitSuite { } @Test - def testPipelinedRequestOrdering() { + def testSocketSelectionKeyState() { val socket = connect() val correlationId = -1 val clientId = SyncProducerConfig.DefaultClientId @@ -122,20 +123,19 @@ class SocketServerTest extends JUnitSuite { val serializedBytes = new Array[Byte](byteBuffer.remaining) byteBuffer.get(serializedBytes) - sendRequest(socket, 0, serializedBytes) sendRequest(socket, 0, serializedBytes) - // here the socket server should've read only the first request completely and since the response is not sent yet - // the selection key should not be readable val request = server.requestChannel.receiveRequest + // Since the response is not sent yet, the selection key should not be readable. Assert.assertFalse((request.requestKey.asInstanceOf[SelectionKey].interestOps & SelectionKey.OP_READ) == SelectionKey.OP_READ) server.requestChannel.sendResponse(new RequestChannel.Response(0, request, null)) - // if everything is working correctly, until you send a response for the first request, - // the 2nd request will not be read by the socket server - val request2 = server.requestChannel.receiveRequest - server.requestChannel.sendResponse(new RequestChannel.Response(0, request2, null)) - Assert.assertFalse((request.requestKey.asInstanceOf[SelectionKey].interestOps & SelectionKey.OP_READ) == SelectionKey.OP_READ) + // After the response is sent to the client (which is async and may take a bit of time), the socket key should be available for reads. + Assert.assertTrue( + TestUtils.waitUntilTrue( + () => { (request.requestKey.asInstanceOf[SelectionKey].interestOps & SelectionKey.OP_READ) == SelectionKey.OP_READ }, + 5000) + ) } } diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index 69c88c73521d5..210583f2d1ab6 100644 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -158,16 +158,17 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testPartitionAndCollateEvents() { val producerDataList = new ArrayBuffer[KeyedMessage[Int,Message]] - producerDataList.append(new KeyedMessage[Int,Message]("topic1", 0, new Message("msg1".getBytes))) - producerDataList.append(new KeyedMessage[Int,Message]("topic2", 1, new Message("msg2".getBytes))) - producerDataList.append(new KeyedMessage[Int,Message]("topic1", 2, new Message("msg3".getBytes))) - producerDataList.append(new KeyedMessage[Int,Message]("topic1", 3, new Message("msg4".getBytes))) - producerDataList.append(new KeyedMessage[Int,Message]("topic2", 4, new Message("msg5".getBytes))) + // use bogus key and partition key override for some messages + producerDataList.append(new KeyedMessage[Int,Message]("topic1", key = 0, message = new Message("msg1".getBytes))) + producerDataList.append(new KeyedMessage[Int,Message]("topic2", key = -99, partKey = 1, message = new Message("msg2".getBytes))) + producerDataList.append(new KeyedMessage[Int,Message]("topic1", key = 2, message = new Message("msg3".getBytes))) + producerDataList.append(new KeyedMessage[Int,Message]("topic1", key = -101, partKey = 3, message = new Message("msg4".getBytes))) + producerDataList.append(new KeyedMessage[Int,Message]("topic2", key = 4, message = new Message("msg5".getBytes))) val props = new Properties() props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) - val broker1 = new Broker(0, "localhost", 9092) - val broker2 = new Broker(1, "localhost", 9093) + val broker1 = new Broker(0, "localhost", 9092, 0) + val broker2 = new Broker(1, "localhost", 9093, 0) broker1 // form expected partitions metadata val partition1Metadata = new PartitionMetadata(0, Some(broker1), List(broker1, broker2)) @@ -179,8 +180,8 @@ class AsyncProducerTest extends JUnit3Suite { topicPartitionInfos.put("topic1", topic1Metadata) topicPartitionInfos.put("topic2", topic2Metadata) - val intPartitioner = new Partitioner[Int] { - def partition(key: Int, numPartitions: Int): Int = key % numPartitions + val intPartitioner = new Partitioner { + def partition(key: Any, numPartitions: Int): Int = key.asInstanceOf[Int] % numPartitions } val config = new ProducerConfig(props) @@ -195,9 +196,9 @@ class AsyncProducerTest extends JUnit3Suite { val topic1Broker1Data = ArrayBuffer[KeyedMessage[Int,Message]](new KeyedMessage[Int,Message]("topic1", 0, new Message("msg1".getBytes)), new KeyedMessage[Int,Message]("topic1", 2, new Message("msg3".getBytes))) - val topic1Broker2Data = ArrayBuffer[KeyedMessage[Int,Message]](new KeyedMessage[Int,Message]("topic1", 3, new Message("msg4".getBytes))) + val topic1Broker2Data = ArrayBuffer[KeyedMessage[Int,Message]](new KeyedMessage[Int,Message]("topic1", -101, 3, new Message("msg4".getBytes))) val topic2Broker1Data = ArrayBuffer[KeyedMessage[Int,Message]](new KeyedMessage[Int,Message]("topic2", 4, new Message("msg5".getBytes))) - val topic2Broker2Data = ArrayBuffer[KeyedMessage[Int,Message]](new KeyedMessage[Int,Message]("topic2", 1, new Message("msg2".getBytes))) + val topic2Broker2Data = ArrayBuffer[KeyedMessage[Int,Message]](new KeyedMessage[Int,Message]("topic2", -99, 1, new Message("msg2".getBytes))) val expectedResult = Some(Map( 0 -> Map( TopicAndPartition("topic1", 0) -> topic1Broker1Data, @@ -225,7 +226,7 @@ class AsyncProducerTest extends JUnit3Suite { val producerPool = new ProducerPool(config) val handler = new DefaultEventHandler[String,String](config, - partitioner = null.asInstanceOf[Partitioner[String]], + partitioner = null.asInstanceOf[Partitioner], encoder = new StringEncoder, keyEncoder = new StringEncoder, producerPool = producerPool, @@ -285,7 +286,7 @@ class AsyncProducerTest extends JUnit3Suite { val producerDataList = new ArrayBuffer[KeyedMessage[String,String]] producerDataList.append(new KeyedMessage[String,String]("topic1", "msg1")) val handler = new DefaultEventHandler[String,String](config, - partitioner = null.asInstanceOf[Partitioner[String]], + partitioner = null.asInstanceOf[Partitioner], encoder = new StringEncoder, keyEncoder = new StringEncoder, producerPool = producerPool, @@ -332,7 +333,7 @@ class AsyncProducerTest extends JUnit3Suite { val producerPool = new ProducerPool(config) val handler = new DefaultEventHandler[String,String](config, - partitioner = null.asInstanceOf[Partitioner[String]], + partitioner = null.asInstanceOf[Partitioner], encoder = null.asInstanceOf[Encoder[String]], keyEncoder = null.asInstanceOf[Encoder[String]], producerPool = producerPool, @@ -373,7 +374,7 @@ class AsyncProducerTest extends JUnit3Suite { val msgs = TestUtils.getMsgStrings(10) val handler = new DefaultEventHandler[String,String](config, - partitioner = null.asInstanceOf[Partitioner[String]], + partitioner = null.asInstanceOf[Partitioner], encoder = new StringEncoder, keyEncoder = new StringEncoder, producerPool = producerPool, @@ -397,11 +398,12 @@ class AsyncProducerTest extends JUnit3Suite { props.put("request.required.acks", "1") props.put("serializer.class", classOf[StringEncoder].getName.toString) props.put("key.serializer.class", classOf[NullEncoder[Int]].getName.toString) + props.put("producer.num.retries", 3.toString) val config = new ProducerConfig(props) val topic1 = "topic1" - val topic1Metadata = getTopicMetadata(topic1, Array(0, 1), 0, "localhost", 9092) + val topic1Metadata = getTopicMetadata(topic1, Array(0, 1), 0, "localhost", 9092, 0) val topicPartitionInfos = new collection.mutable.HashMap[String, TopicMetadata] topicPartitionInfos.put("topic1", topic1Metadata) @@ -450,8 +452,10 @@ class AsyncProducerTest extends JUnit3Suite { val topic = "topic1" val msgs = TestUtils.getMsgStrings(5) val scalaProducerData = msgs.map(m => new KeyedMessage[String, String](topic, m)) - import scala.collection.JavaConversions._ - val javaProducerData: java.util.List[KeyedMessage[String, String]] = scalaProducerData + val javaProducerData: java.util.List[KeyedMessage[String, String]] = { + import scala.collection.JavaConversions._ + scalaProducerData + } val mockScalaProducer = EasyMock.createMock(classOf[kafka.producer.Producer[String, String]]) mockScalaProducer.send(scalaProducerData.head) @@ -488,12 +492,12 @@ class AsyncProducerTest extends JUnit3Suite { producerDataList } - private def getTopicMetadata(topic: String, partition: Int, brokerId: Int, brokerHost: String, brokerPort: Int): TopicMetadata = { - getTopicMetadata(topic, List(partition), brokerId, brokerHost, brokerPort) + private def getTopicMetadata(topic: String, partition: Int, brokerId: Int, brokerHost: String, brokerPort: Int, rackId: Int = 0): TopicMetadata = { + getTopicMetadata(topic, List(partition), brokerId, brokerHost, brokerPort, rackId) } - private def getTopicMetadata(topic: String, partition: Seq[Int], brokerId: Int, brokerHost: String, brokerPort: Int): TopicMetadata = { - val broker1 = new Broker(brokerId, brokerHost, brokerPort) + private def getTopicMetadata(topic: String, partition: Seq[Int], brokerId: Int, brokerHost: String, brokerPort: Int, rackId: Int): TopicMetadata = { + val broker1 = new Broker(brokerId, brokerHost, brokerPort, rackId) new TopicMetadata(topic, partition.map(new PartitionMetadata(_, Some(broker1), List(broker1)))) } @@ -506,6 +510,6 @@ class AsyncProducerTest extends JUnit3Suite { } } -class NegativePartitioner(props: VerifiableProperties = null) extends Partitioner[String] { - def partition(data: String, numPartitions: Int): Int = -1 +class NegativePartitioner(props: VerifiableProperties = null) extends Partitioner { + def partition(data: Any, numPartitions: Int): Int = -1 } diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 2cabfbb761657..4b2e4ade70f6f 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -27,14 +27,13 @@ import org.apache.log4j.{Level, Logger} import org.junit.Test import kafka.utils._ import java.util -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import util.Properties import kafka.api.FetchRequestBuilder -import kafka.common.FailedToSendMessageException import org.junit.Assert.assertTrue import org.junit.Assert.assertFalse import org.junit.Assert.assertEquals - +import kafka.common.{ErrorMapping, FailedToSendMessageException} class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ @@ -50,15 +49,11 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ private var servers = List.empty[KafkaServer] private val props1 = TestUtils.createBrokerConfig(brokerId1, port1) - private val config1 = new KafkaConfig(props1) { - override val hostName = "localhost" - override val numPartitions = 4 - } + props1.put("num.partitions", "4") + private val config1 = new KafkaConfig(props1) private val props2 = TestUtils.createBrokerConfig(brokerId2, port2) - private val config2 = new KafkaConfig(props2) { - override val hostName = "localhost" - override val numPartitions = 4 - } + props2.put("num.partitions", "4") + private val config2 = new KafkaConfig(props2) override def setUp() { super.setUp() @@ -82,18 +77,16 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ // restore set request handler logger to a higher level requestHandlerLogger.setLevel(Level.ERROR) server1.shutdown - server1.awaitShutdown() server2.shutdown - server2.awaitShutdown() Utils.rm(server1.config.logDirs) Utils.rm(server2.config.logDirs) super.tearDown() } - + @Test def testUpdateBrokerPartitionInfo() { val topic = "new-topic" - CreateTopicCommand.createTopic(zkClient, "new-topic", 1, 2) + AdminUtils.createTopic(zkClient, topic, 1, 2) // wait until the update metadata request for new topic reaches all servers TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 500) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) @@ -159,7 +152,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val topic = "new-topic" // create topic with 1 partition and await leadership - CreateTopicCommand.createTopic(zkClient, topic, 1, 2) + AdminUtils.createTopic(zkClient, topic, 1, 2) TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) @@ -210,7 +203,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val topic = "new-topic" // create topic - CreateTopicCommand.createTopic(zkClient, topic, 4, 2, "0,0,0,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) @@ -273,7 +266,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val topic = "new-topic" // create topics in ZK - CreateTopicCommand.createTopic(zkClient, topic, 4, 2, "0:1,0:1,0:1,0:1") + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) @@ -303,7 +296,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ case e: FailedToSendMessageException => /* success */ case e: Exception => fail("Not expected", e) } finally { - producer.close + producer.close() } val t2 = SystemTime.milliseconds @@ -311,5 +304,27 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ // we do this because the DefaultEventHandler retries a number of times assertTrue((t2-t1) >= timeoutMs*config.messageSendMaxRetries) } -} + + @Test + def testSendNullMessage() { + val props = new Properties() + props.put("serializer.class", "kafka.serializer.StringEncoder") + props.put("partitioner.class", "kafka.utils.StaticPartitioner") + props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + + val config = new ProducerConfig(props) + val producer = new Producer[String, String](config) + try { + // create topic + AdminUtils.createTopic(zkClient, "new-topic", 2, 1) + assertTrue("Topic new-topic not created after timeout", TestUtils.waitUntilTrue(() => + AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, zookeeper.tickTime)) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0, 500) + + producer.send(new KeyedMessage[String, String]("new-topic", "key", null)) + } finally { + producer.close() + } + } +} diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 3592bff263da0..8d63e312d3eda 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -19,7 +19,7 @@ package kafka.producer import java.net.SocketTimeoutException import junit.framework.Assert -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import kafka.integration.KafkaServerTestHarness import kafka.message._ import kafka.server.KafkaConfig @@ -92,7 +92,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { val props = TestUtils.getSyncProducerConfig(server.socketServer.port) val producer = new SyncProducer(new SyncProducerConfig(props)) - CreateTopicCommand.createTopic(zkClient, "test", 1, 1) + AdminUtils.createTopic(zkClient, "test", 1, 1) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "test", 0, 500) val message1 = new Message(new Array[Byte](configs(0).messageMaxBytes + 1)) @@ -121,7 +121,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { props.put("request.required.acks", "0") val producer = new SyncProducer(new SyncProducerConfig(props)) - CreateTopicCommand.createTopic(zkClient, "test", 1, 1) + AdminUtils.createTopic(zkClient, "test", 1, 1) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "test", 0, 500) // This message will be dropped silently since message size too large. @@ -162,9 +162,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { } // #2 - test that we get correct offsets when partition is owned by broker - CreateTopicCommand.createTopic(zkClient, "topic1", 1, 1) + AdminUtils.createTopic(zkClient, "topic1", 1, 1) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "topic1", 0, 500) - CreateTopicCommand.createTopic(zkClient, "topic3", 1, 1) + AdminUtils.createTopic(zkClient, "topic3", 1, 1) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "topic3", 0, 500) val response2 = producer.send(request) diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala new file mode 100644 index 0000000000000..f0c4a56b61b4f --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import org.scalatest.junit.JUnit3Suite +import kafka.zk.ZooKeeperTestHarness +import junit.framework.Assert._ +import kafka.utils.{ZkUtils, Utils, TestUtils} + +class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { + var server : KafkaServer = null + val brokerId = 0 + val advertisedHostName = "routable-host" + val advertisedPort = 1234 + + override def setUp() { + super.setUp() + val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) + props.put("advertised.host.name", advertisedHostName) + props.put("advertised.port", advertisedPort.toString) + + server = TestUtils.createServer(new KafkaConfig(props)) + } + + override def tearDown() { + server.shutdown() + Utils.rm(server.config.logDirs) + super.tearDown() + } + + def testBrokerAdvertiseToZK { + val brokerInfo = ZkUtils.getBrokerInfo(zkClient, brokerId) + assertEquals(advertisedHostName, brokerInfo.get.host) + assertEquals(advertisedPort, brokerInfo.get.port) + } + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 2719055087299..02c188a412995 100644 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -16,13 +16,14 @@ */ package kafka.server -import kafka.log.LogManager +import kafka.log._ +import java.io.File import org.I0Itec.zkclient.ZkClient import org.scalatest.junit.JUnit3Suite import org.easymock.EasyMock import org.junit._ import org.junit.Assert._ -import kafka.common.KafkaException +import kafka.common._ import kafka.cluster.Replica import kafka.utils.{SystemTime, KafkaScheduler, TestUtils, MockTime, Utils} import java.util.concurrent.atomic.AtomicBoolean @@ -31,7 +32,15 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_)) val topic = "foo" - val logManagers = configs.map(config => new LogManager(config, new KafkaScheduler(1), new MockTime)) + val logManagers = configs.map(config => new LogManager(logDirs = config.logDirs.map(new File(_)).toArray, + topicConfigs = Map(), + defaultConfig = LogConfig(), + cleanerConfig = CleanerConfig(), + flushCheckMs = 30000, + flushCheckpointMs = 10000L, + retentionCheckMs = 30000, + scheduler = new KafkaScheduler(1), + time = new MockTime)) @After def teardown() { @@ -55,7 +64,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { assertEquals(0L, fooPartition0Hw) val partition0 = replicaManager.getOrCreatePartition(topic, 0, 1) // create leader and follower replicas - val log0 = logManagers(0).getOrCreateLog(topic, 0) + val log0 = logManagers(0).createLog(TopicAndPartition(topic, 0), LogConfig()) val leaderReplicaPartition0 = new Replica(configs.head.brokerId, partition0, SystemTime, 0, Some(log0)) partition0.addReplicaIfNotExists(leaderReplicaPartition0) val followerReplicaPartition0 = new Replica(configs.last.brokerId, partition0, SystemTime) @@ -94,7 +103,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { assertEquals(0L, topic1Partition0Hw) val topic1Partition0 = replicaManager.getOrCreatePartition(topic1, 0, 1) // create leader log - val topic1Log0 = logManagers(0).getOrCreateLog(topic1, 0) + val topic1Log0 = logManagers(0).createLog(TopicAndPartition(topic1, 0), LogConfig()) // create a local replica for topic1 val leaderReplicaTopic1Partition0 = new Replica(configs.head.brokerId, topic1Partition0, SystemTime, 0, Some(topic1Log0)) topic1Partition0.addReplicaIfNotExists(leaderReplicaTopic1Partition0) @@ -110,7 +119,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { // add another partition and set highwatermark val topic2Partition0 = replicaManager.getOrCreatePartition(topic2, 0, 1) // create leader log - val topic2Log0 = logManagers(0).getOrCreateLog(topic2, 0) + val topic2Log0 = logManagers(0).createLog(TopicAndPartition(topic2, 0), LogConfig()) // create a local replica for topic2 val leaderReplicaTopic2Partition0 = new Replica(configs.head.brokerId, topic2Partition0, SystemTime, 0, Some(topic2Log0)) topic2Partition0.addReplicaIfNotExists(leaderReplicaTopic2Partition0) @@ -134,7 +143,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { } def hwmFor(replicaManager: ReplicaManager, topic: String, partition: Int): Long = { - replicaManager.highWatermarkCheckpoints(replicaManager.config.logDirs(0)).read(topic, partition) + replicaManager.highWatermarkCheckpoints(replicaManager.config.logDirs(0)).read.getOrElse(TopicAndPartition(topic, partition), 0L) } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index 702643246939f..2cd3a3faf7be2 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -31,6 +31,7 @@ class IsrExpirationTest extends JUnit3Suite { var topicPartitionIsr: Map[(String, Int), Seq[Int]] = new HashMap[(String, Int), Seq[Int]]() val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) { override val replicaLagTimeMaxMs = 100L + override val replicaFetchWaitMaxMs = 100 override val replicaLagMaxMessages = 10L }) val topic = "foo" diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala new file mode 100644 index 0000000000000..89c207a3f56c7 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -0,0 +1,97 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import org.junit.Test +import junit.framework.Assert._ +import org.scalatest.junit.JUnit3Suite +import kafka.utils.TestUtils + +class KafkaConfigTest extends JUnit3Suite { + + @Test + def testLogRetentionTimeHoursProvided() { + val props = TestUtils.createBrokerConfig(0, 8181) + props.put("log.retention.hours", "1") + + val cfg = new KafkaConfig(props) + assertEquals(60L * 60L * 1000L, cfg.logRetentionTimeMillis) + + } + + @Test + def testLogRetentionTimeMinutesProvided() { + val props = TestUtils.createBrokerConfig(0, 8181) + props.put("log.retention.minutes", "30") + + val cfg = new KafkaConfig(props) + assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) + + } + + @Test + def testLogRetentionTimeNoConfigProvided() { + val props = TestUtils.createBrokerConfig(0, 8181) + + val cfg = new KafkaConfig(props) + assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRetentionTimeMillis) + + } + + @Test + def testLogRetentionTimeBothMinutesAndHoursProvided() { + val props = TestUtils.createBrokerConfig(0, 8181) + props.put("log.retention.minutes", "30") + props.put("log.retention.hours", "1") + + val cfg = new KafkaConfig(props) + assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) + + } + + @Test + def testAdvertiseDefaults() { + val port = 9999 + val hostName = "fake-host" + + val props = TestUtils.createBrokerConfig(0, port) + props.put("host.name", hostName) + + val serverConfig = new KafkaConfig(props) + + assertEquals(serverConfig.advertisedHostName, hostName) + assertEquals(serverConfig.advertisedPort, port) + } + + @Test + def testAdvertiseConfigured() { + val port = 9999 + val advertisedHostName = "routable-host" + val advertisedPort = 1234 + + val props = TestUtils.createBrokerConfig(0, port) + props.put("advertised.host.name", advertisedHostName) + props.put("advertised.port", advertisedPort.toString) + + val serverConfig = new KafkaConfig(props) + + assertEquals(serverConfig.advertisedHostName, advertisedHostName) + assertEquals(serverConfig.advertisedPort, advertisedPort) + } + +} diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 70e4b51a84239..a2215cfabbfe3 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -19,7 +19,7 @@ package kafka.server import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} @@ -61,7 +61,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val partitionId = 0 // create topic with 1 partition, 2 replicas, one on each broker - CreateTopicCommand.createTopic(zkClient, topic, 1, 2, "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 - CreateTopicCommand.createTopic(zkClient, topic, 1, 2, "0:1") + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(0, 1))) // wait until leader is elected val leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) @@ -123,7 +123,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // start another controller 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 brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.config.port, s.config.rackId)) val controllerContext = new ControllerContext(zkClient, 6000) controllerContext.liveBrokers = brokers.toSet val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) diff --git a/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala similarity index 84% rename from core/src/test/scala/unit/kafka/log/LogOffsetTest.scala rename to core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 1a9cc0167f95f..9aea67b140e50 100644 --- a/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package kafka.log +package kafka.server import java.io.File import kafka.utils._ -import kafka.server.{KafkaConfig, KafkaServer} import junit.framework.Assert._ import java.util.{Random, Properties} import kafka.consumer.SimpleConsumer @@ -27,16 +26,18 @@ import org.junit.{After, Before, Test} import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} import kafka.zk.ZooKeeperTestHarness import org.scalatest.junit.JUnit3Suite -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import kafka.api.{PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest} import kafka.utils.TestUtils._ import kafka.common.{ErrorMapping, TopicAndPartition} - -object LogOffsetTest { - val random = new Random() -} +import kafka.utils.nonthreadsafe +import kafka.utils.threadsafe +import org.junit.After +import org.junit.Before +import org.junit.Test class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { + val random = new Random() var logDir: File = null var topicLogDir: File = null var server: KafkaServer = null @@ -81,17 +82,17 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { val part = Integer.valueOf(topicPartition.split("-").last).intValue // setup brokers in zookeeper as owners of partitions for this test - CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "1") + AdminUtils.createTopic(zkClient, topic, 1, 1) val logManager = server.getLogManager - val log = logManager.getOrCreateLog(topic, part) + val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig) val message = new Message(Integer.toString(42).getBytes()) for(i <- 0 until 20) log.append(new ByteBufferMessageSet(NoCompressionCodec, message)) log.flush() - val offsets = log.getOffsetsBefore(OffsetRequest.LatestTime, 10) + val offsets = server.apis.fetchOffsets(logManager, TopicAndPartition(topic, part), OffsetRequest.LatestTime, 10) assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), offsets) waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), 1000) @@ -111,7 +112,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testEmptyLogsGetOffsets() { - val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(10) + val topicPartition = "kafka-" + random.nextInt(10) val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition topicLogDir = new File(topicPartitionPath) topicLogDir.mkdir @@ -119,7 +120,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { val topic = topicPartition.split("-").head // setup brokers in zookeeper as owners of partitions for this test - CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "1") + AdminUtils.createTopic(zkClient, topic, 1, 1) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) var offsetChanged = false @@ -139,24 +140,23 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testGetOffsetsBeforeNow() { - val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(3) + val topicPartition = "kafka-" + random.nextInt(3) val topic = topicPartition.split("-").head val part = Integer.valueOf(topicPartition.split("-").last).intValue // setup brokers in zookeeper as owners of partitions for this test - CreateTopicCommand.createTopic(zkClient, topic, 3, 1, "1,1,1") + AdminUtils.createTopic(zkClient, topic, 3, 1) val logManager = server.getLogManager - val log = logManager.getOrCreateLog(topic, part) + val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig) val message = new Message(Integer.toString(42).getBytes()) for(i <- 0 until 20) log.append(new ByteBufferMessageSet(NoCompressionCodec, message)) log.flush() - time.sleep(20) - val now = time.milliseconds + val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs - val offsets = log.getOffsetsBefore(now, 10) + val offsets = server.apis.fetchOffsets(logManager, TopicAndPartition(topic, part), now, 10) assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), offsets) waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), 1000) @@ -169,21 +169,21 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testGetOffsetsBeforeEarliestTime() { - val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(3) + val topicPartition = "kafka-" + random.nextInt(3) val topic = topicPartition.split("-").head val part = Integer.valueOf(topicPartition.split("-").last).intValue // setup brokers in zookeeper as owners of partitions for this test - CreateTopicCommand.createTopic(zkClient, topic, 3, 1, "1,1,1") + AdminUtils.createTopic(zkClient, topic, 3, 1) val logManager = server.getLogManager - val log = logManager.getOrCreateLog(topic, part) + val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig) val message = new Message(Integer.toString(42).getBytes()) for(i <- 0 until 20) log.append(new ByteBufferMessageSet(NoCompressionCodec, message)) log.flush() - val offsets = log.getOffsetsBefore(OffsetRequest.EarliestTime, 10) + val offsets = server.apis.fetchOffsets(logManager, TopicAndPartition(topic, part), OffsetRequest.EarliestTime, 10) assertEquals(Seq(0L), offsets) @@ -205,7 +205,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { props.put("enable.zookeeper", "false") props.put("num.partitions", "20") props.put("log.retention.hours", "10") - props.put("log.cleanup.interval.mins", "5") + props.put("log.retention.check.interval.ms", (5*1000*60).toString) props.put("log.segment.bytes", logSize.toString) props.put("zookeeper.connect", zkConnect.toString) props diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index db46247c48e2f..17a99f182f64c 100644 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -18,11 +18,13 @@ package kafka.server import org.scalatest.junit.JUnit3Suite import org.junit.Assert._ -import kafka.admin.CreateTopicCommand +import java.io.File +import kafka.admin.AdminUtils import kafka.utils.TestUtils._ import kafka.utils.IntEncoder import kafka.utils.{Utils, TestUtils} import kafka.zk.ZooKeeperTestHarness +import kafka.common._ import kafka.producer.{ProducerConfig, KeyedMessage, Producer} class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -30,6 +32,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) { override val replicaLagTimeMaxMs = 5000L override val replicaLagMaxMessages = 10L + override val replicaFetchWaitMaxMs = 1000 override val replicaFetchMinBytes = 20 }) val topic = "new-topic" @@ -44,14 +47,21 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { val message = "hello" var producer: Producer[Int, String] = null - var hwFile1: HighwaterMarkCheckpoint = new HighwaterMarkCheckpoint(configProps1.logDirs(0)) - var hwFile2: HighwaterMarkCheckpoint = new HighwaterMarkCheckpoint(configProps2.logDirs(0)) + var hwFile1: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps1.logDirs(0), ReplicaManager.HighWatermarkFilename)) + var hwFile2: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps2.logDirs(0), ReplicaManager.HighWatermarkFilename)) var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] val producerProps = getProducerConfig(TestUtils.getBrokerListStrFromConfigs(configs)) producerProps.put("key.serializer.class", classOf[IntEncoder].getName.toString) producerProps.put("request.required.acks", "-1") - + + override def tearDown() { + super.tearDown() + for(server <- servers) { + server.shutdown() + Utils.rm(server.config.logDirs(0)) + } + } def testHWCheckpointNoFailuresSingleLogSegment { // start both servers @@ -62,7 +72,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 - CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(_.brokerId).mkString(":")) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) // wait until leader is elected var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) @@ -80,11 +90,10 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { servers.foreach(server => server.replicaManager.checkpointHighWatermarks()) producer.close() - val leaderHW = hwFile1.read(topic, 0) + val leaderHW = hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L) assertEquals(numMessages, leaderHW) - val followerHW = hwFile2.read(topic, 0) + val followerHW = hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L) assertEquals(numMessages, followerHW) - servers.foreach(server => { server.shutdown(); Utils.rm(server.config.logDirs(0))}) } def testHWCheckpointWithFailuresSingleLogSegment { @@ -96,7 +105,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 - CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(_.brokerId).mkString(":")) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) // wait until leader is elected var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) @@ -104,7 +113,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // NOTE: this is to avoid transient test failures assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1)) - assertEquals(0L, hwFile1.read(topic, 0)) + assertEquals(0L, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) sendMessages(1) Thread.sleep(1000) @@ -112,7 +121,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // kill the server hosting the preferred replica server1.shutdown() - assertEquals(hw, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) // check if leader moves to the other server leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader) @@ -125,10 +134,10 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue("Leader must remain on broker 1, in case of zookeeper session expiration it can move to broker 0", leader.isDefined && (leader.get == 0 || leader.get == 1)) - assertEquals(hw, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) // since server 2 was never shut down, the hw value of 30 is probably not checkpointed to disk yet server2.shutdown() - assertEquals(hw, hwFile2.read(topic, 0)) + assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) server2.startup() leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader) @@ -144,9 +153,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // shutdown the servers to allow the hw to be checkpointed servers.foreach(server => server.shutdown()) producer.close() - assertEquals(hw, hwFile1.read(topic, 0)) - assertEquals(hw, hwFile2.read(topic, 0)) - servers.foreach(server => Utils.rm(server.config.logDirs)) + assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) + assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) } def testHWCheckpointNoFailuresMultipleLogSegments { @@ -155,13 +163,13 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { server2 = TestUtils.createServer(configs.last) servers ++= List(server1, server2) - hwFile1 = new HighwaterMarkCheckpoint(server1.config.logDirs(0)) - hwFile2 = new HighwaterMarkCheckpoint(server2.config.logDirs(0)) + hwFile1 = new OffsetCheckpoint(new File(server1.config.logDirs(0), ReplicaManager.HighWatermarkFilename)) + hwFile2 = new OffsetCheckpoint(new File(server2.config.logDirs(0), ReplicaManager.HighWatermarkFilename)) producer = new Producer[Int, String](new ProducerConfig(producerProps)) // create topic with 1 partition, 2 replicas, one on each broker - CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(_.brokerId).mkString(":")) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) // wait until leader is elected var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) @@ -176,11 +184,10 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // shutdown the servers to allow the hw to be checkpointed servers.foreach(server => server.shutdown()) producer.close() - val leaderHW = hwFile1.read(topic, 0) + val leaderHW = hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L) assertEquals(hw, leaderHW) - val followerHW = hwFile2.read(topic, 0) + val followerHW = hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L) assertEquals(hw, followerHW) - servers.foreach(server => Utils.rm(server.config.logDirs)) } def testHWCheckpointWithFailuresMultipleLogSegments { @@ -189,13 +196,13 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { server2 = TestUtils.createServer(configs.last) servers ++= List(server1, server2) - hwFile1 = new HighwaterMarkCheckpoint(server1.config.logDirs(0)) - hwFile2 = new HighwaterMarkCheckpoint(server2.config.logDirs(0)) + hwFile1 = new OffsetCheckpoint(new File(server1.config.logDirs(0), ReplicaManager.HighWatermarkFilename)) + hwFile2 = new OffsetCheckpoint(new File(server2.config.logDirs(0), ReplicaManager.HighWatermarkFilename)) producer = new Producer[Int, String](new ProducerConfig(producerProps)) // create topic with 1 partition, 2 replicas, one on each broker - CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(_.brokerId).mkString(":")) + 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) @@ -212,21 +219,21 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // kill the server hosting the preferred replica server1.shutdown() server2.shutdown() - assertEquals(hw, hwFile1.read(topic, 0)) - assertEquals(hw, hwFile2.read(topic, 0)) + assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) + assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) server2.startup() // check if leader moves to the other server leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader) assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1)) - assertEquals(hw, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) // bring the preferred replica back server1.startup() - assertEquals(hw, hwFile1.read(topic, 0)) - assertEquals(hw, hwFile2.read(topic, 0)) + assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) + assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) sendMessages(2) hw += 2 @@ -237,9 +244,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // shutdown the servers to allow the hw to be checkpointed servers.foreach(server => server.shutdown()) producer.close() - assertEquals(hw, hwFile1.read(topic, 0)) - assertEquals(hw, hwFile2.read(topic, 0)) - servers.foreach(server => Utils.rm(server.config.logDirs)) + assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) + assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) } private def sendMessages(n: Int = 1) { diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala new file mode 100644 index 0000000000000..c0475d07a778f --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -0,0 +1,183 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.io.File +import kafka.utils._ +import junit.framework.Assert._ +import java.util.Properties +import kafka.consumer.SimpleConsumer +import org.junit.{After, Before, Test} +import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} +import kafka.zk.ZooKeeperTestHarness +import org.scalatest.junit.JUnit3Suite +import kafka.api.{OffsetCommitRequest, OffsetFetchRequest} +import kafka.utils.TestUtils._ +import kafka.common.{ErrorMapping, TopicAndPartition, OffsetMetadataAndError} +import scala.util.Random + +class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { + val random: Random = new Random() + var logDir: File = null + var topicLogDir: File = null + var server: KafkaServer = null + var logSize: Int = 100 + val brokerPort: Int = 9099 + var simpleConsumer: SimpleConsumer = null + var time: Time = new MockTime() + + @Before + override def setUp() { + super.setUp() + val config: Properties = createBrokerConfig(1, brokerPort) + val logDirPath = config.getProperty("log.dir") + logDir = new File(logDirPath) + time = new MockTime() + server = TestUtils.createServer(new KafkaConfig(config), time) + simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024, "test-client") + } + + @After + override def tearDown() { + simpleConsumer.close + server.shutdown + Utils.rm(logDir) + super.tearDown() + } + + @Test + def testUpdateOffsets() { + val topic = "topic" + + // Commit an offset + val topicAndPartition = TopicAndPartition(topic, 0) + val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError(offset=42L))) + val commitResponse = simpleConsumer.commitOffsets(commitRequest) + + assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get) + + // Fetch it and verify + val fetchRequest = OffsetFetchRequest("test-group", Seq(topicAndPartition)) + val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) + + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(topicAndPartition).get.error) + //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(topicAndPartition).get.metadata) + assertEquals(42L, fetchResponse.requestInfo.get(topicAndPartition).get.offset) + + // Commit a new offset + val commitRequest1 = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( + offset=100L, + metadata="some metadata" + ))) + val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) + + assertEquals(ErrorMapping.NoError, commitResponse1.requestInfo.get(topicAndPartition).get) + + // Fetch it and verify + val fetchRequest1 = OffsetFetchRequest("test-group", Seq(topicAndPartition)) + val fetchResponse1 = simpleConsumer.fetchOffsets(fetchRequest1) + + assertEquals(ErrorMapping.NoError, fetchResponse1.requestInfo.get(topicAndPartition).get.error) + //assertEquals("some metadata", fetchResponse1.requestInfo.get(topicAndPartition).get.metadata) + assertEquals(100L, fetchResponse1.requestInfo.get(topicAndPartition).get.offset) + + } + + @Test + def testCommitAndFetchOffsets() { + val topic1 = "topic-1" + val topic2 = "topic-2" + val topic3 = "topic-3" + val topic4 = "topic-4" + + val commitRequest = OffsetCommitRequest("test-group", Map( + TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(offset=42L, metadata="metadata one"), + TopicAndPartition(topic2, 0) -> OffsetMetadataAndError(offset=43L, metadata="metadata two"), + TopicAndPartition(topic3, 0) -> OffsetMetadataAndError(offset=44L, metadata="metadata three"), + TopicAndPartition(topic2, 1) -> OffsetMetadataAndError(offset=45L) + )) + val commitResponse = simpleConsumer.commitOffsets(commitRequest) + assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get) + + val fetchRequest = OffsetFetchRequest("test-group", Seq( + TopicAndPartition(topic1, 0), + TopicAndPartition(topic2, 0), + TopicAndPartition(topic3, 0), + TopicAndPartition(topic2, 1), + TopicAndPartition(topic3, 1), // An unknown partition + TopicAndPartition(topic4, 0) // An unknown topic + )) + val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) + + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.error) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.error) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) + assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) + assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) + + //assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata) + //assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata) + //assertEquals("metadata three", fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.metadata) + //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) + //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) + //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) + + assertEquals(42L, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.offset) + assertEquals(43L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.offset) + assertEquals(44L, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.offset) + assertEquals(45L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.offset) + assertEquals(OffsetMetadataAndError.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) + assertEquals(OffsetMetadataAndError.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) + } + + @Test + def testLargeMetadataPayload() { + val topicAndPartition = TopicAndPartition("large-metadata", 0) + val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( + offset=42L, + metadata=random.nextString(server.config.offsetMetadataMaxSize) + ))) + val commitResponse = simpleConsumer.commitOffsets(commitRequest) + + assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get) + + val commitRequest1 = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( + offset=42L, + metadata=random.nextString(server.config.offsetMetadataMaxSize + 1) + ))) + val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) + + assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.requestInfo.get(topicAndPartition).get) + + } + + @Test + def testNullMetadata() { + val topicAndPartition = TopicAndPartition("null-metadata", 0) + val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( + offset=42L, + metadata=null + ))) + val commitResponse = simpleConsumer.commitOffsets(commitRequest) + assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get) + } +} diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala index d0e35909f05ba..dd85c71816715 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala @@ -22,9 +22,10 @@ import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import kafka.producer.KeyedMessage import kafka.serializer.StringEncoder -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import kafka.utils.TestUtils import junit.framework.Assert._ +import kafka.common._ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { val props = createBrokerConfigs(2) @@ -50,7 +51,7 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { // create a topic and partition and await leadership for (topic <- List(topic1,topic2)) { - CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(c => c.brokerId).mkString(":")) + AdminUtils.createTopic(zkClient, topic, 1, 2) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) } @@ -65,9 +66,10 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { def logsMatch(): Boolean = { var result = true for (topic <- List(topic1, topic2)) { - val expectedOffset = brokers.head.getLogManager().getLog(topic, partition).get.logEndOffset + val topicAndPart = TopicAndPartition(topic, partition) + val expectedOffset = brokers.head.getLogManager().getLog(topicAndPart).get.logEndOffset result = result && expectedOffset > 0 && brokers.foldLeft(true) { (total, item) => total && - (expectedOffset == item.getLogManager().getLog(topic, partition).get.logEndOffset) } + (expectedOffset == item.getLogManager().getLog(topicAndPart).get.logEndOffset) } } result } diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 947e795345104..20fe93e623319 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -26,7 +26,7 @@ import kafka.zk.ZooKeeperTestHarness import kafka.producer._ import kafka.utils.IntEncoder import kafka.utils.TestUtils._ -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import kafka.api.FetchRequestBuilder import kafka.utils.{TestUtils, Utils} @@ -49,17 +49,18 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { var producer = new Producer[Int, String](new ProducerConfig(producerConfig)) // create topic - CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0") + AdminUtils.createTopic(zkClient, topic, 1, 1) TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) // send some messages producer.send(sent1.map(m => new KeyedMessage[Int, String](topic, 0, m)):_*) - // do a clean shutdown and check that the clean shudown file is written out + // do a clean shutdown and check that offset checkpoint file exists server.shutdown() for(logDir <- config.logDirs) { - val cleanShutDownFile = new File(logDir, server.logManager.CleanShutdownFile) - assertTrue(cleanShutDownFile.exists) + val OffsetCheckpointFile = new File(logDir, server.logManager.RecoveryPointCheckpointFile) + assertTrue(OffsetCheckpointFile.exists) + assertTrue(OffsetCheckpointFile.length() > 0) } producer.close() diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 23a8cb5108f74..03e6266ffdad5 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -34,13 +34,14 @@ class SimpleFetchTest extends JUnit3Suite { val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) { override val replicaLagTimeMaxMs = 100L + override val replicaFetchWaitMaxMs = 100 override val replicaLagMaxMessages = 10L }) val topic = "foo" val partitionId = 0 /** - * The scenario for this test is that there is one topic, "test-topic", on broker "0" that has + * The scenario for this test is that there is one topic, "test-topic", one broker "0" that has * one partition with one follower replica on broker "1". The leader replica on "0" * has HW of "5" and LEO of "20". The follower on broker "1" has a local replica * with a HW matching the leader's ("5") and LEO of "15", meaning it's not in-sync @@ -64,12 +65,12 @@ class SimpleFetchTest extends JUnit3Suite { val log = EasyMock.createMock(classOf[kafka.log.Log]) EasyMock.expect(log.logEndOffset).andReturn(leo).anyTimes() + EasyMock.expect(log) EasyMock.expect(log.read(0, fetchSize, Some(hw))).andReturn(new ByteBufferMessageSet(messages)) EasyMock.replay(log) val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(topic, partitionId)).andReturn(Some(log)).anyTimes() - EasyMock.expect(logManager.config).andReturn(configs.head).anyTimes() + EasyMock.expect(logManager.getLog(TopicAndPartition(topic, partitionId))).andReturn(Some(log)).anyTimes() EasyMock.replay(logManager) val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) @@ -92,7 +93,7 @@ class SimpleFetchTest extends JUnit3Suite { // start a request channel with 2 processors and a queue size of 5 (this is more or less arbitrary) // don't provide replica or leader callbacks since they will not be tested here val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, controller) + val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) // This request (from a follower) wants to read up to 2*HW but should only get back up to HW bytes into the log val goodFetch = new FetchRequestBuilder() @@ -106,33 +107,6 @@ class SimpleFetchTest extends JUnit3Suite { // make sure the log only reads bytes between 0->HW (5) EasyMock.verify(log) - - // Test offset request from non-replica - val topicAndPartition = TopicAndPartition(topic, partition.partitionId) - val offsetRequest = OffsetRequest( - Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1))) - val offsetRequestBB = TestUtils.createRequestByteBuffer(offsetRequest) - - EasyMock.reset(logManager) - EasyMock.reset(replicaManager) - - EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(logManager.getOffsets(topicAndPartition, OffsetRequest.LatestTime, 1)).andReturn(Seq(leo)) - - EasyMock.replay(replicaManager) - EasyMock.replay(logManager) - - apis.handleOffsetRequest(new RequestChannel.Request(processor = 0, - requestKey = 5, - buffer = offsetRequestBB, - startTimeMs = 1)) - val offsetResponseBuffer = requestChannel.receiveResponse(0).responseSend.asInstanceOf[BoundedByteBufferSend].buffer - val offsetResponse = OffsetResponse.readFrom(offsetResponseBuffer) - EasyMock.verify(replicaManager) - EasyMock.verify(logManager) - assertEquals(1, offsetResponse.partitionErrorAndOffsets(topicAndPartition).offsets.size) - assertEquals(hw.toLong, offsetResponse.partitionErrorAndOffsets(topicAndPartition).offsets.head) } /** @@ -165,8 +139,7 @@ class SimpleFetchTest extends JUnit3Suite { EasyMock.replay(log) val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(topic, 0)).andReturn(Some(log)).anyTimes() - EasyMock.expect(logManager.config).andReturn(configs.head).anyTimes() + EasyMock.expect(logManager.getLog(TopicAndPartition(topic, 0))).andReturn(Some(log)).anyTimes() EasyMock.replay(logManager) val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) @@ -189,7 +162,7 @@ class SimpleFetchTest extends JUnit3Suite { val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, controller) + val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) /** * This fetch, coming from a replica, requests all data at offset "15". Because the request is coming @@ -210,34 +183,6 @@ class SimpleFetchTest extends JUnit3Suite { * an offset of 15 */ EasyMock.verify(log) - - // Test offset request from replica - val topicAndPartition = TopicAndPartition(topic, partition.partitionId) - val offsetRequest = OffsetRequest( - Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1)), - replicaId = followerReplicaId) - val offsetRequestBB = TestUtils.createRequestByteBuffer(offsetRequest) - - EasyMock.reset(logManager) - EasyMock.reset(replicaManager) - - EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(logManager.getOffsets(topicAndPartition, OffsetRequest.LatestTime, 1)).andReturn(Seq(leo)) - - EasyMock.replay(replicaManager) - EasyMock.replay(logManager) - - apis.handleOffsetRequest(new RequestChannel.Request(processor = 1, - requestKey = 5, - buffer = offsetRequestBB, - startTimeMs = 1)) - val offsetResponseBuffer = requestChannel.receiveResponse(1).responseSend.asInstanceOf[BoundedByteBufferSend].buffer - val offsetResponse = OffsetResponse.readFrom(offsetResponseBuffer) - EasyMock.verify(replicaManager) - EasyMock.verify(logManager) - assertEquals(1, offsetResponse.partitionErrorAndOffsets(topicAndPartition).offsets.size) - assertEquals(leo.toLong, offsetResponse.partitionErrorAndOffsets(topicAndPartition).offsets.head) } private def getPartitionWithAllReplicasInISR(topic: String, partitionId: Int, time: Time, leaderId: Int, diff --git a/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala new file mode 100644 index 0000000000000..02054c61b7294 --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala @@ -0,0 +1,41 @@ +package kafka.utils + +import junit.framework.Assert._ +import org.scalatest.Assertions +import org.junit.{Test, After, Before} + +class IteratorTemplateTest extends Assertions { + + val lst = (0 until 10).toSeq + val iterator = new IteratorTemplate[Int]() { + var i = 0 + override def makeNext() = { + if(i >= lst.size) { + allDone() + } else { + val item = lst(i) + i += 1 + item + } + } + } + + @Test + def testIterator() { + for(i <- 0 until 10) { + assertEquals("We should have an item to read.", true, iterator.hasNext) + assertEquals("Checking again shouldn't change anything.", true, iterator.hasNext) + assertEquals("Peeking at the item should show the right thing.", i, iterator.peek) + assertEquals("Peeking again shouldn't change anything", i, iterator.peek) + assertEquals("Getting the item should give the right thing.", i, iterator.next) + } + assertEquals("All gone!", false, iterator.hasNext) + intercept[NoSuchElementException] { + iterator.peek + } + intercept[NoSuchElementException] { + iterator.next + } + } + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/utils/JsonTest.scala b/core/src/test/scala/unit/kafka/utils/JsonTest.scala new file mode 100644 index 0000000000000..4482dab56bea3 --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/JsonTest.scala @@ -0,0 +1,27 @@ +package kafka.utils + +import junit.framework.Assert._ +import org.junit.{Test, After, Before} + +class JsonTest { + + @Test + def testJsonEncoding() { + assertEquals("null", Json.encode(null)) + assertEquals("1", Json.encode(1)) + assertEquals("1", Json.encode(1L)) + assertEquals("1", Json.encode(1.toByte)) + assertEquals("1", Json.encode(1.toShort)) + assertEquals("1.0", Json.encode(1.0)) + assertEquals("\"str\"", Json.encode("str")) + assertEquals("true", Json.encode(true)) + assertEquals("false", Json.encode(false)) + assertEquals("[]", Json.encode(Seq())) + assertEquals("[1,2,3]", Json.encode(Seq(1,2,3))) + assertEquals("[1,\"2\",[3]]", Json.encode(Seq(1,"2",Seq(3)))) + assertEquals("{}", Json.encode(Map())) + assertEquals("{\"a\":1,\"b\":2}", Json.encode(Map("a" -> 1, "b" -> 2))) + assertEquals("{\"a\":[1,2],\"c\":[3,4]}", Json.encode(Map("a" -> Seq(1,2), "c" -> Seq(3,4)))) + } + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala new file mode 100644 index 0000000000000..d5896ed4d3b73 --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala @@ -0,0 +1,88 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.utils + +import scala.collection.mutable.PriorityQueue +import java.util.concurrent.TimeUnit + +/** + * A mock scheduler that executes tasks synchronously using a mock time instance. Tasks are executed synchronously when + * the time is advanced. This class is meant to be used in conjunction with MockTime. + * + * Example usage + * + * val time = new MockTime + * time.scheduler.schedule("a task", println("hello world: " + time.milliseconds), delay = 1000) + * time.sleep(1001) // this should cause our scheduled task to fire + * + * + * Incrementing the time to the exact next execution time of a task will result in that task executing (it as if execution itself takes no time). + */ +class MockScheduler(val time: Time) extends Scheduler { + + /* a priority queue of tasks ordered by next execution time */ + var tasks = new PriorityQueue[MockTask]() + + def startup() {} + + def shutdown() { + this synchronized { + tasks.clear() + } + } + + /** + * Check for any tasks that need to execute. Since this is a mock scheduler this check only occurs + * when this method is called and the execution happens synchronously in the calling thread. + * If you are using the scheduler associated with a MockTime instance this call be triggered automatically. + */ + def tick() { + this synchronized { + val now = time.milliseconds + while(!tasks.isEmpty && tasks.head.nextExecution <= now) { + /* pop and execute the task with the lowest next execution time */ + val curr = tasks.dequeue + curr.fun() + /* if the task is periodic, reschedule it and re-enqueue */ + if(curr.periodic) { + curr.nextExecution += curr.period + this.tasks += curr + } + } + } + } + + def schedule(name: String, fun: ()=>Unit, delay: Long = 0, period: Long = -1, unit: TimeUnit = TimeUnit.MILLISECONDS) { + this synchronized { + tasks += MockTask(name, fun, time.milliseconds + delay, period = period) + tick() + } + } + +} + +case class MockTask(val name: String, val fun: () => Unit, var nextExecution: Long, val period: Long) extends Ordered[MockTask] { + def periodic = period >= 0 + def compare(t: MockTask): Int = { + if(t.nextExecution == nextExecution) + return 0 + else if (t.nextExecution < nextExecution) + return -1 + else + return 1 + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/Range.scala b/core/src/test/scala/unit/kafka/utils/MockTime.scala similarity index 52% rename from core/src/main/scala/kafka/utils/Range.scala rename to core/src/test/scala/unit/kafka/utils/MockTime.scala index ca7d699ec4139..ee65748afefd5 100644 --- a/core/src/main/scala/kafka/utils/Range.scala +++ b/core/src/test/scala/unit/kafka/utils/MockTime.scala @@ -17,26 +17,33 @@ package kafka.utils +import java.util.concurrent._ /** - * A generic range value with a start and end + * A class used for unit testing things which depend on the Time interface. + * + * This class never manually advances the clock, it only does so when you call + * sleep(ms) + * + * It also comes with an associated scheduler instance for managing background tasks in + * a deterministic way. */ -trait Range { - /** The first index in the range */ - def start: Long - /** The total number of indexes in the range */ - def size: Long - /** Return true iff the range is empty */ - def isEmpty: Boolean = size == 0 +class MockTime(@volatile private var currentMs: Long) extends Time { + + val scheduler = new MockScheduler(this) + + def this() = this(System.currentTimeMillis) + + def milliseconds: Long = currentMs + + def nanoseconds: Long = + TimeUnit.NANOSECONDS.convert(currentMs, TimeUnit.MILLISECONDS) - /** if value is in range */ - def contains(value: Long): Boolean = { - if( (size == 0 && value == start) || - (size > 0 && value >= start && value <= start + size - 1) ) - return true - else - return false + def sleep(ms: Long) { + this.currentMs += ms + scheduler.tick() } - override def toString() = "(start=" + start + ", size=" + size + ")" + override def toString() = "MockTime(%d)".format(milliseconds) + } diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala new file mode 100644 index 0000000000000..b364ac2d6d623 --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.utils + +import junit.framework.Assert._ +import java.util.concurrent.atomic._ +import org.junit.{Test, After, Before} +import kafka.utils.TestUtils.retry + +class SchedulerTest { + + val scheduler = new KafkaScheduler(1) + val mockTime = new MockTime + val counter1 = new AtomicInteger(0) + val counter2 = new AtomicInteger(0) + + @Before + def setup() { + scheduler.startup() + } + + @After + def teardown() { + scheduler.shutdown() + } + + @Test + def testMockSchedulerNonPeriodicTask() { + mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1) + mockTime.scheduler.schedule("test2", counter2.getAndIncrement, delay=100) + assertEquals("Counter1 should not be incremented prior to task running.", 0, counter1.get) + assertEquals("Counter2 should not be incremented prior to task running.", 0, counter2.get) + mockTime.sleep(1) + assertEquals("Counter1 should be incremented", 1, counter1.get) + assertEquals("Counter2 should not be incremented", 0, counter2.get) + mockTime.sleep(100000) + assertEquals("More sleeping should not result in more incrementing on counter1.", 1, counter1.get) + assertEquals("Counter2 should now be incremented.", 1, counter2.get) + } + + @Test + def testMockSchedulerPeriodicTask() { + mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1, period=1) + mockTime.scheduler.schedule("test2", counter2.getAndIncrement, delay=100, period=100) + assertEquals("Counter1 should not be incremented prior to task running.", 0, counter1.get) + assertEquals("Counter2 should not be incremented prior to task running.", 0, counter2.get) + mockTime.sleep(1) + assertEquals("Counter1 should be incremented", 1, counter1.get) + assertEquals("Counter2 should not be incremented", 0, counter2.get) + mockTime.sleep(100) + assertEquals("Counter1 should be incremented 101 times", 101, counter1.get) + assertEquals("Counter2 should not be incremented once", 1, counter2.get) + } + + @Test + def testReentrantTaskInMockScheduler() { + mockTime.scheduler.schedule("test1", () => mockTime.scheduler.schedule("test2", counter2.getAndIncrement, delay=0), delay=1) + mockTime.sleep(1) + assertEquals(1, counter2.get) + } + + @Test + def testNonPeriodicTask() { + scheduler.schedule("test", counter1.getAndIncrement, delay = 0) + retry(30000) { + assertEquals(counter1.get, 1) + } + Thread.sleep(5) + assertEquals("Should only run once", 1, counter1.get) + } + + @Test + def testPeriodicTask() { + scheduler.schedule("test", counter1.getAndIncrement, delay = 0, period = 5) + retry(30000){ + assertTrue("Should count to 20", counter1.get >= 20) + } + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index ee591d092b95c..98f9066eb3621 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -23,6 +23,7 @@ import java.nio._ import java.nio.channels._ import java.util.Random import java.util.Properties +import junit.framework.AssertionFailedError import junit.framework.Assert._ import kafka.server._ import kafka.producer._ @@ -44,6 +45,8 @@ import junit.framework.Assert * Utility functions to help with testing */ object TestUtils extends Logging { + + val IoTmpDir = System.getProperty("java.io.tmpdir") val Letters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz" val Digits = "0123456789" @@ -75,8 +78,7 @@ object TestUtils extends Logging { * Create a temporary directory */ def tempDir(): File = { - val ioDir = System.getProperty("java.io.tmpdir") - val f = new File(ioDir, "kafka-" + random.nextInt(1000000)) + val f = new File(IoTmpDir, "kafka-" + random.nextInt(1000000)) f.mkdirs() f.deleteOnExit() f @@ -122,13 +124,12 @@ object TestUtils extends Logging { /** * Create a test config for the given node id */ - def createBrokerConfig(nodeId: Int, port: Int): Properties = { + def createBrokerConfig(nodeId: Int, port: Int = choosePort()): Properties = { val props = new Properties props.put("broker.id", nodeId.toString) props.put("host.name", "localhost") props.put("port", port.toString) props.put("log.dir", TestUtils.tempDir().getAbsolutePath) - props.put("log.flush.interval.messages", "1") props.put("zookeeper.connect", TestZKUtils.zookeeperConnect) props.put("replica.socket.timeout.ms", "1500") props @@ -157,8 +158,8 @@ object TestUtils extends Logging { * Wrap the message in a message set * @param payload The bytes of the message */ - def singleMessageSet(payload: Array[Byte], codec: CompressionCodec = NoCompressionCodec) = - new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload)) + def singleMessageSet(payload: Array[Byte], codec: CompressionCodec = NoCompressionCodec, key: Array[Byte] = null) = + new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key)) /** * Generate an array of random bytes @@ -344,13 +345,13 @@ 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, 6000, jmxPort = -1)) + val brokers = ids.map(id => new Broker(id, "localhost", 6667, 0)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.host, b.port, b.rack, 6000, jmxPort = -1)) brokers } def deleteBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { - val brokers = ids.map(id => new Broker(id, "localhost", 6667)) + val brokers = ids.map(id => new Broker(id, "localhost", 6667, 0)) brokers.foreach(b => ZkUtils.deletePath(zkClient, ZkUtils.BrokerIdsPath + "/" + b)) brokers } @@ -450,17 +451,23 @@ object TestUtils extends Logging { * Execute the given block. If it throws an assert error, retry. Repeat * until no error is thrown or the time limit ellapses */ - def retry(waitTime: Long, block: () => Unit) { + def retry(maxWaitMs: Long)(block: => Unit) { + var wait = 1L val startTime = System.currentTimeMillis() while(true) { try { - block() + block + return } catch { - case e: AssertionError => - if(System.currentTimeMillis - startTime > waitTime) + case e: AssertionFailedError => + val ellapsed = System.currentTimeMillis - startTime + if(ellapsed > maxWaitMs) { throw e - else - Thread.sleep(100) + } else { + info("Attempt failed, sleeping for " + wait + ", and then retrying.") + Thread.sleep(wait) + wait += math.min(wait, 1000) + } } } } @@ -508,28 +515,43 @@ object TestUtils extends Logging { servers.foldLeft(true)(_ && _.apis.leaderCache.keySet.contains(TopicAndPartition(topic, partition))), timeout)) } + def writeNonsenseToFile(fileName: File, position: Long, size: Int) { + val file = new RandomAccessFile(fileName, "rw") + file.seek(position) + for(i <- 0 until size) + file.writeByte(random.nextInt(255)) + file.close() + } + + def appendNonsenseToFile(fileName: File, size: Int) { + val file = new FileOutputStream(fileName, true) + for(i <- 0 until size) + file.write(random.nextInt(255)) + file.close() + } + } object TestZKUtils { - val zookeeperConnect = "127.0.0.1:2182" + val zookeeperConnect = "127.0.0.1:" + TestUtils.choosePort() } class IntEncoder(props: VerifiableProperties = null) extends Encoder[Int] { override def toBytes(n: Int) = n.toString.getBytes } -class StaticPartitioner(props: VerifiableProperties = null) extends Partitioner[String] { - def partition(data: String, numPartitions: Int): Int = { - (data.length % numPartitions) +class StaticPartitioner(props: VerifiableProperties = null) extends Partitioner{ + def partition(data: Any, numPartitions: Int): Int = { + (data.asInstanceOf[String].length % numPartitions) } } -class HashPartitioner(props: VerifiableProperties = null) extends Partitioner[String] { - def partition(data: String, numPartitions: Int): Int = { +class HashPartitioner(props: VerifiableProperties = null) extends Partitioner { + def partition(data: Any, numPartitions: Int): Int = { (data.hashCode % numPartitions) } } -class FixedValuePartitioner(props: VerifiableProperties = null) extends Partitioner[Int] { - def partition(data: Int, numPartitions: Int): Int = data +class FixedValuePartitioner(props: VerifiableProperties = null) extends Partitioner { + def partition(data: Any, numPartitions: Int): Int = data.asInstanceOf[Int] } diff --git a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala index 6b21554feee09..920f318c85a74 100644 --- a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala @@ -18,11 +18,14 @@ package kafka.utils import java.util.Arrays +import java.util.concurrent.locks.ReentrantLock import java.nio.ByteBuffer +import java.io._ import org.apache.log4j.Logger import org.scalatest.junit.JUnitSuite import org.junit.Assert._ import kafka.common.KafkaException +import kafka.utils.Utils.inLock import org.junit.Test @@ -61,6 +64,25 @@ class UtilsTest extends JUnitSuite { assertTrue(Arrays.equals(bytes, Utils.readBytes(ByteBuffer.wrap(bytes)))) } } + + @Test + def testReplaceSuffix() { + assertEquals("blah.foo.text", Utils.replaceSuffix("blah.foo.txt", ".txt", ".text")) + assertEquals("blah.foo", Utils.replaceSuffix("blah.foo.txt", ".txt", "")) + assertEquals("txt.txt", Utils.replaceSuffix("txt.txt.txt", ".txt", "")) + assertEquals("foo.txt", Utils.replaceSuffix("foo", "", ".txt")) + } + + @Test + def testReadInt() { + val values = Array(0, 1, -1, Byte.MaxValue, Short.MaxValue, 2 * Short.MaxValue, Int.MaxValue/2, Int.MinValue/2, Int.MaxValue, Int.MinValue, Int.MaxValue) + val buffer = ByteBuffer.allocate(4 * values.size) + for(i <- 0 until values.length) { + buffer.putInt(i*4, values(i)) + assertEquals("Written value should match read value.", values(i), Utils.readInt(buffer.array, i*4)) + } + + } @Test def testCsvList() { @@ -74,4 +96,16 @@ class UtilsTest extends JUnitSuite { assertTrue(emptyStringList.equals(emptyListFromNullString)) assertTrue(emptyStringList.equals(emptyList)) } + + @Test + def testInLock() { + val lock = new ReentrantLock() + val result = inLock(lock) { + assertTrue("Should be in lock", lock.isHeldByCurrentThread) + 1 + 1 + } + assertEquals(2, result) + assertFalse("Should be unlocked", lock.isLocked) + + } } diff --git a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala index 6763357f02727..d883bdeee1f58 100644 --- a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala +++ b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala @@ -33,7 +33,8 @@ class EmbeddedZookeeper(val connectString: String) { factory.startup(zookeeper) def shutdown() { - factory.shutdown() + Utils.swallow(zookeeper.shutdown()) + Utils.swallow(factory.shutdown()) Utils.rm(logDir) Utils.rm(snapshotDir) } diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 63e528f6e710f..4e25b926d32e4 100644 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -19,7 +19,7 @@ package kafka.zk import org.scalatest.junit.JUnit3Suite import org.I0Itec.zkclient.ZkClient -import kafka.utils.{ZKStringSerializer, TestZKUtils} +import kafka.utils.{ZKStringSerializer, TestZKUtils, Utils} trait ZooKeeperTestHarness extends JUnit3Suite { val zkConnect: String = TestZKUtils.zookeeperConnect @@ -36,8 +36,8 @@ trait ZooKeeperTestHarness extends JUnit3Suite { override def tearDown() { super.tearDown - zkClient.close() - zookeeper.shutdown() + Utils.swallow(zkClient.close()) + Utils.swallow(zookeeper.shutdown()) } } diff --git a/examples/README b/examples/README index d33f6c55dcd62..61de2868de29e 100644 --- a/examples/README +++ b/examples/README @@ -6,10 +6,10 @@ messages from Kafka server. In order to run demo from SBT: 1. Start Zookeeper and the Kafka server 2. ./sbt from top-level kafka directory - 3. Switch to the kafka java examples project -> project Kafka Java Examples + 3. Switch to the kafka java examples project -> project kafka-examples 4. execute run -> run - 5. For unlimited producer-consumer run, select option 1 - For simple consumer demo, select option 2 + 5. For simple consumer demo, select option 1 + For unlimited producer-consumer run, select option 2 To run the demo using scripts: diff --git a/examples/src/main/java/kafka/examples/Consumer.java b/examples/src/main/java/kafka/examples/Consumer.java index 63f099ade5a10..13135b954f307 100644 --- a/examples/src/main/java/kafka/examples/Consumer.java +++ b/examples/src/main/java/kafka/examples/Consumer.java @@ -25,7 +25,6 @@ import kafka.consumer.ConsumerIterator; import kafka.consumer.KafkaStream; import kafka.javaapi.consumer.ConsumerConnector; -import kafka.message.Message; public class Consumer extends Thread diff --git a/kafka-patch-review.py b/kafka-patch-review.py index f1d5192057755..dc6664d22a0b1 100644 --- a/kafka-patch-review.py +++ b/kafka-patch-review.py @@ -36,7 +36,27 @@ def main(): ts = time.time() st = datetime.datetime.fromtimestamp(ts).strftime('%Y-%m-%d_%H:%M:%S') patch_file=tempfile.gettempdir() + "/" + opt.jira + '_' + st + '.patch' - + + # first check if rebase is needed + git_branch_hash="git rev-parse " + opt.branch + p_now=os.popen(git_branch_hash) + branch_now=p_now.read() + p_now.close() + + git_common_ancestor="git merge-base " + opt.branch + " HEAD" + p_then=os.popen(git_common_ancestor) + branch_then=p_then.read() + p_then.close() + + if branch_now != branch_then: + print 'ERROR: Your current working branch is from an older version of ' + opt.branch + '. Please rebase first by using git pull --rebase' + sys.exit(1) + + git_configure_reviewboard="git config reviewboard.url https://reviews.apache.org" + print "Configuring reviewboard url to https://reviews.apache.org" + p=os.popen(git_configure_reviewboard) + p.close() + git_remote_update="git remote update" print "Updating your remote branches to pull the latest changes" p=os.popen(git_remote_update) @@ -90,12 +110,12 @@ def main(): comment="Created reviewboard " if not opt.reviewboard: - print 'Created a new reviewboard ',rb_url + print 'Created a new reviewboard ',rb_url, else: - print 'Updated reviewboard',opt.reviewboard + print 'Updated reviewboard' comment="Updated reviewboard " - comment = comment + rb_url + comment = comment + rb_url + ' against branch ' + opt.branch jira.add_comment(opt.jira, comment) if __name__ == '__main__': diff --git a/project/Build.scala b/project/Build.scala index 33bb76f4ebb89..098e874b8a140 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -44,7 +44,7 @@ object KafkaBuild extends Build { crossScalaVersions := Seq("2.8.0","2.8.2", "2.9.1", "2.9.2", "2.10.1"), excludeFilter in unmanagedSources <<= scalaVersion(v => if (v.startsWith("2.8")) "*_2.9+.scala" else "*_2.8.scala"), scalaVersion := "2.8.0", - version := "0.8.0-beta1", + version := "0.8.1", publishTo := Some("Apache Maven Repo" at "https://repository.apache.org/service/local/staging/deploy/maven2"), credentials += Credentials(Path.userHome / ".m2" / ".credentials"), buildNumber := System.getProperty("build.number", ""), @@ -53,6 +53,7 @@ object KafkaBuild extends Build { javacOptions in compile ++= Seq("-Xlint:unchecked", "-source", "1.5"), javacOptions in doc ++= Seq("-source", "1.5"), parallelExecution in Test := false, // Prevent tests from overrunning each other + publishArtifact in Test := true, libraryDependencies ++= Seq( "log4j" % "log4j" % "1.2.15" exclude("javax.jms", "jms"), "net.sf.jopt-simple" % "jopt-simple" % "3.2", @@ -70,7 +71,9 @@ object KafkaBuild extends Build { - + , + mappings in packageBin in Compile += file("LICENSE") -> "LICENSE", + mappings in packageBin in Compile += file("NOTICE") -> "NOTICE" ) val hadoopSettings = Seq( @@ -112,6 +115,8 @@ object KafkaBuild extends Build { val jarFiles = deps.files.filter(f => !products.files.contains(f) && f.getName.endsWith(".jar")) val destination = target / "RELEASE" / releaseName IO.copyFile(packageBin, destination / packageBin.getName) + IO.copyFile(file("LICENSE"), destination / "LICENSE") + IO.copyFile(file("NOTICE"), destination / "NOTICE") IO.copy(jarFiles.map { f => (f, destination / "libs" / f.getName) }) IO.copyDirectory(file("config"), destination / "config") IO.copyDirectory(file("bin"), destination / "bin") diff --git a/system_test/migration_tool_testsuite/0.7/bin/kafka-run-class.sh b/system_test/migration_tool_testsuite/0.7/bin/kafka-run-class.sh index 8ebe7a7f3eb4a..08eaea87f5ae0 100755 --- a/system_test/migration_tool_testsuite/0.7/bin/kafka-run-class.sh +++ b/system_test/migration_tool_testsuite/0.7/bin/kafka-run-class.sh @@ -66,8 +66,14 @@ done 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 + +# Log4j settings +if [ -z "$KAFKA_LOG4J_OPTS" ]; then + KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/config/log4j.properties" +fi + if [ -z "$KAFKA_OPTS" ]; then - KAFKA_OPTS="-Xmx512M -server -Dlog4j.configuration=file:$base_dir/config/log4j.properties" + KAFKA_OPTS="-Xmx512M -server $KAFKA_LOG4J_OPTS" fi if [ $JMX_PORT ]; then KAFKA_JMX_OPTS="$KAFKA_JMX_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT " diff --git a/system_test/migration_tool_testsuite/0.7/config/test-log4j.properties b/system_test/migration_tool_testsuite/0.7/config/test-log4j.properties new file mode 100644 index 0000000000000..a3ae33f20e4b7 --- /dev/null +++ b/system_test/migration_tool_testsuite/0.7/config/test-log4j.properties @@ -0,0 +1,68 @@ +# 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, 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 + +log4j.appender.kafkaAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.kafkaAppender.DatePattern='.'yyyy-MM-dd-HH +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=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=logs/kafka-request.log +log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.requestAppender.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=logs/controller.log +log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n + +# Turn on all our debugging info +#log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender +#log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender +log4j.logger.kafka.perf=DEBUG, kafkaAppender +log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG, kafkaAppender +#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG +log4j.logger.kafka=INFO, kafkaAppender + +log4j.logger.kafka.network.RequestChannel$=TRACE, requestAppender +log4j.additivity.kafka.network.RequestChannel$=false + +#log4j.logger.kafka.network.Processor=TRACE, requestAppender +#log4j.logger.kafka.server.KafkaApis=TRACE, requestAppender +#log4j.additivity.kafka.server.KafkaApis=false +log4j.logger.kafka.request.logger=TRACE, requestAppender +log4j.additivity.kafka.request.logger=false + +log4j.logger.kafka.controller=TRACE, controllerAppender +log4j.additivity.kafka.controller=false + +log4j.logger.state.change.logger=TRACE, stateChangeAppender +log4j.additivity.state.change.logger=false + + diff --git a/system_test/migration_tool_testsuite/config/server.properties b/system_test/migration_tool_testsuite/config/server.properties index 6ecbb71e44aa4..54144a28a0800 100644 --- a/system_test/migration_tool_testsuite/config/server.properties +++ b/system_test/migration_tool_testsuite/config/server.properties @@ -51,8 +51,9 @@ socket.request.max.bytes=104857600 # The directory under which to store log files log.dir=/tmp/kafka_server_logs -# The number of logical partitions per topic per server. More partitions allow greater parallelism -# for consumption, but also mean more files. +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. num.partitions=5 # Overrides for for the default given by num.partitions on a per-topic basis diff --git a/system_test/mirror_maker_testsuite/config/server.properties b/system_test/mirror_maker_testsuite/config/server.properties index 36dd68d786d64..c6284122e3dfa 100644 --- a/system_test/mirror_maker_testsuite/config/server.properties +++ b/system_test/mirror_maker_testsuite/config/server.properties @@ -51,8 +51,9 @@ socket.request.max.bytes=104857600 # The directory under which to store log files log.dir=/tmp/kafka_server_logs -# The number of logical partitions per topic per server. More partitions allow greater parallelism -# for consumption, but also mean more files. +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. num.partitions=5 # Overrides for for the default given by num.partitions on a per-topic basis diff --git a/system_test/replication_testsuite/config/server.properties b/system_test/replication_testsuite/config/server.properties index 36dd68d786d64..c6284122e3dfa 100644 --- a/system_test/replication_testsuite/config/server.properties +++ b/system_test/replication_testsuite/config/server.properties @@ -51,8 +51,9 @@ socket.request.max.bytes=104857600 # The directory under which to store log files log.dir=/tmp/kafka_server_logs -# The number of logical partitions per topic per server. More partitions allow greater parallelism -# for consumption, but also mean more files. +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. num.partitions=5 # Overrides for for the default given by num.partitions on a per-topic basis diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index aec31cbf38ca3..fb4a9c05bf6f3 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -1046,6 +1046,7 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk cmdList = ["ssh " + host, "'JAVA_HOME=" + javaHome, "JMX_PORT=" + jmxPort, + "KAFKA_LOG4J_OPTS=-Dlog4j.configuration=file:%s/config/test-log4j.properties" % kafkaHome, kafkaRunClassBin + " kafka.perf.ProducerPerformance", "--brokerinfo " + brokerInfoStr, "--initial-message-id " + str(initMsgId), @@ -1127,7 +1128,7 @@ def create_topic(systemTestEnv, testcaseEnv): zkHost = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "role", "zookeeper", "hostname") kafkaHome = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "entity_id", zkEntityId, "kafka_home") javaHome = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "entity_id", zkEntityId, "java_home") - createTopicBin = kafkaHome + "/bin/kafka-create-topic.sh" + createTopicBin = kafkaHome + "/bin/kafka-topics.sh --create" logger.debug("zkEntityId : " + zkEntityId, extra=d) logger.debug("createTopicBin : " + createTopicBin, extra=d) @@ -1154,8 +1155,8 @@ def create_topic(systemTestEnv, testcaseEnv): createTopicBin, " --topic " + topic, " --zookeeper " + zkConnectStr, - " --replica " + testcaseEnv.testcaseArgumentsDict["replica_factor"], - " --partition " + testcaseEnv.testcaseArgumentsDict["num_partition"] + " >> ", + " --replication-factor " + testcaseEnv.testcaseArgumentsDict["replica_factor"], + " --partitions " + testcaseEnv.testcaseArgumentsDict["num_partition"] + " >> ", testcaseBaseDir + "/logs/create_source_cluster_topic.log'"] cmdStr = " ".join(cmdList)