diff --git a/.gitignore b/.gitignore index 553a077d031a3..1f3ba7dbbf524 100644 --- a/.gitignore +++ b/.gitignore @@ -1,11 +1,13 @@ dist *classes target/ +build/ +.gradle/ lib_managed/ src_managed/ project/boot/ project/plugins/project/ -project/sbt_project_definition.iml +patch-process/* .idea .svn .classpath @@ -14,3 +16,16 @@ project/sbt_project_definition.iml .#* rat.out TAGS +*.iml +.project +.settings +.gradle +kafka.ipr +kafka.iws +.vagrant +Vagrantfile.local + +config/server-* +config/zookeeper-* +core/data/* +gradle/wrapper/* diff --git a/.rat-excludes b/.rat-excludes deleted file mode 100644 index 01d629817c8a4..0000000000000 --- a/.rat-excludes +++ /dev/null @@ -1,26 +0,0 @@ -.rat-excludes -rat.out -sbt -sbt.boot.lock -README* -.gitignore -.git -.svn -build.properties -target -src_managed -update.log -clients/target -core/target -contrib/target -project/plugins/target -project/build/target -*.iml -*.csproj -TODO -Makefile* -*.html -*.xml -*expected.out -*.kafka - diff --git a/LICENSE b/LICENSE index cb1800b0c39af..d645695673349 100644 --- a/LICENSE +++ b/LICENSE @@ -200,34 +200,3 @@ 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. - ------------------------------------------------------------------------ - -SBT LICENSE - -Copyright (c) 2008, 2009, 2010 Mark Harrah, Jason Zaugg -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions -are met: -1. Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. -3. The name of the author may not be used to endorse or promote products - derived from this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR -IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES -OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. -IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, -INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT -NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF -THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - ------------------------------------------------------------------------ \ No newline at end of file diff --git a/README-sbt.md b/README-sbt.md deleted file mode 100644 index 10b8d2523605e..0000000000000 --- a/README-sbt.md +++ /dev/null @@ -1,60 +0,0 @@ -# Apache Kafka # - -See our [web site](http://kafka.apache.org) for details on the project. - -## Building it ## -1. ./sbt update -2. ./sbt package -3. ./sbt assembly-package-dependency - -To build for a particular version of Scala (either 2.8.0, 2.8.2, 2.9.1, 2.9.2 or 2.10.1), change step 2 above to: -2. ./sbt "++2.8.0 package" - -To build for all supported versions of Scala, change step 2 above to: -2. ./sbt +package - -## Running it ## -Follow instuctions in http://kafka.apache.org/documentation.html#quickstart - -## Running unit tests ## - ./sbt test - -## Building a binary release zip or gzipped tar ball ## - ./sbt release-zip - ./sbt release-tar -The release file can be found inside ./target/RELEASE/. - -## Other Build Tips ## -Here are some useful sbt commands, to be executed at the sbt command prompt (./sbt). Prefixing with "++ " runs the -command for a specific Scala version, prefixing with "+" will perform the action for all versions of Scala, and no prefix -runs the command for the default (2.8.0) version of Scala. - - -tasks : Lists all the sbt commands and their descriptions -clean : Deletes all generated files (the target directory). -compile : Compile all the sub projects, but not create the jars -test : Run all unit tests in all sub projects -release-zip : Create all the jars, run unit tests and create a deployable release zip -release-tar : Create all the jars, run unit tests and create a deployable release gzipped tar tall -package: Creates jars for src, test, docs etc -projects : List all the sub projects -project sub_project_name : Switch to a particular sub-project. For example, to switch to the core kafka code, use "project core-kafka" - -The following commands can be run only on a particular sub project - -test-only package.test.TestName : Runs only the specified test in the current sub project -run : Provides options to run any of the classes that have a main method. For example, you can switch to project java-examples, and run the examples there by executing "project java-examples" followed by "run" - -For more details please see the [SBT documentation](https://github.com/harrah/xsbt/wiki) - -## Contribution ## - -Kafka is a new project, and we are interested in building the community; we would welcome any thoughts or [patches](https://issues.apache.org/jira/browse/KAFKA). You can reach us [on the Apache mailing lists](http://kafka.apache.org/contact.html). - -To contribute follow the instructions here: - * http://kafka.apache.org/contributing.html - -We also welcome patches for the website and documentation which can be found here: - * https://svn.apache.org/repos/asf/kafka/site - - - - diff --git a/README.md b/README.md index 9b272b52c8b65..75877c067fb6e 100644 --- a/README.md +++ b/README.md @@ -1,81 +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. - -# Apache Kafka # +Apache Kafka +================= See our [web site](http://kafka.apache.org) for details on the project. -## Building a jar and running it ## -1. ./gradlew copyDependantLibs -2. ./gradlew jar -3. Follow instuctions in http://kafka.apache.org/documentation.html#quickstart +You need to have [gradle](http://www.gradle.org/installation) installed. -## Running unit tests ## -./gradlew test +### First bootstrap and download the wrapper ### + cd kafka_source_dir + gradle -## Forcing re-running unit tests w/o code change ## -./gradlew cleanTest test +Now everything else will work -## Running a particular unit test ## -./gradlew -Dtest.single=RequestResponseSerializationTest core:test +### Building a jar and running it ### + ./gradlew jar + +Follow instuctions in http://kafka.apache.org/documentation.html#quickstart + +### Building source jar ### + ./gradlew srcJar + +### Building javadocs and scaladocs ### + ./gradlew javadoc + ./gradlew javadocJar # builds a jar from the javadocs + ./gradlew scaladoc + ./gradlew scaladocJar # builds a jar from the scaladocs + ./gradlew docsJar # builds both javadoc and scaladoc jar + +### Running unit tests ### + ./gradlew test + +### Forcing re-running unit tests w/o code change ### + ./gradlew cleanTest test + +### Running a particular unit test ### + ./gradlew -Dtest.single=RequestResponseSerializationTest core:test + +### Running a particular test method within a unit test ### + ./gradlew core:test --tests kafka.api.test.ProducerFailureHandlingTest.testCannotSendToInternalTopic + ./gradlew clients:test --tests org.apache.kafka.clients.producer.MetadataTest.testMetadataUpdateWaitTime + + +### Running a particular unit test with log4j output ### + change the log4j setting in either clients/src/test/resources/log4j.properties or core/src/test/resources/log4j.properties + ./gradlew -i -Dtest.single=RequestResponseSerializationTest core:test + +### Building a binary release gzipped tar ball ### + ./gradlew clean + ./gradlew releaseTarGz + The above command will fail if you haven't set up the signing key. To bypass signing the artifact, you can run + ./gradlew releaseTarGz -x signArchives -## Building a binary release gzipped tar ball ## -./gradlew clean -./gradlew releaseTarGz The release file can be found inside ./core/build/distributions/. -## Cleaning the build ## -./gradlew clean +### Cleaning the build ### + ./gradlew clean + +### Running a task on a particular version of Scala (either 2.9.1, 2.9.2, 2.10.5 or 2.11.7) ### +#### (If building a jar with a version other than 2.10, need to set SCALA_BINARY_VERSION variable or change it in bin/kafka-run-class.sh to run quick start.) #### + ./gradlew -PscalaVersion=2.9.1 jar + ./gradlew -PscalaVersion=2.9.1 test + ./gradlew -PscalaVersion=2.9.1 releaseTarGz + +### Running a task for a specific project ### +This is for 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples' and 'clients' + ./gradlew core:jar + ./gradlew core:test + +### Listing all gradle tasks ### + ./gradlew tasks + +### Building IDE project #### + ./gradlew eclipse + ./gradlew idea + +### Building the jar for all scala versions and for all projects ### + ./gradlew jarAll -## Running a task on a particular version of Scala (either 2.8.0, 2.8.2, 2.9.1, 2.9.2 or 2.10.1) ## -## (If building a jar with a version other than 2.8.0, the scala version variable in bin/kafka-run-class.sh needs to be changed to run quick start.) ## -./gradlew -PscalaVersion=2.9.1 jar -./gradlew -PscalaVersion=2.9.1 test -./gradlew -PscalaVersion=2.9.1 releaseTarGz +### Running unit tests for all scala versions and for all projects ### + ./gradlew testAll -## Running a task for a specific project in 'core', 'perf', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients' ## -./gradlew core:jar -./gradlew core:test +### Building a binary release gzipped tar ball for all scala versions ### + ./gradlew releaseTarGzAll -## Listing all gradle tasks ## -./gradlew tasks +### Publishing the jar for all version of Scala and for all projects to maven ### + ./gradlew uploadArchivesAll -# Building IDE project ## -./gradlew eclipse -./gradlew idea +Please note for this to work you should create/update `~/.gradle/gradle.properties` and assign the following variables -# Building the jar for all scala versions and for all projects ## -./gradlew jarAll + mavenUrl= + mavenUsername= + mavenPassword= + signing.keyId= + signing.password= + signing.secretKeyRingFile= -## Running unit tests for all scala versions and for all projects ## -./gradlew testAll +### Publishing the jars without signing to a local repository ### + ./gradlew -Dorg.gradle.project.skipSigning=true -Dorg.gradle.project.mavenUrl=file://path/to/repo uploadArchivesAll -## Building a binary release gzipped tar ball for all scala versions ## -./gradlew releaseTarGzAll +### Building the test jar ### + ./gradlew testJar -## Publishing the jar for all version of Scala and for all projects to maven (To test locally, change mavenUrl in gradle.properties to a local dir.) ## -./gradlew uploadArchivesAll +### Determining how transitive dependencies are added ### + ./gradlew core:dependencies --configuration runtime + +### Running checkstyle on the java code ### + ./gradlew checkstyleMain checkstyleTest -## Building the test jar ## -./gradlew testJar +### Running in Vagrant ### -## Determining how transitive dependencies are added ## -./gradlew core:dependencies --configuration runtime +See [vagrant/README.md](vagrant/README.md). -## Contribution ## +### Contribution ### -Kafka is a new project, and we are interested in building the community; we would welcome any thoughts or [patches](https://issues.apache.org/jira/browse/KAFKA). You can reach us [on the Apache mailing lists](http://kafka.apache.org/contact.html). +Apache Kafka is interested in building the community; we would welcome any thoughts or [patches](https://issues.apache.org/jira/browse/KAFKA). You can reach us [on the Apache mailing lists](http://kafka.apache.org/contact.html). To contribute follow the instructions here: * http://kafka.apache.org/contributing.html diff --git a/Vagrantfile b/Vagrantfile new file mode 100644 index 0000000000000..1d7cc01b106f8 --- /dev/null +++ b/Vagrantfile @@ -0,0 +1,171 @@ +# 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. +# -*- mode: ruby -*- +# vi: set ft=ruby : + +require 'socket' + +# Vagrantfile API/syntax version. Don't touch unless you know what you're doing! +VAGRANTFILE_API_VERSION = "2" + +# General config +enable_dns = false +enable_jmx = false +num_zookeepers = 1 +num_brokers = 3 +num_workers = 0 # Generic workers that get the code, but don't start any services +ram_megabytes = 1280 + +# EC2 +ec2_access_key = ENV['AWS_ACCESS_KEY'] +ec2_secret_key = ENV['AWS_SECRET_KEY'] +ec2_keypair_name = nil +ec2_keypair_file = nil + +ec2_region = "us-east-1" +ec2_az = nil # Uses set by AWS +ec2_ami = "ami-9eaa1cf6" +ec2_instance_type = "m3.medium" +ec2_user = "ubuntu" +ec2_security_groups = nil +ec2_subnet_id = nil +# Only override this by setting it to false if you're running in a VPC and you +# are running Vagrant from within that VPC as well. +ec2_associate_public_ip = nil + +local_config_file = File.join(File.dirname(__FILE__), "Vagrantfile.local") +if File.exists?(local_config_file) then + eval(File.read(local_config_file), binding, "Vagrantfile.local") +end + +# TODO(ksweeney): RAM requirements are not empirical and can probably be significantly lowered. +Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| + config.hostmanager.enabled = true + config.hostmanager.manage_host = enable_dns + config.hostmanager.include_offline = false + + ## Provider-specific global configs + config.vm.provider :virtualbox do |vb,override| + override.vm.box = "ubuntu/trusty64" + + override.hostmanager.ignore_private_ip = false + + # Brokers started with the standard script currently set Xms and Xmx to 1G, + # plus we need some extra head room. + vb.customize ["modifyvm", :id, "--memory", ram_megabytes.to_s] + + if Vagrant.has_plugin?("vagrant-cachier") + config.cache.scope = :box + # Besides the defaults, we use a custom cache to handle the Oracle JDK + # download, which downloads via wget during an apt install. Because of the + # way the installer ends up using its cache directory, we need to jump + # through some hoops instead of just specifying a cache directly -- we + # share to a temporary location and the provisioning scripts symlink data + # to the right location. + config.cache.enable :generic, { + "oracle-jdk7" => { cache_dir: "/tmp/oracle-jdk7-installer-cache" }, + } + end + end + + config.vm.provider :aws do |aws,override| + # The "box" is specified as an AMI + override.vm.box = "dummy" + override.vm.box_url = "https://github.com/mitchellh/vagrant-aws/raw/master/dummy.box" + + override.hostmanager.ignore_private_ip = true + + override.ssh.username = ec2_user + override.ssh.private_key_path = ec2_keypair_file + + aws.access_key_id = ec2_access_key + aws.secret_access_key = ec2_secret_key + aws.keypair_name = ec2_keypair_name + + aws.region = ec2_region + aws.availability_zone = ec2_az + aws.instance_type = ec2_instance_type + aws.ami = ec2_ami + aws.security_groups = ec2_security_groups + aws.subnet_id = ec2_subnet_id + # If a subnet is specified, default to turning on a public IP unless the + # user explicitly specifies the option. Without a public IP, Vagrant won't + # be able to SSH into the hosts unless Vagrant is also running in the VPC. + if ec2_associate_public_ip.nil? + aws.associate_public_ip = true unless ec2_subnet_id.nil? + else + aws.associate_public_ip = ec2_associate_public_ip + end + + # Exclude some directories that can grow very large from syncing + config.vm.synced_folder ".", "/vagrant", type: "rsync", :rsync_excludes => ['.git', 'core/data/', 'logs/', 'system_test/'] + end + + def name_node(node, name) + node.vm.hostname = name + node.vm.provider :aws do |aws| + aws.tags = { 'Name' => "kafka-vagrant-" + Socket.gethostname + "-" + name } + end + end + + def assign_local_ip(node, ip_address) + node.vm.provider :virtualbox do |vb,override| + override.vm.network :private_network, ip: ip_address + end + end + + ## Cluster definition + zookeepers = [] + (1..num_zookeepers).each { |i| + name = "zk" + i.to_s + zookeepers.push(name) + config.vm.define name do |zookeeper| + name_node(zookeeper, name) + ip_address = "192.168.50." + (10 + i).to_s + assign_local_ip(zookeeper, ip_address) + zookeeper.vm.provision "shell", path: "vagrant/base.sh" + zk_jmx_port = enable_jmx ? (8000 + i).to_s : "" + zookeeper.vm.provision "shell", path: "vagrant/zk.sh", :args => [i.to_s, num_zookeepers, zk_jmx_port] + end + } + + (1..num_brokers).each { |i| + name = "broker" + i.to_s + config.vm.define name do |broker| + name_node(broker, name) + ip_address = "192.168.50." + (50 + i).to_s + assign_local_ip(broker, ip_address) + # We need to be careful about what we list as the publicly routable + # address since this is registered in ZK and handed out to clients. If + # host DNS isn't setup, we shouldn't use hostnames -- IP addresses must be + # used to support clients running on the host. + zookeeper_connect = zookeepers.map{ |zk_addr| zk_addr + ":2181"}.join(",") + broker.vm.provision "shell", path: "vagrant/base.sh" + kafka_jmx_port = enable_jmx ? (9000 + i).to_s : "" + broker.vm.provision "shell", path: "vagrant/broker.sh", :args => [i.to_s, enable_dns ? name : ip_address, zookeeper_connect, kafka_jmx_port] + end + } + + (1..num_workers).each { |i| + name = "worker" + i.to_s + config.vm.define name do |worker| + name_node(worker, name) + ip_address = "192.168.50." + (100 + i).to_s + assign_local_ip(worker, ip_address) + worker.vm.provision "shell", path: "vagrant/base.sh" + end + } + +end diff --git a/bin/kafka-console-consumer.sh b/bin/kafka-console-consumer.sh index b86ea56c5c01f..07c90a9bebfed 100755 --- a/bin/kafka-console-consumer.sh +++ b/bin/kafka-console-consumer.sh @@ -5,14 +5,17 @@ # 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. -export KAFKA_HEAP_OPTS="-Xmx512M" -exec $(dirname $0)/kafka-run-class.sh kafka.consumer.ConsoleConsumer $@ +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" +fi + +exec $(dirname $0)/kafka-run-class.sh kafka.tools.ConsoleConsumer $@ diff --git a/bin/kafka-console-producer.sh b/bin/kafka-console-producer.sh index ec0f21f9a2e64..ccca66de44384 100755 --- a/bin/kafka-console-producer.sh +++ b/bin/kafka-console-producer.sh @@ -5,14 +5,16 @@ # 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. -export KAFKA_HEAP_OPTS="-Xmx512M" -exec $(dirname $0)/kafka-run-class.sh kafka.producer.ConsoleProducer $@ +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" +fi +exec $(dirname $0)/kafka-run-class.sh kafka.tools.ConsoleProducer $@ diff --git a/bin/kafka-simple-consumer-perf-test.sh b/bin/kafka-consumer-groups.sh similarity index 87% rename from bin/kafka-simple-consumer-perf-test.sh rename to bin/kafka-consumer-groups.sh index c466cc8886a25..f4786dbda01f2 100755 --- a/bin/kafka-simple-consumer-perf-test.sh +++ b/bin/kafka-consumer-groups.sh @@ -14,5 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -export KAFKA_HEAP_OPTS="-Xmx512M" -exec $(dirname $0)/kafka-run-class.sh kafka.perf.SimpleConsumerPerformance $@ +exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConsumerGroupCommand $@ diff --git a/sbt b/bin/kafka-consumer-offset-checker.sh similarity index 87% rename from sbt rename to bin/kafka-consumer-offset-checker.sh index 944ebf8c383ec..c275f7ef6975c 100755 --- a/sbt +++ b/bin/kafka-consumer-offset-checker.sh @@ -1,16 +1,17 @@ +#!/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. -java -Xmx1024M -XX:MaxPermSize=512m -Dbuild.number="$BUILD_NUMBER" -jar `dirname $0`/lib/sbt-launch.jar "$@" +exec $(dirname $0)/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker $@ diff --git a/bin/kafka-consumer-perf-test.sh b/bin/kafka-consumer-perf-test.sh index 38c56c78304c2..ebc513aa7331e 100755 --- a/bin/kafka-consumer-perf-test.sh +++ b/bin/kafka-consumer-perf-test.sh @@ -5,14 +5,16 @@ # 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. -export KAFKA_HEAP_OPTS="-Xmx512M" -exec $(dirname $0)/kafka-run-class.sh kafka.perf.ConsumerPerformance $@ +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" +fi +exec $(dirname $0)/kafka-run-class.sh kafka.tools.ConsumerPerformance $@ diff --git a/project/build.properties b/bin/kafka-mirror-maker.sh old mode 100644 new mode 100755 similarity index 90% rename from project/build.properties rename to bin/kafka-mirror-maker.sh index eea46f3ca0a07..56e342cd65788 --- a/project/build.properties +++ b/bin/kafka-mirror-maker.sh @@ -1,3 +1,4 @@ +#!/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. @@ -12,6 +13,5 @@ # 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. -#Project properties -#Mon Feb 28 11:55:49 PST 2011 -sbt.version=0.12.1 + +exec $(dirname $0)/kafka-run-class.sh kafka.tools.MirrorMaker $@ diff --git a/bin/kafka-producer-perf-test.sh b/bin/kafka-producer-perf-test.sh index d75ab7d1f303e..84ac9497c5cbf 100755 --- a/bin/kafka-producer-perf-test.sh +++ b/bin/kafka-producer-perf-test.sh @@ -5,14 +5,16 @@ # 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. -export KAFKA_HEAP_OPTS="-Xmx512M" -exec $(dirname $0)/kafka-run-class.sh kafka.perf.ProducerPerformance $@ +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" +fi +exec $(dirname $0)/kafka-run-class.sh kafka.tools.ProducerPerformance $@ diff --git a/system_test/mirror_maker/config/zookeeper_target.properties b/bin/kafka-replica-verification.sh old mode 100644 new mode 100755 similarity index 85% rename from system_test/mirror_maker/config/zookeeper_target.properties rename to bin/kafka-replica-verification.sh index 55a7eb189d642..ee6d19e7ffa49 --- a/system_test/mirror_maker/config/zookeeper_target.properties +++ b/bin/kafka-replica-verification.sh @@ -1,3 +1,4 @@ +#!/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. @@ -12,7 +13,5 @@ # 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. -# the directory where the snapshot is stored. -dataDir=/tmp/zookeeper_target -# the port at which the clients will connect -clientPort=2183 + +exec $(dirname $0)/kafka-run-class.sh kafka.tools.ReplicaVerificationTool $@ diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 75a3fc42a2e41..8c3fa28614534 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -5,9 +5,9 @@ # 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. @@ -23,49 +23,44 @@ fi base_dir=$(dirname $0)/.. # create logs directory -LOG_DIR=$base_dir/logs -if [ ! -d $LOG_DIR ]; then - mkdir $LOG_DIR +if [ "x$LOG_DIR" = "x" ]; then + LOG_DIR="$base_dir/logs" +fi + +if [ ! -d "$LOG_DIR" ]; then + mkdir -p "$LOG_DIR" fi if [ -z "$SCALA_VERSION" ]; then - SCALA_VERSION=2.8.0 + SCALA_VERSION=2.10.5 fi -# TODO: remove when removing sbt -# assume all dependencies have been packaged into one jar with sbt-assembly's task "assembly-package-dependency" -for file in $base_dir/core/target/scala-${SCALA_VERSION}/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done +if [ -z "$SCALA_BINARY_VERSION" ]; then + SCALA_BINARY_VERSION=2.10 +fi # run ./gradlew copyDependantLibs to get all dependant jars in a local dir -for file in $base_dir/core/build/dependant-libs-${SCALA_VERSION}/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/perf/build/libs//kafka-perf_${SCALA_VERSION}*.jar; +for file in $base_dir/core/build/dependant-libs-${SCALA_VERSION}*/*.jar; do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/clients/build/libs//kafka-clients*.jar; +for file in $base_dir/examples/build/libs//kafka-examples*.jar; do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/examples/build/libs//kafka-examples*.jar; +for file in $base_dir/contrib/hadoop-consumer/build/libs//kafka-hadoop-consumer*.jar; do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/contrib/hadoop-consumer/build/libs//kafka-hadoop-consumer*.jar; +for file in $base_dir/contrib/hadoop-producer/build/libs//kafka-hadoop-producer*.jar; do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/contrib/hadoop-producer/build/libs//kafka-hadoop-producer*.jar; +for file in $base_dir/clients/build/libs/kafka-clients*.jar; do CLASSPATH=$CLASSPATH:$file done @@ -76,7 +71,7 @@ do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/core/build/libs/kafka_${SCALA_VERSION}*.jar; +for file in $base_dir/core/build/libs/kafka_${SCALA_BINARY_VERSION}*.jar; do CLASSPATH=$CLASSPATH:$file done @@ -117,7 +112,7 @@ 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 -Djava.awt.headless=true" + KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC -Djava.awt.headless=true" fi @@ -130,7 +125,7 @@ while [ $# -gt 0 ]; do shift 2 ;; -loggc) - if [ -z "$KAFKA_GC_LOG_OPTS"] ; then + if [ -z "$KAFKA_GC_LOG_OPTS" ]; then GC_LOG_ENABLED="true" fi shift @@ -159,6 +154,3 @@ if [ "x$DAEMON_MODE" = "xtrue" ]; then 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 7050649e6795b..dc01d46a13f16 100755 --- a/bin/kafka-server-start.sh +++ b/bin/kafka-server-start.sh @@ -16,12 +16,18 @@ if [ $# -lt 1 ]; then - echo "USAGE: $0 [-daemon] server.properties" + echo "USAGE: $0 [-daemon] server.properties [--override property=value]*" 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" + +if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then + export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/log4j.properties" +fi + +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx1G -Xms1G" +fi EXTRA_ARGS="-name kafkaServer -loggc" diff --git a/bin/kafka-server-stop.sh b/bin/kafka-server-stop.sh index 35a26a6529a91..cd8160c41e7b0 100755 --- a/bin/kafka-server-stop.sh +++ b/bin/kafka-server-stop.sh @@ -13,4 +13,4 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -ps ax | grep -i 'kafka\.Kafka' | grep java | grep -v grep | awk '{print $1}' | xargs kill -SIGINT +ps ax | grep -i 'kafka\.Kafka' | grep java | grep -v grep | awk '{print $1}' | xargs kill -SIGTERM diff --git a/bin/run-rat.sh b/bin/run-rat.sh deleted file mode 100755 index 1b7bc312e8b42..0000000000000 --- a/bin/run-rat.sh +++ /dev/null @@ -1,35 +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)/.. -rat_excludes_file=$base_dir/.rat-excludes - -if [ -z "$JAVA_HOME" ]; then - JAVA="java" -else - JAVA="$JAVA_HOME/bin/java" -fi - -rat_command="$JAVA -jar $base_dir/lib/apache-rat-0.8.jar --dir $base_dir " - -for f in $(cat $rat_excludes_file); -do - rat_command="${rat_command} -e $f" -done - -echo "Running " $rat_command -$rat_command > $base_dir/rat.out - diff --git a/bin/windows/kafka-console-consumer.bat b/bin/windows/kafka-console-consumer.bat index a287b9e3056e4..f70f98a3899bb 100644 --- a/bin/windows/kafka-console-consumer.bat +++ b/bin/windows/kafka-console-consumer.bat @@ -15,6 +15,6 @@ rem See the License for the specific language governing permissions and rem limitations under the License. SetLocal -set KAFKA_OPTS=-Xmx512M -server -Dcom.sun.management.jmxremote -Dlog4j.configuration=file:"%CD%\kafka-console-consumer-log4j.properties" -kafka-run-class.bat kafka.consumer.ConsoleConsumer %* +set KAFKA_HEAP_OPTS=-Xmx512M +%~dp0kafka-run-class.bat kafka.tools.ConsoleConsumer %* EndLocal diff --git a/bin/windows/kafka-console-producer.bat b/bin/windows/kafka-console-producer.bat index b321ee28e8129..a5b57de7ba6c1 100644 --- a/bin/windows/kafka-console-producer.bat +++ b/bin/windows/kafka-console-producer.bat @@ -14,4 +14,7 @@ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. rem See the License for the specific language governing permissions and rem limitations under the License. -kafka-run-class.bat kafka.producer.ConsoleProducer %* +SetLocal +set KAFKA_HEAP_OPTS=-Xmx512M +%~dp0kafka-run-class.bat kafka.tools.ConsoleProducer %* +EndLocal diff --git a/sbt.bat b/bin/windows/kafka-consumer-offset-checker.bat similarity index 91% rename from sbt.bat rename to bin/windows/kafka-consumer-offset-checker.bat index 9321ada9316ac..b6967c49609c3 100644 --- a/sbt.bat +++ b/bin/windows/kafka-consumer-offset-checker.bat @@ -1,17 +1,17 @@ -@echo off -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. - -java -Xmx1024M -XX:MaxPermSize=512m -jar lib\sbt-launch.jar "%1" \ No newline at end of file +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +%~dp0kafka-run-class.bat kafka.tools.ConsumerOffsetChecker %* diff --git a/bin/windows/kafka-consumer-perf-test.bat b/bin/windows/kafka-consumer-perf-test.bat new file mode 100644 index 0000000000000..afc2259cfd372 --- /dev/null +++ b/bin/windows/kafka-consumer-perf-test.bat @@ -0,0 +1,20 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +SetLocal +set KAFKA_HEAP_OPTS=-Xmx512M -Xms512M +%~dp0kafka-run-class.bat kafka.tools.ConsumerPerformance %* +EndLocal diff --git a/bin/windows/kafka-mirror-maker.bat b/bin/windows/kafka-mirror-maker.bat new file mode 100644 index 0000000000000..819e7d8248163 --- /dev/null +++ b/bin/windows/kafka-mirror-maker.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +%~dp0kafka-run-class.bat kafka.tools.MirrorMaker %* diff --git a/bin/windows/kafka-preferred-replica-election.bat b/bin/windows/kafka-preferred-replica-election.bat new file mode 100644 index 0000000000000..a9a5b7e1dbcac --- /dev/null +++ b/bin/windows/kafka-preferred-replica-election.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +%~dp0kafka-run-class.bat kafka.admin.PreferredReplicaLeaderElectionCommand %* diff --git a/bin/windows/kafka-producer-perf-test.bat b/bin/windows/kafka-producer-perf-test.bat new file mode 100644 index 0000000000000..a894752b0eadd --- /dev/null +++ b/bin/windows/kafka-producer-perf-test.bat @@ -0,0 +1,20 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +SetLocal +set KAFKA_HEAP_OPTS=-Xmx512M +%~dp0kafka-run-class.bat kafka.tools.ProducerPerformance %* +EndLocal diff --git a/bin/windows/kafka-reassign-partitions.bat b/bin/windows/kafka-reassign-partitions.bat new file mode 100644 index 0000000000000..0c13ee38d8df2 --- /dev/null +++ b/bin/windows/kafka-reassign-partitions.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +%~dp0kafka-run-class.bat kafka.admin.ReassignPartitionsCommand %* diff --git a/bin/windows/kafka-replay-log-producer.bat b/bin/windows/kafka-replay-log-producer.bat new file mode 100644 index 0000000000000..2aec32620a68a --- /dev/null +++ b/bin/windows/kafka-replay-log-producer.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +%~dp0kafka-run-class.bat kafka.tools.ReplayLogProducer %* diff --git a/bin/windows/kafka-replica-verification.bat b/bin/windows/kafka-replica-verification.bat new file mode 100644 index 0000000000000..481db57fda022 --- /dev/null +++ b/bin/windows/kafka-replica-verification.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +%~dp0kafka-run-class.bat kafka.tools.ReplicaVerificationTool %* diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat index f274cd7d883ab..4aa2ab8ddb9bf 100644 --- a/bin/windows/kafka-run-class.bat +++ b/bin/windows/kafka-run-class.bat @@ -17,76 +17,117 @@ rem limitations under the License. setlocal enabledelayedexpansion IF [%1] EQU [] ( - echo "USAGE: $0 classname [opts]" - goto :eof + echo USAGE: %0 classname [opts] + EXIT /B 1 ) -set BASE_DIR=%CD%\.. +rem Using pushd popd to set BASE_DIR to the absolute path +pushd %~dp0..\.. +set BASE_DIR=%CD% +popd set CLASSPATH= -echo %BASE_DIR% -set ivyPath=%USERPROFILE%\.ivy2\cache - -set snappy=%ivyPath%/org.xerial.snappy/snappy-java/bundles/snappy-java-1.0.5.jar - call :concat %snappy% - -set library=%ivyPath%/org.scala-lang/scala-library/jars/scala-library-2.8.0.jar - call :concat %library% +IF ["%SCALA_VERSION%"] EQU [""] ( + set SCALA_VERSION=2.10.5 +) -set compiler=%ivyPath%/org.scala-lang/scala-compiler/jars/scala-compiler-2.8.0.jar - call :concat %compiler% +IF ["%SCALA_BINARY_VERSION%"] EQU [""] ( + set SCALA_BINARY_VERSION=2.10 +) -set log4j=%ivyPath%/log4j/log4j/jars/log4j-1.2.15.jar - call :concat %log4j% +rem Classpath addition for kafka-core dependencies +for %%i in (%BASE_DIR%\core\build\dependant-libs-%SCALA_VERSION%\*.jar) do ( + call :concat %%i +) -set slf=%ivyPath%/org.slf4j/slf4j-api/jars/slf4j-api-1.6.4.jar - call :concat %slf% +rem Classpath addition for kafka-perf dependencies +for %%i in (%BASE_DIR%\perf\build\dependant-libs-%SCALA_VERSION%\*.jar) do ( + call :concat %%i +) -set zookeeper=%ivyPath%/org.apache.zookeeper/zookeeper/jars/zookeeper-3.3.4.jar - call :concat %zookeeper% +rem Classpath addition for kafka-clients +for %%i in (%BASE_DIR%\clients\build\libs\kafka-clients-*.jar) do ( + call :concat %%i +) -set jopt=%ivyPath%/net.sf.jopt-simple/jopt-simple/jars/jopt-simple-3.2.jar - call :concat %jopt% +rem Classpath addition for kafka-examples +for %%i in (%BASE_DIR%\examples\build\libs\kafka-examples-*.jar) do ( + call :concat %%i +) -for %%i in (%BASE_DIR%\core\target\scala-2.8.0\*.jar) do ( +rem Classpath addition for contrib/hadoop-consumer +for %%i in (%BASE_DIR%\contrib\hadoop-consumer\build\libs\kafka-hadoop-consumer-*.jar) do ( call :concat %%i ) -for %%i in (%BASE_DIR%\core\lib\*.jar) do ( +rem Classpath addition for contrib/hadoop-producer +for %%i in (%BASE_DIR%\contrib\hadoop-producer\build\libs\kafka-hadoop-producer-*.jar) do ( call :concat %%i ) -for %%i in (%BASE_DIR%\perf\target\scala-2.8.0/kafka*.jar) do ( +rem Classpath addition for release +for %%i in (%BASE_DIR%\libs\*.jar) do ( call :concat %%i ) -IF ["%KAFKA_JMX_OPTS%"] EQU [""] ( - set KAFKA_JMX_OPTS=-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false +rem Classpath addition for core +for %%i in (%BASE_DIR%\core\build\libs\kafka_%SCALA_BINARY_VERSION%*.jar) do ( + call :concat %%i ) -IF ["%KAFKA_OPTS%"] EQU [""] ( - set KAFKA_OPTS=-Xmx512M -server -Dlog4j.configuration=file:"%BASE_DIR%\config\log4j.properties" +rem JMX settings +IF ["%KAFKA_JMX_OPTS%"] EQU [""] ( + set KAFKA_JMX_OPTS=-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false ) +rem JMX port to use IF ["%JMX_PORT%"] NEQ [""] ( set KAFKA_JMX_OPTS=%KAFKA_JMX_OPTS% -Dcom.sun.management.jmxremote.port=%JMX_PORT% ) +rem Log4j settings +IF ["%KAFKA_LOG4J_OPTS%"] EQU [""] ( + set KAFKA_LOG4J_OPTS=-Dlog4j.configuration=file:%BASE_DIR%/config/tools-log4j.properties +) + +rem Generic jvm settings you want to add +IF ["%KAFKA_OPTS%"] EQU [""] ( + set KAFKA_OPTS= +) + +rem Which java to use IF ["%JAVA_HOME%"] EQU [""] ( set JAVA=java ) ELSE ( set JAVA="%JAVA_HOME%/bin/java" ) -set SEARCHTEXT=\bin\.. -set REPLACETEXT= -set CLASSPATH=!CLASSPATH:%SEARCHTEXT%=%REPLACETEXT%! -set COMMAND= %JAVA% %KAFKA_OPTS% %KAFKA_JMX_OPTS% -cp %CLASSPATH% %* -set SEARCHTEXT=-cp ; -set REPLACETEXT=-cp -set COMMAND=!COMMAND:%SEARCHTEXT%=%REPLACETEXT%! +rem Memory options +IF ["%KAFKA_HEAP_OPTS%"] EQU [""] ( + set KAFKA_HEAP_OPTS=-Xmx256M +) + +rem JVM performance options +IF ["%KAFKA_JVM_PERFORMANCE_OPTS%"] EQU [""] ( + set KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UseCompressedOops -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC -Djava.awt.headless=true +) + +IF ["%CLASSPATH%"] EQU [""] ( + echo Classpath is empty. Please build the project first e.g. by running 'gradlew jarAll' + EXIT /B 2 +) + +set COMMAND=%JAVA% %KAFKA_HEAP_OPTS% %KAFKA_JVM_PERFORMANCE_OPTS% %KAFKA_JMX_OPTS% %KAFKA_LOG4J_OPTS% -cp %CLASSPATH% %KAFKA_OPTS% %* +rem echo. +rem echo %COMMAND% +rem echo. %COMMAND% +goto :eof :concat -set CLASSPATH=%CLASSPATH%;"%1" \ No newline at end of file +IF ["%CLASSPATH%"] EQU [""] ( + set CLASSPATH="%1" +) ELSE ( + set CLASSPATH=%CLASSPATH%;"%1" +) diff --git a/bin/windows/kafka-server-start.bat b/bin/windows/kafka-server-start.bat index 7db76dd44bab5..6a5aa9fb44b51 100644 --- a/bin/windows/kafka-server-start.bat +++ b/bin/windows/kafka-server-start.bat @@ -15,13 +15,12 @@ rem See the License for the specific language governing permissions and rem limitations under the License. IF [%1] EQU [] ( - echo "USAGE: $0 server.properties [consumer.properties producer.properties]" - goto :eof + echo USAGE: %0 server.properties + EXIT /B 1 ) -IF [%JMX_PORT%] EQU [] ( - echo Set JMX_PORT to default value : 9999 - set JMX_PORT=9999 -) - -kafka-run-class.bat kafka.Kafka %* \ No newline at end of file +SetLocal +set KAFKA_LOG4J_OPTS=-Dlog4j.configuration=file:%~dp0../../config/log4j.properties +set KAFKA_HEAP_OPTS=-Xmx1G -Xms1G +%~dp0kafka-run-class.bat kafka.Kafka %* +EndLocal diff --git a/bin/windows/kafka-simple-consumer-shell.bat b/bin/windows/kafka-simple-consumer-shell.bat new file mode 100644 index 0000000000000..4e6ea0c575edb --- /dev/null +++ b/bin/windows/kafka-simple-consumer-shell.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +%~dp0kafka-run-class.bat kafka.tools.SimpleConsumerShell %* diff --git a/bin/windows/kafka-topics.bat b/bin/windows/kafka-topics.bat new file mode 100644 index 0000000000000..f1a9e647877f6 --- /dev/null +++ b/bin/windows/kafka-topics.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +%~dp0kafka-run-class.bat kafka.admin.TopicCommand %* diff --git a/bin/windows/zookeeper-server-start.bat b/bin/windows/zookeeper-server-start.bat index 0dd40ba7bb08c..9836283531468 100644 --- a/bin/windows/zookeeper-server-start.bat +++ b/bin/windows/zookeeper-server-start.bat @@ -15,8 +15,12 @@ rem See the License for the specific language governing permissions and rem limitations under the License. IF [%1] EQU [] ( - echo "USAGE: $0 zookeeper.properties" - goto :eof + echo USAGE: %0 zookeeper.properties + EXIT /B 1 ) -kafka-run-class.bat org.apache.zookeeper.server.quorum.QuorumPeerMain %* +SetLocal +set KAFKA_LOG4J_OPTS=-Dlog4j.configuration=file:%~dp0../../config/log4j.properties +set KAFKA_HEAP_OPTS=-Xmx512M -Xms512M +%~dp0kafka-run-class.bat org.apache.zookeeper.server.quorum.QuorumPeerMain %* +EndLocal diff --git a/bin/windows/zookeeper-shell.bat b/bin/windows/zookeeper-shell.bat new file mode 100644 index 0000000000000..e98f069fc57f5 --- /dev/null +++ b/bin/windows/zookeeper-shell.bat @@ -0,0 +1,22 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +IF [%1] EQU [] ( + echo USAGE: %0 zookeeper_host:port[/path] [args...] + EXIT /B 1 +) + +%~dp0kafka-run-class.bat org.apache.zookeeper.ZooKeeperMain -server %* diff --git a/bin/zookeeper-server-start.sh b/bin/zookeeper-server-start.sh index 2e7be7486d20a..d96878657cbee 100755 --- a/bin/zookeeper-server-start.sh +++ b/bin/zookeeper-server-start.sh @@ -20,8 +20,14 @@ then exit 1 fi base_dir=$(dirname $0) -export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/log4j.properties" -export KAFKA_HEAP_OPTS="-Xmx512M -Xms512M" + +if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then + export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/log4j.properties" +fi + +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M -Xms512M" +fi EXTRA_ARGS="-name zookeeper -loggc" diff --git a/build.gradle b/build.gradle index 858d297b9e8bf..fb9084307ae41 100644 --- a/build.gradle +++ b/build.gradle @@ -4,22 +4,31 @@ // 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. +import org.ajoberstar.grgit.Grgit + buildscript { repositories { mavenCentral() } apply from: file('gradle/buildscript.gradle'), to: buildscript + + dependencies { + // For Apache Rat plugin to ignore non-Git files, need ancient version for Java 6 compatibility + classpath group: 'org.ajoberstar', name: 'grgit', version: '0.2.3' + } } +def slf4jlog4j='org.slf4j:slf4j-log4j12:1.7.6' + allprojects { apply plugin: 'idea' repositories { @@ -27,41 +36,126 @@ allprojects { } } -apply from: file('gradle/license.gradle') +ext { + gradleVersion = "2.4" + + skipSigning = project.hasProperty('skipSigning') && skipSigning.toBoolean() + shouldSign = !skipSigning && !version.endsWith("SNAPSHOT") + + mavenUrl = project.hasProperty('mavenUrl') ? project.mavenUrl : '' + mavenUsername = project.hasProperty('mavenUsername') ? project.mavenUsername : '' + mavenPassword = project.hasProperty('mavenPassword') ? project.mavenPassword : '' +} + +apply from: file('wrapper.gradle') +apply from: file('scala.gradle') +apply from: file('gradle/rat.gradle') + +rat { + // Exclude everything under the directory that git should be ignoring via .gitignore or that isn't checked in. These + // restrict us only to files that are checked in or are staged. + def repo = Grgit.open(project.file('.')) + excludes = new ArrayList(repo.clean(ignore: false, directories: true, dryRun: true)) + // And some of the files that we have checked in should also be excluded from this check + excludes.addAll([ + '**/.git/**', + 'build/**', + 'gradlew', + 'gradlew.bat', + '**/README.md', + '.reviewboardrc', + 'system_test/**', + ]) +} subprojects { apply plugin: 'java' apply plugin: 'eclipse' apply plugin: 'maven' - + apply plugin: 'signing' + + sourceCompatibility = 1.7 + uploadArchives { repositories { - // To test locally, replace mavenUrl in gradle.properties to file://localhost/tmp/myRepo/ - mavenDeployer { - repository(url: "${mavenUrl}") { - authentication(userName: "${mavenUsername}", password: "${mavenPassword}") - } - afterEvaluate { - pom.artifactId = "${archivesBaseName}" - pom.project { - name 'Apache Kafka' - packaging 'jar' - url 'http://kafka.apache.org' - licenses { - license { - name 'The Apache Software License, Version 2.0' - url 'http://www.apache.org/licenses/LICENSE-2.0.txt' - distribution 'repo' + signing { + required { shouldSign } + sign configurations.archives + + // To test locally, replace mavenUrl in ~/.gradle/gradle.properties to file://localhost/tmp/myRepo/ + mavenDeployer { + beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) } + repository(url: "${mavenUrl}") { + authentication(userName: "${mavenUsername}", password: "${mavenPassword}") + } + afterEvaluate { + pom.artifactId = "${archivesBaseName}" + pom.project { + name 'Apache Kafka' + packaging 'jar' + url 'http://kafka.apache.org' + licenses { + license { + name 'The Apache Software License, Version 2.0' + url 'http://www.apache.org/licenses/LICENSE-2.0.txt' + distribution 'repo' + } + } + } } - } } - } } } } + jar { + from '../LICENSE' + from '../NOTICE' + } + + task srcJar(type:Jar) { + classifier = 'sources' + from '../LICENSE' + from '../NOTICE' + from sourceSets.main.java + } + + task javadocJar(type: Jar, dependsOn: javadoc) { + classifier 'javadoc' + from '../LICENSE' + from '../NOTICE' + from javadoc.destinationDir + } + + task docsJar(dependsOn: javadocJar) + + artifacts { + archives srcJar + archives javadocJar + } + + plugins.withType(ScalaPlugin) { + //source jar should also contain scala source: + srcJar.from sourceSets.main.scala + + task scaladocJar(type:Jar) { + classifier = 'scaladoc' + from '../LICENSE' + from '../NOTICE' + from scaladoc + } + + //documentation task should also trigger building scala doc jar + docsJar.dependsOn scaladocJar + + artifacts { + archives scaladocJar + } + } + tasks.withType(ScalaCompile) { scalaCompileOptions.useAnt = false + configure(scalaCompileOptions.forkOptions) { memoryMaximumSize = '1g' jvmArgs = ['-XX:MaxPermSize=512m'] @@ -69,7 +163,7 @@ subprojects { } } -for ( sv in ['2_8_0', '2_8_2', '2_9_1', '2_9_2', '2_10_1'] ) { +for ( sv in ['2_9_1', '2_9_2', '2_10_5', '2_11_7'] ) { String svInDot = sv.replaceAll( "_", ".") tasks.create(name: "jar_core_${sv}", type: GradleBuild) { @@ -84,6 +178,18 @@ for ( sv in ['2_8_0', '2_8_2', '2_9_1', '2_9_2', '2_10_1'] ) { startParameter.projectProperties = [scalaVersion: "${svInDot}"] } + tasks.create(name: "srcJar_${sv}", type: GradleBuild) { + buildFile = './build.gradle' + tasks = ['core:srcJar'] + startParameter.projectProperties = [scalaVersion: "${svInDot}"] + } + + tasks.create(name: "docsJar_${sv}", type: GradleBuild) { + buildFile = './build.gradle' + tasks = ['core:docsJar'] + startParameter.projectProperties = [scalaVersion: "${svInDot}"] + } + tasks.create(name: "releaseTarGz_${sv}", type: GradleBuild) { buildFile = './build.gradle' tasks = ['releaseTarGz'] @@ -97,65 +203,52 @@ for ( sv in ['2_8_0', '2_8_2', '2_9_1', '2_9_2', '2_10_1'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_8_0', 'jar_core_2_8_2', 'jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_1', 'clients:jar', 'perf:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar']) { +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_5', 'jar_core_2_11_7', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar']) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_8_0', 'test_core_2_8_2', 'test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_1', 'clients:test']) { +tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_5', 'srcJar_2_11_7', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar']) { } + +tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_5', 'docsJar_2_11_7', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar']) { } + +tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_5', 'test_core_2_11_7', 'clients:test', 'log4j-appender:test']) { } -tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_8_0', 'releaseTarGz_2_8_2', 'releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_1']) { +tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_8_0', 'uploadCoreArchives_2_8_2', 'uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_1', 'perf:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives']) { +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives']) { } project(':core') { println "Building project 'core' with Scala version $scalaVersion" apply plugin: 'scala' - archivesBaseName = "kafka_${scalaVersion}" - - def (major, minor, trivial) = scalaVersion.tokenize('.') - - if(major.toInteger() >= 2 && minor.toInteger() >= 9) { - sourceSets { - main { - scala { - exclude 'kafka/utils/Annotations_2.8.scala' - } - } - } - } else { - sourceSets { - main { - scala { - exclude 'kafka/utils/Annotations_2.9+.scala' - } - } - } - } + archivesBaseName = "kafka_${baseScalaVersion}" dependencies { + compile project(':clients') + compile project(':log4j-appender') compile "org.scala-lang:scala-library:$scalaVersion" - compile 'org.apache.zookeeper:zookeeper:3.3.4' - compile 'com.101tec:zkclient:0.3' + compile 'org.apache.zookeeper:zookeeper:3.4.6' + compile 'com.101tec:zkclient:0.5' compile 'com.yammer.metrics:metrics-core:2.2.0' - compile 'com.yammer.metrics:metrics-annotation:2.2.0' compile 'net.sf.jopt-simple:jopt-simple:3.2' - compile 'org.xerial.snappy:snappy-java:1.0.5' + if (scalaVersion.startsWith('2.11')) { + compile 'org.scala-lang.modules:scala-xml_2.11:1.0.4' + compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.4' + } - testCompile 'junit:junit:4.1' + testCompile 'junit:junit:4.6' testCompile 'org.easymock:easymock:3.0' testCompile 'org.objenesis:objenesis:1.2' - if (scalaVersion.startsWith('2.8')) { - testCompile 'org.scalatest:scalatest:1.2' - } else if (scalaVersion.startsWith('2.10')) { - testCompile 'org.scalatest:scalatest_2.10:1.9.1' - } else { - testCompile "org.scalatest:scalatest_$scalaVersion:1.8" - } + if (scalaVersion.startsWith('2.9')) + testCompile "org.scalatest:scalatest_$scalaVersion:1.9.1" + else + testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5" + + testRuntime "$slf4jlog4j" - zinc 'com.typesafe.zinc:zinc:0.2.5' + zinc 'com.typesafe.zinc:zinc:0.3.7' } configurations { @@ -166,17 +259,22 @@ project(':core') { compile.exclude module: 'jmxri' compile.exclude module: 'jmxtools' compile.exclude module: 'mail' + compile.exclude module: 'netty' } tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.testRuntime) { + include('slf4j-log4j12*') + } + from (configurations.runtime) { + exclude('kafka-clients*') + } into "$buildDir/dependant-libs-${scalaVersion}" - from configurations.runtime } - tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { - into "." - compression = Compression.GZIP - classifier = 'dist' + tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { + into "kafka_${baseScalaVersion}-${version}" + compression = Compression.GZIP from(project.file("../bin")) { into "bin/" } from(project.file("../config")) { into "config/" } from '../LICENSE' @@ -185,8 +283,18 @@ project(':core') { from(configurations.archives.artifacts.files) { into("libs/") } } + jar { + dependsOn 'copyDependantLibs' + } + + jar.manifest { + attributes( + 'Version': "${version}" + ) + } + task testJar(type: Jar) { - appendix = 'test' + classifier = 'test' from sourceSets.test.output } @@ -196,20 +304,9 @@ project(':core') { exceptionFormat = 'full' } } -} - -project(':perf') { - println "Building project 'perf' with Scala version $scalaVersion" - - apply plugin: 'scala' - archivesBaseName = "kafka-perf_${scalaVersion}" - dependencies { - compile project(':core') - compile "org.scala-lang:scala-library:$scalaVersion" - compile 'net.sf.jopt-simple:jopt-simple:3.2' - - zinc 'com.typesafe.zinc:zinc:0.2.5' + artifacts { + archives testJar } } @@ -224,7 +321,6 @@ project(':contrib:hadoop-consumer') { compile "org.codehaus.jackson:jackson-core-asl:1.5.5" compile "org.codehaus.jackson:jackson-mapper-asl:1.5.5" compile "org.apache.hadoop:hadoop-core:0.20.2" - compile files('lib/piggybank.jar') } configurations { @@ -243,13 +339,13 @@ project(':contrib:hadoop-producer') { dependencies { compile project(':core') - compile "org.apache.avro:avro:1.4.0" + compile("org.apache.avro:avro:1.4.0") { force = true } compile "org.apache.pig:pig:0.8.0" compile "commons-logging:commons-logging:1.0.4" compile "org.codehaus.jackson:jackson-core-asl:1.5.5" compile "org.codehaus.jackson:jackson-mapper-asl:1.5.5" compile "org.apache.hadoop:hadoop-core:0.20.2" - compile files('lib/piggybank.jar') + compile "org.apache.pig:piggybank:0.12.0" } configurations { @@ -269,17 +365,24 @@ project(':examples') { dependencies { compile project(':core') } + } project(':clients') { + apply plugin: 'checkstyle' archivesBaseName = "kafka-clients" dependencies { - testCompile 'com.novocode:junit-interface:0.9' + compile "org.slf4j:slf4j-api:1.7.6" + compile 'org.xerial.snappy:snappy-java:1.1.1.7' + compile 'net.jpountz.lz4:lz4:1.2.0' + + testCompile 'junit:junit:4.6' + testRuntime "$slf4jlog4j" } task testJar(type: Jar) { - appendix = 'test' + classifier = 'test' from sourceSets.test.output } @@ -289,4 +392,59 @@ project(':clients') { exceptionFormat = 'full' } } + + javadoc { + include "**/org/apache/kafka/clients/consumer/*" + include "**/org/apache/kafka/clients/producer/*" + include "**/org/apache/kafka/common/*" + include "**/org/apache/kafka/common/errors/*" + include "**/org/apache/kafka/common/serialization/*" + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':log4j-appender') { + apply plugin: 'checkstyle' + archivesBaseName = "kafka-log4j-appender" + + dependencies { + compile project(':clients') + compile "$slf4jlog4j" + + testCompile 'junit:junit:4.6' + testCompile project(path: ':clients', configuration: 'archives') + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/log4jappender/*" + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') } diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml new file mode 100644 index 0000000000000..a215ff36e9252 --- /dev/null +++ b/checkstyle/checkstyle.xml @@ -0,0 +1,83 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml new file mode 100644 index 0000000000000..19e0659ef9385 --- /dev/null +++ b/checkstyle/import-control.xml @@ -0,0 +1,109 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/clients/build.sbt b/clients/build.sbt deleted file mode 100644 index ca3c8ee3d7e56..0000000000000 --- a/clients/build.sbt +++ /dev/null @@ -1,11 +0,0 @@ -import sbt._ -import Keys._ -import AssemblyKeys._ - -name := "clients" - -libraryDependencies ++= Seq( - "com.novocode" % "junit-interface" % "0.9" % "test" -) - -assemblySettings diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java new file mode 100644 index 0000000000000..ed4c0d98596cc --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -0,0 +1,66 @@ +/** + * 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 org.apache.kafka.clients; + +import org.apache.kafka.common.requests.RequestSend; + +/** + * A request being sent to the server. This holds both the network send as well as the client-level metadata. + */ +public final class ClientRequest { + + private final long createdMs; + private final boolean expectResponse; + private final RequestSend request; + private final RequestCompletionHandler callback; + + /** + * @param createdMs The unix timestamp in milliseconds for the time at which this request was created. + * @param expectResponse Should we expect a response message or is this request complete once it is sent? + * @param request The request + * @param callback A callback to execute when the response has been received (or null if no callback is necessary) + */ + public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, RequestCompletionHandler callback) { + this.createdMs = createdMs; + this.callback = callback; + this.request = request; + this.expectResponse = expectResponse; + } + + @Override + public String toString() { + return "ClientRequest(expectResponse=" + expectResponse + ", callback=" + callback + ", request=" + request + + ")"; + } + + public boolean expectResponse() { + return expectResponse; + } + + public RequestSend request() { + return request; + } + + public boolean hasCallback() { + return callback != null; + } + + public RequestCompletionHandler callback() { + return callback; + } + + public long createdTime() { + return createdMs; + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java new file mode 100644 index 0000000000000..14ef69afabc4b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java @@ -0,0 +1,78 @@ +/** + * 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 org.apache.kafka.clients; + +import org.apache.kafka.common.protocol.types.Struct; + +/** + * A response from the server. Contains both the body of the response as well as the correlated request that was + * originally sent. + */ +public class ClientResponse { + + private final long received; + private final boolean disconnected; + private final ClientRequest request; + private final Struct responseBody; + + /** + * @param request The original request + * @param received The unix timestamp when this response was received + * @param disconnected Whether the client disconnected before fully reading a response + * @param responseBody The response contents (or null) if we disconnected or no response was expected + */ + public ClientResponse(ClientRequest request, long received, boolean disconnected, Struct responseBody) { + super(); + this.received = received; + this.disconnected = disconnected; + this.request = request; + this.responseBody = responseBody; + } + + public long receivedTime() { + return received; + } + + public boolean wasDisconnected() { + return disconnected; + } + + public ClientRequest request() { + return request; + } + + public Struct responseBody() { + return responseBody; + } + + public boolean hasResponse() { + return responseBody != null; + } + + public long requestLatencyMs() { + return receivedTime() - this.request.createdTime(); + } + + @Override + public String toString() { + return "ClientResponse(received=" + received + + ", disconnected=" + + disconnected + + ", request=" + + request + + ", responseBody=" + + responseBody + + ")"; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java new file mode 100644 index 0000000000000..0d68bf1e1e90f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -0,0 +1,64 @@ +/** + * 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 org.apache.kafka.clients; + +import java.io.Closeable; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.kafka.common.config.ConfigException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.kafka.common.utils.Utils.getHost; +import static org.apache.kafka.common.utils.Utils.getPort; + +public class ClientUtils { + private static final Logger log = LoggerFactory.getLogger(ClientUtils.class); + + public static List parseAndValidateAddresses(List urls) { + List addresses = new ArrayList(); + for (String url : urls) { + if (url != null && url.length() > 0) { + String host = getHost(url); + Integer port = getPort(url); + if (host == null || port == null) + throw new ConfigException("Invalid url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url); + try { + InetSocketAddress address = new InetSocketAddress(host, port); + if (address.isUnresolved()) + throw new ConfigException("DNS resolution failed for url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url); + addresses.add(address); + } catch (NumberFormatException e) { + throw new ConfigException("Invalid port in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url); + } + } + } + if (addresses.size() < 1) + throw new ConfigException("No bootstrap urls given in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); + return addresses; + } + + public static void closeQuietly(Closeable c, String name, AtomicReference firstException) { + if (c != null) { + try { + c.close(); + } catch (Throwable t) { + firstException.compareAndSet(null, t); + log.error("Failed to close " + name, t); + } + } + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java new file mode 100644 index 0000000000000..9ebda5eae5936 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java @@ -0,0 +1,161 @@ +/** + * 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 org.apache.kafka.clients; + +import java.util.HashMap; +import java.util.Map; + +/** + * The state of our connection to each node in the cluster. + * + */ +final class ClusterConnectionStates { + private final long reconnectBackoffMs; + private final Map nodeState; + + public ClusterConnectionStates(long reconnectBackoffMs) { + this.reconnectBackoffMs = reconnectBackoffMs; + this.nodeState = new HashMap(); + } + + /** + * Return true iff we can currently initiate a new connection. This will be the case if we are not + * connected and haven't been connected for at least the minimum reconnection backoff period. + * @param id The connection id to check + * @param now The current time in MS + * @return true if we can initiate a new connection + */ + public boolean canConnect(String id, long now) { + NodeConnectionState state = nodeState.get(id); + if (state == null) + return true; + else + return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs >= this.reconnectBackoffMs; + } + + /** + * Return true if we are disconnected from the given node and can't re-establish a connection yet + * @param id The connection to check + * @param now The current time in ms + */ + public boolean isBlackedOut(String id, long now) { + NodeConnectionState state = nodeState.get(id); + if (state == null) + return false; + else + return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs < this.reconnectBackoffMs; + } + + /** + * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When + * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled + * connections. + * @param id The connection to check + * @param now The current time in ms + */ + public long connectionDelay(String id, long now) { + NodeConnectionState state = nodeState.get(id); + if (state == null) return 0; + long timeWaited = now - state.lastConnectAttemptMs; + if (state.state == ConnectionState.DISCONNECTED) { + return Math.max(this.reconnectBackoffMs - timeWaited, 0); + } else { + // When connecting or connected, we should be able to delay indefinitely since other events (connection or + // data acked) will cause a wakeup once data can be sent. + return Long.MAX_VALUE; + } + } + + /** + * Enter the connecting state for the given connection. + * @param id The id of the connection + * @param now The current time. + */ + public void connecting(String id, long now) { + nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now)); + } + + /** + * Return true iff a specific connection is connected + * @param id The id of the connection to check + */ + public boolean isConnected(String id) { + NodeConnectionState state = nodeState.get(id); + return state != null && state.state == ConnectionState.CONNECTED; + } + + /** + * Return true iff we are in the process of connecting + * @param id The id of the connection + */ + public boolean isConnecting(String id) { + NodeConnectionState state = nodeState.get(id); + return state != null && state.state == ConnectionState.CONNECTING; + } + + /** + * Enter the connected state for the given connection + * @param id The connection identifier + */ + public void connected(String id) { + NodeConnectionState nodeState = nodeState(id); + nodeState.state = ConnectionState.CONNECTED; + } + + /** + * Enter the disconnected state for the given node + * @param id The connection we have disconnected + */ + public void disconnected(String id) { + NodeConnectionState nodeState = nodeState(id); + nodeState.state = ConnectionState.DISCONNECTED; + } + + /** + * Get the state of a given connection + * @param id The id of the connection + * @return The state of our connection + */ + public ConnectionState connectionState(String id) { + return nodeState(id).state; + } + + /** + * Get the state of a given node + * @param id The connection to fetch the state for + */ + private NodeConnectionState nodeState(String id) { + NodeConnectionState state = this.nodeState.get(id); + if (state == null) + throw new IllegalStateException("No entry found for connection " + id); + return state; + } + + /** + * The state of our connection to a node + */ + private static class NodeConnectionState { + + ConnectionState state; + long lastConnectAttemptMs; + + public NodeConnectionState(ConnectionState state, long lastConnectAttempt) { + this.state = state; + this.lastConnectAttemptMs = lastConnectAttempt; + } + + public String toString() { + return "NodeState(" + state + ", " + lastConnectAttemptMs + ")"; + } + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java new file mode 100644 index 0000000000000..2c421f42ed3fc --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -0,0 +1,60 @@ +/** + * 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 org.apache.kafka.clients; + +/** + * Some configurations shared by both producer and consumer + */ +public class CommonClientConfigs { + + /* + * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE. + */ + + public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + public static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping—this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form " + + "host1:port1,host2:port2,.... Since these servers are just used for the initial connection to " + + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of " + + "servers (you may want more than one, though, in case a server is down)."; + + public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; + public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions."; + + public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes"; + public static final String SEND_BUFFER_DOC = "The size of the TCP send buffer (SO_SNDBUF) to use when sending data."; + + public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes"; + public static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer (SO_RCVBUF) to use when reading data."; + + public static final String CLIENT_ID_CONFIG = "client.id"; + public static final String CLIENT_ID_DOC = "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging."; + + public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; + public static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all requests sent by the consumer to the broker."; + + public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms"; + public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop."; + + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; + public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The number of samples maintained to compute metrics."; + + public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples"; + public static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics."; + + public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; + public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the MetricReporter interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; + + public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = "connections.max.idle.ms"; + public static final String CONNECTIONS_MAX_IDLE_MS_DOC = "Close idle connections after the number of milliseconds specified by this config."; +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java new file mode 100644 index 0000000000000..3867f8ecd5925 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java @@ -0,0 +1,20 @@ +/** + * 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 org.apache.kafka.clients; + +/** + * The states of a node connection + */ +public enum ConnectionState { + DISCONNECTED, CONNECTING, CONNECTED +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java new file mode 100644 index 0000000000000..15d00d4e484bb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java @@ -0,0 +1,126 @@ +/** + * 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 org.apache.kafka.clients; + +import java.util.ArrayDeque; +import java.util.Collections; +import java.util.Deque; +import java.util.HashMap; +import java.util.Map; + +/** + * The set of requests which have been sent or are being sent but haven't yet received a response + */ +final class InFlightRequests { + + private final int maxInFlightRequestsPerConnection; + private final Map> requests = new HashMap>(); + + public InFlightRequests(int maxInFlightRequestsPerConnection) { + this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection; + } + + /** + * Add the given request to the queue for the connection it was directed to + */ + public void add(ClientRequest request) { + Deque reqs = this.requests.get(request.request().destination()); + if (reqs == null) { + reqs = new ArrayDeque(); + this.requests.put(request.request().destination(), reqs); + } + reqs.addFirst(request); + } + + /** + * Get the request queue for the given node + */ + private Deque requestQueue(String node) { + Deque reqs = requests.get(node); + if (reqs == null || reqs.isEmpty()) + throw new IllegalStateException("Response from server for which there are no in-flight requests."); + return reqs; + } + + /** + * Get the oldest request (the one that that will be completed next) for the given node + */ + public ClientRequest completeNext(String node) { + return requestQueue(node).pollLast(); + } + + /** + * Get the last request we sent to the given node (but don't remove it from the queue) + * @param node The node id + */ + public ClientRequest lastSent(String node) { + return requestQueue(node).peekFirst(); + } + + /** + * Complete the last request that was sent to a particular node. + * @param node The node the request was sent to + * @return The request + */ + public ClientRequest completeLastSent(String node) { + return requestQueue(node).pollFirst(); + } + + /** + * Can we send more requests to this node? + * + * @param node Node in question + * @return true iff we have no requests still being sent to the given node + */ + public boolean canSendMore(String node) { + Deque queue = requests.get(node); + return queue == null || queue.isEmpty() || + (queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection); + } + + /** + * Return the number of inflight requests directed at the given node + * @param node The node + * @return The request count. + */ + public int inFlightRequestCount(String node) { + Deque queue = requests.get(node); + return queue == null ? 0 : queue.size(); + } + + /** + * Count all in-flight requests for all nodes + */ + public int inFlightRequestCount() { + int total = 0; + for (Deque deque : this.requests.values()) + total += deque.size(); + return total; + } + + /** + * Clear out all the in-flight requests for the given node and return them + * + * @param node The node + * @return All the in-flight requests for that node that have been removed + */ + public Iterable clearAll(String node) { + Deque reqs = requests.get(node); + if (reqs == null) { + return Collections.emptyList(); + } else { + return requests.remove(node); + } + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java new file mode 100644 index 0000000000000..7ab2503794ff3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -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 org.apache.kafka.clients; + +import java.io.Closeable; +import java.util.List; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.requests.RequestHeader; + +/** + * The interface for {@link NetworkClient} + */ +public interface KafkaClient extends Closeable { + + /** + * Check if we are currently ready to send another request to the given node but don't attempt to connect if we + * aren't. + * + * @param node The node to check + * @param now The current timestamp + */ + public boolean isReady(Node node, long now); + + /** + * Initiate a connection to the given node (if necessary), and return true if already connected. The readiness of a + * node will change only when poll is invoked. + * + * @param node The node to connect to. + * @param now The current time + * @return true iff we are ready to immediately initiate the sending of another request to the given node. + */ + public boolean ready(Node node, long now); + + /** + * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When + * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled + * connections. + * + * @param node The node to check + * @param now The current timestamp + * @return The number of milliseconds to wait. + */ + public long connectionDelay(Node node, long now); + + /** + * Check if the connection of the node has failed, based on the connection state. Such connection failure are + * usually transient and can be resumed in the next {@link #ready(org.apache.kafka.common.Node, long)} } + * call, but there are cases where transient failures needs to be caught and re-acted upon. + * + * @param node the node to check + * @return true iff the connection has failed and the node is disconnected + */ + public boolean connectionFailed(Node node); + + /** + * Queue up the given request for sending. Requests can only be sent on ready connections. + * + * @param request The request + */ + public void send(ClientRequest request); + + /** + * Do actual reads and writes from sockets. + * + * @param timeout The maximum amount of time to wait for responses in ms + * @param now The current time in ms + * @throws IllegalStateException If a request is sent to an unready node + */ + public List poll(long timeout, long now); + + /** + * Complete all in-flight requests for a given connection + * + * @param id The connection to complete requests for + * @param now The current time in ms + * @return All requests that complete during this time period. + */ + public List completeAll(String id, long now); + + /** + * Complete all in-flight requests + * + * @param now The current time in ms + * @return All requests that complete during this time period. + */ + public List completeAll(long now); + + /** + * Choose the node with the fewest outstanding requests. This method will prefer a node with an existing connection, + * but will potentially choose a node for which we don't yet have a connection if all existing connections are in + * use. + * + * @param now The current time in ms + * @return The node with the fewest in-flight requests. + */ + public Node leastLoadedNode(long now); + + /** + * The number of currently in-flight requests for which we have not yet returned a response + */ + public int inFlightRequestCount(); + + /** + * Get the total in-flight requests for a particular node + * + * @param nodeId The id of the node + */ + public int inFlightRequestCount(String nodeId); + + /** + * Generate a request header for the next request + * + * @param key The API key of the request + */ + public RequestHeader nextRequestHeader(ApiKeys key); + + /** + * Wake up the client if it is currently blocked waiting for I/O + */ + public void wakeup(); + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java new file mode 100644 index 0000000000000..0387f2602c93a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -0,0 +1,186 @@ +/** + * 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 org.apache.kafka.clients; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.errors.TimeoutException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A class encapsulating some of the logic around metadata. + *

+ * This class is shared by the client thread (for partitioning) and the background sender thread. + * + * Metadata is maintained for only a subset of topics, which can be added to over time. When we request metadata for a + * topic we don't have any metadata for it will trigger a metadata update. + */ +public final class Metadata { + + private static final Logger log = LoggerFactory.getLogger(Metadata.class); + + private final long refreshBackoffMs; + private final long metadataExpireMs; + private int version; + private long lastRefreshMs; + private long lastSuccessfulRefreshMs; + private Cluster cluster; + private boolean needUpdate; + private final Set topics; + + /** + * Create a metadata instance with reasonable defaults + */ + public Metadata() { + this(100L, 60 * 60 * 1000L); + } + + /** + * Create a new Metadata instance + * @param refreshBackoffMs The minimum amount of time that must expire between metadata refreshes to avoid busy + * polling + * @param metadataExpireMs The maximum amount of time that metadata can be retained without refresh + */ + public Metadata(long refreshBackoffMs, long metadataExpireMs) { + this.refreshBackoffMs = refreshBackoffMs; + this.metadataExpireMs = metadataExpireMs; + this.lastRefreshMs = 0L; + this.lastSuccessfulRefreshMs = 0L; + this.version = 0; + this.cluster = Cluster.empty(); + this.needUpdate = false; + this.topics = new HashSet(); + } + + /** + * Get the current cluster info without blocking + */ + public synchronized Cluster fetch() { + return this.cluster; + } + + /** + * Add the topic to maintain in the metadata + */ + public synchronized void add(String topic) { + topics.add(topic); + } + + /** + * The next time to update the cluster info is the maximum of the time the current info will expire and the time the + * current info can be updated (i.e. backoff time has elapsed); If an update has been request then the expiry time + * is now + */ + public synchronized long timeToNextUpdate(long nowMs) { + long timeToExpire = needUpdate ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0); + long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs; + return Math.max(timeToExpire, timeToAllowUpdate); + } + + /** + * Request an update of the current cluster metadata info, return the current version before the update + */ + public synchronized int requestUpdate() { + this.needUpdate = true; + return this.version; + } + + /** + * Wait for metadata update until the current version is larger than the last version we know of + */ + public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs) throws InterruptedException { + if (maxWaitMs < 0) { + throw new IllegalArgumentException("Max time to wait for metadata updates should not be < 0 milli seconds"); + } + long begin = System.currentTimeMillis(); + long remainingWaitMs = maxWaitMs; + while (this.version <= lastVersion) { + if (remainingWaitMs != 0) + wait(remainingWaitMs); + long elapsed = System.currentTimeMillis() - begin; + if (elapsed >= maxWaitMs) + throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms."); + remainingWaitMs = maxWaitMs - elapsed; + } + } + + /** + * Add one or more topics to maintain metadata for + */ + public synchronized void addTopics(String... topics) { + for (String topic : topics) + this.topics.add(topic); + requestUpdate(); + } + + /** + * Get the list of topics we are currently maintaining metadata for + */ + public synchronized Set topics() { + return new HashSet(this.topics); + } + + /** + * Check if a topic is already in the topic set. + * @param topic topic to check + * @return true if the topic exists, false otherwise + */ + public synchronized boolean containsTopic(String topic) { + return this.topics.contains(topic); + } + + /** + * Update the cluster metadata + */ + public synchronized void update(Cluster cluster, long now) { + this.needUpdate = false; + this.lastRefreshMs = now; + this.lastSuccessfulRefreshMs = now; + this.version += 1; + this.cluster = cluster; + notifyAll(); + log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster); + } + + /** + * Record an attempt to update the metadata that failed. We need to keep track of this + * to avoid retrying immediately. + */ + public synchronized void failedUpdate(long now) { + this.lastRefreshMs = now; + } + + /** + * @return The current metadata version + */ + public synchronized int version() { + return this.version; + } + + /** + * The last time metadata was successfully updated. + */ + public synchronized long lastSuccessfulUpdate() { + return this.lastSuccessfulRefreshMs; + } + + /** + * The metadata refresh backoff in ms + */ + public long refreshBackoff() { + return refreshBackoffMs; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java new file mode 100644 index 0000000000000..48fe7961e2215 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -0,0 +1,519 @@ +/** + * 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 org.apache.kafka.clients; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.Set; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.network.NetworkReceive; +import org.apache.kafka.common.network.Selectable; +import org.apache.kafka.common.network.Send; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.requests.ResponseHeader; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A network client for asynchronous request/response network i/o. This is an internal class used to implement the + * user-facing producer and consumer clients. + *

+ * This class is not thread-safe! + */ +public class NetworkClient implements KafkaClient { + + private static final Logger log = LoggerFactory.getLogger(NetworkClient.class); + + /* the selector used to perform network i/o */ + private final Selectable selector; + + /* the current cluster metadata */ + private final Metadata metadata; + + /* the state of each node's connection */ + private final ClusterConnectionStates connectionStates; + + /* the set of requests currently being sent or awaiting a response */ + private final InFlightRequests inFlightRequests; + + /* the socket send buffer size in bytes */ + private final int socketSendBuffer; + + /* the socket receive size buffer in bytes */ + private final int socketReceiveBuffer; + + /* the client id used to identify this client in requests to the server */ + private final String clientId; + + /* a random offset to use when choosing nodes to avoid having all nodes choose the same node */ + private final int nodeIndexOffset; + + /* the current correlation id to use when sending requests to servers */ + private int correlation; + + /* true iff there is a metadata request that has been sent and for which we have not yet received a response */ + private boolean metadataFetchInProgress; + + /* the last timestamp when no broker node is available to connect */ + private long lastNoNodeAvailableMs; + + public NetworkClient(Selectable selector, + Metadata metadata, + String clientId, + int maxInFlightRequestsPerConnection, + long reconnectBackoffMs, + int socketSendBuffer, + int socketReceiveBuffer) { + this.selector = selector; + this.metadata = metadata; + this.clientId = clientId; + this.inFlightRequests = new InFlightRequests(maxInFlightRequestsPerConnection); + this.connectionStates = new ClusterConnectionStates(reconnectBackoffMs); + this.socketSendBuffer = socketSendBuffer; + this.socketReceiveBuffer = socketReceiveBuffer; + this.correlation = 0; + this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE); + this.metadataFetchInProgress = false; + this.lastNoNodeAvailableMs = 0; + } + + /** + * Begin connecting to the given node, return true if we are already connected and ready to send to that node. + * + * @param node The node to check + * @param now The current timestamp + * @return True if we are ready to send to the given node + */ + @Override + public boolean ready(Node node, long now) { + if (isReady(node, now)) + return true; + + if (connectionStates.canConnect(node.idString(), now)) + // if we are interested in sending to a node and we don't have a connection to it, initiate one + initiateConnect(node, now); + + return false; + } + + /** + * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When + * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled + * connections. + * + * @param node The node to check + * @param now The current timestamp + * @return The number of milliseconds to wait. + */ + @Override + public long connectionDelay(Node node, long now) { + return connectionStates.connectionDelay(node.idString(), now); + } + + /** + * Check if the connection of the node has failed, based on the connection state. Such connection failure are + * usually transient and can be resumed in the next {@link #ready(org.apache.kafka.common.Node, long)} } + * call, but there are cases where transient failures needs to be caught and re-acted upon. + * + * @param node the node to check + * @return true iff the connection has failed and the node is disconnected + */ + @Override + public boolean connectionFailed(Node node) { + return connectionStates.connectionState(node.idString()).equals(ConnectionState.DISCONNECTED); + } + + /** + * Check if the node with the given id is ready to send more requests. + * + * @param node The node + * @param now The current time in ms + * @return true if the node is ready + */ + @Override + public boolean isReady(Node node, long now) { + String nodeId = node.idString(); + if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0) + // if we need to update our metadata now declare all requests unready to make metadata requests first + // priority + return false; + else + // otherwise we are ready if we are connected and can send more requests + return isSendable(nodeId); + } + + /** + * Are we connected and ready and able to send more requests to the given connection? + * + * @param node The node + */ + private boolean isSendable(String node) { + return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node); + } + + /** + * Return the state of the connection to the given node + * + * @param node The node to check + * @return The connection state + */ + public ConnectionState connectionState(String node) { + return connectionStates.connectionState(node); + } + + /** + * Queue up the given request for sending. Requests can only be sent out to ready nodes. + * + * @param request The request + */ + @Override + public void send(ClientRequest request) { + String nodeId = request.request().destination(); + if (!isSendable(nodeId)) + throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); + + this.inFlightRequests.add(request); + selector.send(request.request()); + } + + /** + * Do actual reads and writes to sockets. + * + * @param timeout The maximum amount of time to wait (in ms) for responses if there are none immediately + * @param now The current time in milliseconds + * @return The list of responses received + */ + @Override + public List poll(long timeout, long now) { + // should we update our metadata? + long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now); + long timeToNextReconnectAttempt = Math.max(this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now, 0); + long waitForMetadataFetch = this.metadataFetchInProgress ? Integer.MAX_VALUE : 0; + // if there is no node available to connect, back off refreshing metadata + long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt), + waitForMetadataFetch); + if (metadataTimeout == 0) + maybeUpdateMetadata(now); + // do the I/O + try { + this.selector.poll(Math.min(timeout, metadataTimeout)); + } catch (IOException e) { + log.error("Unexpected error during I/O in producer network thread", e); + } + + // process completed actions + List responses = new ArrayList(); + handleCompletedSends(responses, now); + handleCompletedReceives(responses, now); + handleDisconnections(responses, now); + handleConnections(); + + // invoke callbacks + for (ClientResponse response : responses) { + if (response.request().hasCallback()) { + try { + response.request().callback().onComplete(response); + } catch (Exception e) { + log.error("Uncaught error in request completion:", e); + } + } + } + + return responses; + } + + /** + * Await all the outstanding responses for requests on the given connection + * + * @param node The node to block on + * @param now The current time in ms + * @return All the collected responses + */ + @Override + public List completeAll(String node, long now) { + try { + this.selector.muteAll(); + this.selector.unmute(node); + List responses = new ArrayList(); + while (inFlightRequestCount(node) > 0) + responses.addAll(poll(Integer.MAX_VALUE, now)); + return responses; + } finally { + this.selector.unmuteAll(); + } + } + + /** + * Wait for all outstanding requests to complete. + */ + @Override + public List completeAll(long now) { + List responses = new ArrayList(); + while (inFlightRequestCount() > 0) + responses.addAll(poll(Integer.MAX_VALUE, now)); + return responses; + } + + /** + * Get the number of in-flight requests + */ + @Override + public int inFlightRequestCount() { + return this.inFlightRequests.inFlightRequestCount(); + } + + /** + * Get the number of in-flight requests for a given node + */ + @Override + public int inFlightRequestCount(String node) { + return this.inFlightRequests.inFlightRequestCount(node); + } + + /** + * Generate a request header for the given API key + * + * @param key The api key + * @return A request header with the appropriate client id and correlation id + */ + @Override + public RequestHeader nextRequestHeader(ApiKeys key) { + return new RequestHeader(key.id, clientId, correlation++); + } + + /** + * Interrupt the client if it is blocked waiting on I/O. + */ + @Override + public void wakeup() { + this.selector.wakeup(); + } + + /** + * Close the network client + */ + @Override + public void close() { + this.selector.close(); + } + + /** + * Choose the node with the fewest outstanding requests which is at least eligible for connection. This method will + * prefer a node with an existing connection, but will potentially choose a node for which we don't yet have a + * connection if all existing connections are in use. This method will never choose a node for which there is no + * existing connection and from which we have disconnected within the reconnect backoff period. + * + * @return The node with the fewest in-flight requests. + */ + public Node leastLoadedNode(long now) { + List nodes = this.metadata.fetch().nodes(); + int inflight = Integer.MAX_VALUE; + Node found = null; + for (int i = 0; i < nodes.size(); i++) { + int idx = Utils.abs((this.nodeIndexOffset + i) % nodes.size()); + Node node = nodes.get(idx); + int currInflight = this.inFlightRequests.inFlightRequestCount(node.idString()); + if (currInflight == 0 && this.connectionStates.isConnected(node.idString())) { + // if we find an established connection with no in-flight requests we can stop right away + return node; + } else if (!this.connectionStates.isBlackedOut(node.idString(), now) && currInflight < inflight) { + // otherwise if this is the best we have found so far, record that + inflight = currInflight; + found = node; + } + } + return found; + } + + /** + * Handle any completed request send. In particular if no response is expected consider the request complete. + * + * @param responses The list of responses to update + * @param now The current time + */ + private void handleCompletedSends(List responses, long now) { + // if no response is expected then when the send is completed, return it + for (Send send : this.selector.completedSends()) { + ClientRequest request = this.inFlightRequests.lastSent(send.destination()); + if (!request.expectResponse()) { + this.inFlightRequests.completeLastSent(send.destination()); + responses.add(new ClientResponse(request, now, false, null)); + } + } + } + + /** + * Handle any completed receives and update the response list with the responses received. + * + * @param responses The list of responses to update + * @param now The current time + */ + private void handleCompletedReceives(List responses, long now) { + for (NetworkReceive receive : this.selector.completedReceives()) { + String source = receive.source(); + ClientRequest req = inFlightRequests.completeNext(source); + ResponseHeader header = ResponseHeader.parse(receive.payload()); + short apiKey = req.request().header().apiKey(); + Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload()); + correlate(req.request().header(), header); + if (apiKey == ApiKeys.METADATA.id) { + handleMetadataResponse(req.request().header(), body, now); + } else { + // need to add body/header to response here + responses.add(new ClientResponse(req, now, false, body)); + } + } + } + + private void handleMetadataResponse(RequestHeader header, Struct body, long now) { + this.metadataFetchInProgress = false; + MetadataResponse response = new MetadataResponse(body); + Cluster cluster = response.cluster(); + // check if any topics metadata failed to get updated + if (response.errors().size() > 0) { + log.warn("Error while fetching metadata with correlation id {} : {}", header.correlationId(), response.errors()); + } + // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being + // created which means we will get errors and no nodes until it exists + if (cluster.nodes().size() > 0) { + this.metadata.update(cluster, now); + } else { + log.trace("Ignoring empty metadata response with correlation id {}.", header.correlationId()); + this.metadata.failedUpdate(now); + } + } + + /** + * Handle any disconnected connections + * + * @param responses The list of responses that completed with the disconnection + * @param now The current time + */ + private void handleDisconnections(List responses, long now) { + for (String node : this.selector.disconnected()) { + connectionStates.disconnected(node); + log.debug("Node {} disconnected.", node); + for (ClientRequest request : this.inFlightRequests.clearAll(node)) { + log.trace("Cancelled request {} due to node {} being disconnected", request, node); + ApiKeys requestKey = ApiKeys.forId(request.request().header().apiKey()); + if (requestKey == ApiKeys.METADATA) + metadataFetchInProgress = false; + else + responses.add(new ClientResponse(request, now, true, null)); + } + } + // we got a disconnect so we should probably refresh our metadata and see if that broker is dead + if (this.selector.disconnected().size() > 0) + this.metadata.requestUpdate(); + } + + /** + * Record any newly completed connections + */ + private void handleConnections() { + for (String node : this.selector.connected()) { + log.debug("Completed connection to node {}", node); + this.connectionStates.connected(node); + } + } + + /** + * Validate that the response corresponds to the request we expect or else explode + */ + private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) { + if (requestHeader.correlationId() != responseHeader.correlationId()) + throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() + + ") does not match request (" + requestHeader.correlationId() + ")"); + } + + /** + * Create a metadata request for the given topics + */ + private ClientRequest metadataRequest(long now, String node, Set topics) { + MetadataRequest metadata = new MetadataRequest(new ArrayList(topics)); + RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct()); + return new ClientRequest(now, true, send, null); + } + + /** + * Add a metadata request to the list of sends if we can make one + */ + private void maybeUpdateMetadata(long now) { + // Beware that the behavior of this method and the computation of timeouts for poll() are + // highly dependent on the behavior of leastLoadedNode. + Node node = this.leastLoadedNode(now); + if (node == null) { + log.debug("Give up sending metadata request since no node is available"); + // mark the timestamp for no node available to connect + this.lastNoNodeAvailableMs = now; + return; + } + String nodeConnectionId = node.idString(); + + + if (connectionStates.isConnected(nodeConnectionId) && inFlightRequests.canSendMore(nodeConnectionId)) { + Set topics = metadata.topics(); + this.metadataFetchInProgress = true; + ClientRequest metadataRequest = metadataRequest(now, nodeConnectionId, topics); + log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); + this.selector.send(metadataRequest.request()); + this.inFlightRequests.add(metadataRequest); + } else if (connectionStates.canConnect(nodeConnectionId, now)) { + // we don't have a connection to this node right now, make one + log.debug("Initialize connection to node {} for sending metadata request", node.id()); + initiateConnect(node, now); + // If initiateConnect failed immediately, this node will be put into blackout and we + // should allow immediately retrying in case there is another candidate node. If it + // is still connecting, the worst case is that we end up setting a longer timeout + // on the next round and then wait for the response. + } else { // connected, but can't send more OR connecting + // In either case, we just need to wait for a network event to let us know the selected + // connection might be usable again. + this.lastNoNodeAvailableMs = now; + } + } + + /** + * Initiate a connection to the given node + */ + private void initiateConnect(Node node, long now) { + String nodeConnectionId = node.idString(); + try { + log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port()); + this.connectionStates.connecting(nodeConnectionId, now); + selector.connect(nodeConnectionId, + new InetSocketAddress(node.host(), node.port()), + this.socketSendBuffer, + this.socketReceiveBuffer); + } catch (IOException e) { + /* attempt failed, we'll try again after the backoff */ + connectionStates.disconnected(nodeConnectionId); + /* maybe the problem is our metadata, update it */ + metadata.requestUpdate(); + log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java new file mode 100644 index 0000000000000..6fee4e45986d2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java @@ -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 org.apache.kafka.clients; + +/** + * A callback interface for attaching an action to be executed when a request is complete and the corresponding response + * has been received. This handler will also be invoked if there is a disconnection while handling the request. + */ +public interface RequestCompletionHandler { + + public void onComplete(ClientResponse response); + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java new file mode 100644 index 0000000000000..7548a9ba0163a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java @@ -0,0 +1,17 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +public enum CommitType { + SYNC, ASYNC +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java new file mode 100644 index 0000000000000..252b759c0801f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -0,0 +1,126 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.MetricName; + +/** + * @see KafkaConsumer + * @see MockConsumer + */ +public interface Consumer extends Closeable { + + /** + * @see KafkaConsumer#subscriptions() + */ + public Set subscriptions(); + + /** + * @see KafkaConsumer#subscribe(String...) + */ + public void subscribe(String... topics); + + /** + * @see KafkaConsumer#subscribe(TopicPartition...) + */ + public void subscribe(TopicPartition... partitions); + + /** + * @see KafkaConsumer#unsubscribe(String...) + */ + public void unsubscribe(String... topics); + + /** + * @see KafkaConsumer#unsubscribe(TopicPartition...) + */ + public void unsubscribe(TopicPartition... partitions); + + /** + * @see KafkaConsumer#poll(long) + */ + public ConsumerRecords poll(long timeout); + + /** + * @see KafkaConsumer#commit(CommitType) + */ + public void commit(CommitType commitType); + + /** + * @see KafkaConsumer#commit(CommitType, ConsumerCommitCallback) + */ + public void commit(CommitType commitType, ConsumerCommitCallback callback); + + /** + * @see KafkaConsumer#commit(Map, CommitType) + */ + public void commit(Map offsets, CommitType commitType); + + /** + * @see KafkaConsumer#commit(Map, CommitType, ConsumerCommitCallback) + */ + public void commit(Map offsets, CommitType commitType, ConsumerCommitCallback callback); + + /** + * @see KafkaConsumer#seek(TopicPartition, long) + */ + public void seek(TopicPartition partition, long offset); + + /** + * @see KafkaConsumer#seekToBeginning(TopicPartition...) + */ + public void seekToBeginning(TopicPartition... partitions); + + /** + * @see KafkaConsumer#seekToEnd(TopicPartition...) + */ + public void seekToEnd(TopicPartition... partitions); + + /** + * @see KafkaConsumer#position(TopicPartition) + */ + public long position(TopicPartition partition); + + /** + * @see KafkaConsumer#committed(TopicPartition) + */ + public long committed(TopicPartition partition); + + /** + * @see KafkaConsumer#metrics() + */ + public Map metrics(); + + /** + * @see KafkaConsumer#partitionsFor(String) + */ + public List partitionsFor(String topic); + + /** + * @see KafkaConsumer#close() + */ + public void close(); + + /** + * @see KafkaConsumer#wakeup() + */ + public void wakeup(); + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java new file mode 100644 index 0000000000000..f084385a37d09 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerCommitCallback.java @@ -0,0 +1,33 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; + +/** + * A callback interface that the user can implement to trigger custom actions when a commit request completes. The callback + * may be executed in any thread calling {@link Consumer#poll(long) poll()}. + */ +public interface ConsumerCommitCallback { + + /** + * A callback method the user can implement to provide asynchronous handling of commit request completion. + * This method will be called when the commit request sent to the server has been acknowledged. + * + * @param offsets A map of the offsets that this callback applies to + * @param exception The exception thrown during processing of the request, or null if the commit completed successfully + */ + void onComplete(Map offsets, Exception exception); +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java new file mode 100644 index 0000000000000..70377ae2fa46d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -0,0 +1,325 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceCallback; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.serialization.Deserializer; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; +import static org.apache.kafka.common.config.ConfigDef.ValidString.in; + +/** + * The consumer configuration keys + */ +public class ConsumerConfig extends AbstractConfig { + private static final ConfigDef CONFIG; + + /* + * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS + * THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE. + */ + + /** + * group.id + */ + public static final String GROUP_ID_CONFIG = "group.id"; + private static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using subscribe(topic) or the Kafka-based offset management strategy."; + + /** + * session.timeout.ms + */ + public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms"; + private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities."; + + /** + * bootstrap.servers + */ + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + + /** + * enable.auto.commit + */ + public static final String ENABLE_AUTO_COMMIT_CONFIG = "enable.auto.commit"; + private static final String ENABLE_AUTO_COMMIT_DOC = "If true the consumer's offset will be periodically committed in the background."; + + /** + * auto.commit.interval.ms + */ + public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms"; + private static final String AUTO_COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if enable.auto.commit is set to true."; + + /** + * partition.assignment.strategy + */ + public static final String PARTITION_ASSIGNMENT_STRATEGY_CONFIG = "partition.assignment.strategy"; + private static final String PARTITION_ASSIGNMENT_STRATEGY_DOC = "The friendly name of the partition assignment strategy that the server will use to distribute partition ownership amongst consumer instances when group management is used"; + + /** + * auto.offset.reset + */ + public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset"; + private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted):

  • smallest: automatically reset the offset to the smallest offset
  • largest: automatically reset the offset to the largest offset
  • none: throw exception to the consumer if no previous offset is found for the consumer's group
  • anything else: throw exception to the consumer.
"; + + /** + * fetch.min.bytes + */ + public static final String FETCH_MIN_BYTES_CONFIG = "fetch.min.bytes"; + private static final String FETCH_MIN_BYTES_DOC = "The minimum amount of data the server should return for a fetch request. If insufficient data is available the request will wait for that much data to accumulate before answering the request. The default setting of 1 byte means that fetch requests are answered as soon as a single byte of data is available or the fetch request times out waiting for data to arrive. Setting this to something greater than 1 will cause the server to wait for larger amounts of data to accumulate which can improve server throughput a bit at the cost of some additional latency."; + + /** + * fetch.max.wait.ms + */ + public static final String FETCH_MAX_WAIT_MS_CONFIG = "fetch.max.wait.ms"; + private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch.min.bytes."; + + /** metadata.max.age.ms */ + public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; + + /** + * max.partition.fetch.bytes + */ + public static final String MAX_PARTITION_FETCH_BYTES_CONFIG = "max.partition.fetch.bytes"; + private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server will return. The maximum total memory used for a request will be #partitions * max.partition.fetch.bytes. This size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large message on a certain partition."; + + /** send.buffer.bytes */ + public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; + + /** receive.buffer.bytes */ + public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; + + /** + * client.id + */ + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; + + /** + * reconnect.backoff.ms + */ + public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; + + /** + * retry.backoff.ms + */ + public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; + + /** + * metrics.sample.window.ms + */ + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; + + /** + * metrics.num.samples + */ + public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; + + /** + * metric.reporters + */ + public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG; + + /** + * rebalance.callback.class + */ + public static final String CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG = "rebalance.callback.class"; + private static final String CONSUMER_REBALANCE_CALLBACK_CLASS_DOC = "A user-provided callback to execute when partition assignments change."; + + /** + * check.crcs + */ + public static final String CHECK_CRCS_CONFIG = "check.crcs"; + private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance."; + + /** key.deserializer */ + public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; + private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; + + /** value.deserializer */ + public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; + private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; + + /** connections.max.idle.ms */ + public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; + + + static { + CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, + Type.LIST, + Importance.HIGH, + CommonClientConfigs.BOOSTRAP_SERVERS_DOC) + .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC) + .define(SESSION_TIMEOUT_MS_CONFIG, + Type.INT, + 30000, + Importance.HIGH, + SESSION_TIMEOUT_MS_DOC) + .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, + Type.STRING, + "range", + in("range", "roundrobin"), + Importance.MEDIUM, + PARTITION_ASSIGNMENT_STRATEGY_DOC) + .define(METADATA_MAX_AGE_CONFIG, + Type.LONG, + 5 * 60 * 1000, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METADATA_MAX_AGE_DOC) + .define(ENABLE_AUTO_COMMIT_CONFIG, + Type.BOOLEAN, + true, + Importance.MEDIUM, + ENABLE_AUTO_COMMIT_DOC) + .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, + Type.LONG, + 5000, + atLeast(0), + Importance.LOW, + AUTO_COMMIT_INTERVAL_MS_DOC) + .define(CLIENT_ID_CONFIG, + Type.STRING, + "", + Importance.LOW, + CommonClientConfigs.CLIENT_ID_DOC) + .define(MAX_PARTITION_FETCH_BYTES_CONFIG, + Type.INT, + 1 * 1024 * 1024, + atLeast(0), + Importance.HIGH, + MAX_PARTITION_FETCH_BYTES_DOC) + .define(SEND_BUFFER_CONFIG, + Type.INT, + 128 * 1024, + atLeast(0), + Importance.MEDIUM, + CommonClientConfigs.SEND_BUFFER_DOC) + .define(RECEIVE_BUFFER_CONFIG, + Type.INT, + 32 * 1024, + atLeast(0), + Importance.MEDIUM, + CommonClientConfigs.RECEIVE_BUFFER_DOC) + .define(FETCH_MIN_BYTES_CONFIG, + Type.INT, + 1024, + atLeast(0), + Importance.HIGH, + FETCH_MIN_BYTES_DOC) + .define(FETCH_MAX_WAIT_MS_CONFIG, + Type.INT, + 500, + atLeast(0), + Importance.LOW, + FETCH_MAX_WAIT_MS_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, + Type.LONG, + 50L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, + Type.LONG, + 100L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RETRY_BACKOFF_MS_DOC) + .define(AUTO_OFFSET_RESET_CONFIG, + Type.STRING, + "latest", + in("latest", "earliest", "none"), + Importance.MEDIUM, + AUTO_OFFSET_RESET_DOC) + .define(CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG, + Type.CLASS, + NoOpConsumerRebalanceCallback.class, + Importance.LOW, + CONSUMER_REBALANCE_CALLBACK_CLASS_DOC) + .define(CHECK_CRCS_CONFIG, + Type.BOOLEAN, + true, + Importance.LOW, + CHECK_CRCS_DOC) + .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, + Type.LONG, + 30000, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) + .define(METRICS_NUM_SAMPLES_CONFIG, + Type.INT, + 2, + atLeast(1), + Importance.LOW, + CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, + "", + Importance.LOW, + CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) + .define(KEY_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + KEY_DESERIALIZER_CLASS_DOC) + .define(VALUE_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + VALUE_DESERIALIZER_CLASS_DOC) + /* default is set to be a bit lower than the server default (10 min), to avoid both client and server closing connection at same time */ + .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, + Type.LONG, + 9 * 60 * 1000, + Importance.MEDIUM, + CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC); + } + + public static Map addDeserializerToConfig(Map configs, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + Map newConfigs = new HashMap(); + newConfigs.putAll(configs); + if (keyDeserializer != null) + newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass()); + if (keyDeserializer != null) + newConfigs.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass()); + return newConfigs; + } + + public static Properties addDeserializerToConfig(Properties properties, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + Properties newProperties = new Properties(); + newProperties.putAll(properties); + if (keyDeserializer != null) + newProperties.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName()); + if (keyDeserializer != null) + newProperties.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName()); + return newProperties; + } + + ConsumerConfig(Map props) { + super(CONFIG, props); + } + + public static void main(String[] args) { + System.out.println(CONFIG.toHtmlTable()); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java new file mode 100644 index 0000000000000..ff3f50f929685 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java @@ -0,0 +1,95 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import java.util.Collection; + +import org.apache.kafka.common.TopicPartition; + +/** + * A callback interface that the user can implement to trigger custom actions when the set of partitions assigned to the + * consumer changes. + *

+ * This is applicable when the consumer is having Kafka auto-manage group membership, if the consumer's directly subscribe to partitions + * those partitions will never be reassigned and this callback is not applicable. + *

+ * When Kafka is managing the group membership, a partition re-assignment will be triggered any time the members of the group changes or the subscription + * of the members changes. This can occur when processes die, new process instances are added or old instances come back to life after failure. + *

+ * There are many uses for this functionality. One common use is saving offsets in a custom store. By saving offsets in + * the {@link #onPartitionsRevoked(Consumer, Collection)} call we can ensure that any time partition assignment changes + * the offset gets saved. + *

+ * Another use is flushing out any kind of cache of intermediate results the consumer may be keeping. For example, + * consider a case where the consumer is subscribed to a topic containing user page views, and the goal is to count the + * number of page views per users for each five minute window. Let's say the topic is partitioned by the user id so that + * all events for a particular user will go to a single consumer instance. The consumer can keep in memory a running + * tally of actions per user and only flush these out to a remote data store when it's cache gets to big. However if a + * partition is reassigned it may want to automatically trigger a flush of this cache, before the new owner takes over + * consumption. + *

+ * This callback will execute in the user thread as part of the {@link Consumer#poll(long) poll(long)} call whenever partition assignment changes. + *

+ * It is guaranteed that all consumer processes will invoke {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} prior to + * any process invoking {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned}. So if offsets or other state is saved in the + * {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} call it is guaranteed to be saved by the time the process taking over that + * partition has their {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned} callback called to load the state. + *

+ * Here is pseudo-code for a callback implementation for saving offsets: + *

+ * {@code
+ *   public class SaveOffsetsOnRebalance implements ConsumerRebalanceCallback {
+ *       public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
+ *           // read the offsets from an external store using some custom code not described here
+ *           for(TopicPartition partition: partitions)
+ *              consumer.position(partition, readOffsetFromExternalStore(partition));
+ *       }      
+ *       public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
+ *           // save the offsets in an external store using some custom code not described here
+ *           for(TopicPartition partition: partitions)
+ *              saveOffsetInExternalStore(consumer.position(partition));
+ *       }
+ *   }
+ * }
+ * 
+ */ +public interface ConsumerRebalanceCallback { + + /** + * A callback method the user can implement to provide handling of customized offsets on completion of a successful + * partition re-assignement. This method will be called after an offset re-assignement completes and before the + * consumer starts fetching data. + *

+ * It is guaranteed that all the processes in a consumer group will execute their + * {@link #onPartitionsRevoked(Consumer, Collection)} callback before any instance executes its + * {@link #onPartitionsAssigned(Consumer, Collection)} callback. + * + * @param consumer Reference to the consumer for convenience + * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously + * assigned to the consumer) + */ + public void onPartitionsAssigned(Consumer consumer, Collection partitions); + + /** + * A callback method the user can implement to provide handling of offset commits to a customized store on the start + * of a rebalance operation. This method will be called before a rebalance operation starts and after the consumer + * stops fetching data. It is recommended that offsets should be committed in this callback to either Kafka or a + * custom offset store to prevent duplicate data + *

+ * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer} + * + * @param consumer Reference to the consumer for convenience + * @param partitions The list of partitions that were assigned to the consumer on the last rebalance + */ + public void onPartitionsRevoked(Consumer consumer, Collection partitions); +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java new file mode 100644 index 0000000000000..49d9527afeb25 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -0,0 +1,84 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +/** + * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the + * record is being received and an offset that points to the record in a Kafka partition. + */ +public final class ConsumerRecord { + private final String topic; + private final int partition; + private final long offset; + private final K key; + private final V value; + + /** + * Create a record with no key + * + * @param topic The topic this record is received from + * @param partition The partition of the topic this record is received from + * @param offset The offset of this record in the corresponding Kafka partition + * @param value The record contents + */ + public ConsumerRecord(String topic, int partition, long offset, K key, V value) { + if (topic == null) + throw new IllegalArgumentException("Topic cannot be null"); + this.topic = topic; + this.partition = partition; + this.offset = offset; + this.key = key; + this.value = value; + } + + /** + * The topic this record is received from + */ + public String topic() { + return this.topic; + } + + /** + * The partition from which this record is received + */ + public int partition() { + return this.partition; + } + + /** + * The key (or null if no key is specified) + */ + public K key() { + return key; + } + + /** + * The value + */ + public V value() { + return value; + } + + /** + * The position of this record in the corresponding Kafka partition. + */ + public long offset() { + return offset; + } + + @Override + public String toString() { + return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset() + + ", key = " + key + ", value = " + value + ")"; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java new file mode 100644 index 0000000000000..16a8357252d7d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.AbstractIterator; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * A container that holds the list {@link ConsumerRecord} per partition for a + * particular topic. There is one for every topic returned by a + * {@link Consumer#poll(long)} operation. + */ +public class ConsumerRecords implements Iterable> { + public static final ConsumerRecords EMPTY = + new ConsumerRecords(Collections.EMPTY_MAP); + + private final Map>> records; + + public ConsumerRecords(Map>> records) { + this.records = records; + } + + /** + * Get just the records for the given partition + * + * @param partition The partition to get records for + */ + public Iterable> records(TopicPartition partition) { + List> recs = this.records.get(partition); + if (recs == null) + return Collections.emptyList(); + else + return recs; + } + + /** + * Get just the records for the given topic + */ + public Iterable> records(String topic) { + if (topic == null) + throw new IllegalArgumentException("Topic must be non-null."); + List>> recs = new ArrayList>>(); + for (Map.Entry>> entry : records.entrySet()) { + if (entry.getKey().topic().equals(topic)) + recs.add(entry.getValue()); + } + return new ConcatenatedIterable(recs); + } + + @Override + public Iterator> iterator() { + return new ConcatenatedIterable(records.values()).iterator(); + } + + /** + * The number of records for all topics + */ + public int count() { + int count = 0; + for (List> recs: this.records.values()) + count += recs.size(); + return count; + } + + private static class ConcatenatedIterable implements Iterable> { + + private final Iterable>> iterables; + + public ConcatenatedIterable(Iterable>> iterables) { + this.iterables = iterables; + } + + @Override + public Iterator> iterator() { + return new AbstractIterator>() { + Iterator>> iters = iterables.iterator(); + Iterator> current; + + public ConsumerRecord makeNext() { + if (current == null || !current.hasNext()) { + if (iters.hasNext()) + current = iters.next().iterator(); + else + return allDone(); + } + return current.next(); + } + }; + } + } + + public boolean isEmpty() { + return records.isEmpty(); + } + + @SuppressWarnings("unchecked") + public static ConsumerRecords empty() { + return (ConsumerRecords) EMPTY; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java new file mode 100644 index 0000000000000..35f1ec9869fad --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerWakeupException.java @@ -0,0 +1,20 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.KafkaException; + +public class ConsumerWakeupException extends KafkaException { + private static final long serialVersionUID = 1L; + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java new file mode 100644 index 0000000000000..bea3d737c51be --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -0,0 +1,1121 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.NetworkClient; +import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient; +import org.apache.kafka.clients.consumer.internals.Coordinator; +import org.apache.kafka.clients.consumer.internals.DelayedTask; +import org.apache.kafka.clients.consumer.internals.Fetcher; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.JmxReporter; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetSocketAddress; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.ConcurrentModificationException; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.kafka.common.utils.Utils.min; + +/** + * A Kafka client that consumes records from a Kafka cluster. + *

+ * It will transparently handle the failure of servers in the Kafka cluster, and transparently adapt as partitions of + * data it subscribes to migrate within the cluster. This client also interacts with the server to allow groups of + * consumers to load balance consumption using consumer groups (as described below). + *

+ * The consumer maintains TCP connections to the necessary brokers to fetch data for the topics it subscribes to. + * Failure to close the consumer after use will leak these connections. + *

+ * The consumer is not thread-safe. See Multi-threaded Processing for more details. + * + *

Offsets and Consumer Position

+ * Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of + * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer + * which has position 5 has consumed records with offsets 0 through 4 and will next receive record with offset 5. There + * are actually two notions of position relevant to the user of the consumer. + *

+ * The {@link #position(TopicPartition) position} of the consumer gives the offset of the next record that will be given + * out. It will be one larger than the highest offset the consumer has seen in that partition. It automatically advances + * every time the consumer receives data calls {@link #poll(long)} and receives messages. + *

+ * The {@link #commit(CommitType) committed position} is the last offset that has been saved securely. Should the + * process fail and restart, this is the offset that it will recover to. The consumer can either automatically commit + * offsets periodically, or it can choose to control this committed position manually by calling + * {@link #commit(CommitType) commit}. + *

+ * This distinction gives the consumer control over when a record is considered consumed. It is discussed in further + * detail below. + * + *

Consumer Groups

+ * + * Kafka uses the concept of consumer groups to allow a pool of processes to divide up the work of consuming and + * processing records. These processes can either be running on the same machine or, as is more likely, they can be + * distributed over many machines to provide additional scalability and fault tolerance for processing. + *

+ * Each Kafka consumer must specify a consumer group that it belongs to. Kafka will deliver each message in the + * subscribed topics to one process in each consumer group. This is achieved by balancing the partitions in the topic + * over the consumer processes in each group. So if there is a topic with four partitions, and a consumer group with two + * processes, each process would consume from two partitions. This group membership is maintained dynamically: if a + * process fails the partitions assigned to it will be reassigned to other processes in the same group, and if a new + * process joins the group, partitions will be moved from existing consumers to this new process. + *

+ * So if two processes subscribe to a topic both specifying different groups they will each get all the records in that + * topic; if they both specify the same group they will each get about half the records. + *

+ * Conceptually you can think of a consumer group as being a single logical subscriber that happens to be made up of + * multiple processes. As a multi-subscriber system, Kafka naturally supports having any number of consumer groups for a + * given topic without duplicating data (additional consumers are actually quite cheap). + *

+ * This is a slight generalization of the functionality that is common in messaging systems. To get semantics similar to + * a queue in a traditional messaging system all processes would be part of a single consumer group and hence record + * delivery would be balanced over the group like with a queue. Unlike a traditional messaging system, though, you can + * have multiple such groups. To get semantics similar to pub-sub in a traditional messaging system each process would + * have it's own consumer group, so each process would subscribe to all the records published to the topic. + *

+ * In addition, when offsets are committed they are always committed for a given consumer group. + *

+ * It is also possible for the consumer to manually specify the partitions it subscribes to, which disables this dynamic + * partition balancing. + * + *

Usage Examples

+ * The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to + * demonstrate how to use them. + * + *

Simple Processing

+ * This example demonstrates the simplest usage of Kafka's consumer api. + * + *
+ *     Properties props = new Properties();
+ *     props.put("bootstrap.servers", "localhost:9092");
+ *     props.put("group.id", "test");
+ *     props.put("enable.auto.commit", "true");
+ *     props.put("auto.commit.interval.ms", "1000");
+ *     props.put("session.timeout.ms", "30000");
+ *     props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
+ *     props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
+ *     KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props);
+ *     consumer.subscribe("foo", "bar");
+ *     while (true) {
+ *         ConsumerRecords<String, String> records = consumer.poll(100);
+ *         for (ConsumerRecord<String, String> record : records)
+ *             System.out.printf("offset = %d, key = %s, value = %s", record.offset(), record.key(), record.value());
+ *     }
+ * 
+ * + * Setting enable.auto.commit means that offsets are committed automatically with a frequency controlled by + * the config auto.commit.interval.ms. + *

+ * The connection to the cluster is bootstrapped by specifying a list of one or more brokers to contact using the + * configuration bootstrap.servers. This list is just used to discover the rest of the brokers in the + * cluster and need not be an exhaustive list of servers in the cluster (though you may want to specify more than one in + * case there are servers down when the client is connecting). + *

+ * In this example the client is subscribing to the topics foo and bar as part of a group of consumers + * called test as described above. + *

+ * The broker will automatically detect failed processes in the test group by using a heartbeat mechanism. The + * consumer will automatically ping the cluster periodically, which let's the cluster know that it is alive. As long as + * the consumer is able to do this it is considered alive and retains the right to consume from the partitions assigned + * to it. If it stops heartbeating for a period of time longer than session.timeout.ms then it will be + * considered dead and it's partitions will be assigned to another process. + *

+ * The deserializer settings specify how to turn bytes into objects. For example, by specifying string deserializers, we + * are saying that our record's key and value will just be simple strings. + * + *

Controlling When Messages Are Considered Consumed

+ * + * In this example we will consume a batch of records and batch them up in memory, when we have sufficient records + * batched we will insert them into a database. If we allowed offsets to auto commit as in the previous example messages + * would be considered consumed after they were given out by the consumer, and it would be possible that our process + * could fail after we have read messages into our in-memory buffer but before they had been inserted into the database. + * To avoid this we will manually commit the offsets only once the corresponding messages have been inserted into the + * database. This gives us exact control of when a message is considered consumed. This raises the opposite possibility: + * the process could fail in the interval after the insert into the database but before the commit (even though this + * would likely just be a few milliseconds, it is a possibility). In this case the process that took over consumption + * would consume from last committed offset and would repeat the insert of the last batch of data. Used in this way + * Kafka provides what is often called "at-least once delivery" guarantees, as each message will likely be delivered one + * time but in failure cases could be duplicated. + * + *
+ *     Properties props = new Properties();
+ *     props.put("bootstrap.servers", "localhost:9092");
+ *     props.put("group.id", "test");
+ *     props.put("enable.auto.commit", "false");
+ *     props.put("auto.commit.interval.ms", "1000");
+ *     props.put("session.timeout.ms", "30000");
+ *     props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
+ *     props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
+ *     KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props);
+ *     consumer.subscribe("foo", "bar");
+ *     int commitInterval = 200;
+ *     List<ConsumerRecord<String, String>> buffer = new ArrayList<ConsumerRecord<String, String>>();
+ *     while (true) {
+ *         ConsumerRecords<String, String> records = consumer.poll(100);
+ *         for (ConsumerRecord<String, String> record : records) {
+ *             buffer.add(record);
+ *             if (buffer.size() >= commitInterval) {
+ *                 insertIntoDb(buffer);
+ *                 consumer.commit(CommitType.SYNC);
+ *                 buffer.clear();
+ *             }
+ *         }
+ *     }
+ * 
+ * + *

Subscribing To Specific Partitions

+ * + * In the previous examples we subscribed to the topics we were interested in and let Kafka give our particular process + * a fair share of the partitions for those topics. This provides a simple load balancing mechanism so multiple + * instances of our program can divided up the work of processing records. + *

+ * In this mode the consumer will just get the partitions it subscribes to and if the consumer instance fails no attempt + * will be made to rebalance partitions to other instances. + *

+ * There are several cases where this makes sense: + *

    + *
  • The first case is if the process is maintaining some kind of local state associated with that partition (like a + * local on-disk key-value store) and hence it should only get records for the partition it is maintaining on disk. + *
  • Another case is if the process itself is highly available and will be restarted if it fails (perhaps using a + * cluster management framework like YARN, Mesos, or AWS facilities, or as part of a stream processing framework). In + * this case there is no need for Kafka to detect the failure and reassign the partition, rather the consuming process + * will be restarted on another machine. + *
+ *

+ * This mode is easy to specify, rather than subscribing to the topic, the consumer just subscribes to particular + * partitions: + * + *

+ *     String topic = "foo";
+ *     TopicPartition partition0 = new TopicPartition(topic, 0);
+ *     TopicPartition partition1 = new TopicPartition(topic, 1);
+ *     consumer.subscribe(partition0);
+ *     consumer.subscribe(partition1);
+ * 
+ * + * The group that the consumer specifies is still used for committing offsets, but now the set of partitions will only + * be changed if the consumer specifies new partitions, and no attempt at failure detection will be made. + *

+ * It isn't possible to mix both subscription to specific partitions (with no load balancing) and to topics (with load + * balancing) using the same consumer instance. + * + *

Managing Your Own Offsets

+ * + * The consumer application need not use Kafka's built-in offset storage, it can store offsets in a store of it's own + * choosing. The primary use case for this is allowing the application to store both the offset and the results of the + * consumption in the same system in a way that both the results and offsets are stored atomically. This is not always + * possible, but when it is it will make the consumption fully atomic and give "exactly once" semantics that are + * stronger than the default "at-least once" semantics you get with Kafka's offset commit functionality. + *

+ * Here are a couple of examples of this type of usage: + *

    + *
  • If the results of the consumption are being stored in a relational database, storing the offset in the database + * as well can allow committing both the results and offset in a single transaction. Thus either the transaction will + * succeed and the offset will be updated based on what was consumed or the result will not be stored and the offset + * won't be updated. + *
  • If the results are being stored in a local store it may be possible to store the offset there as well. For + * example a search index could be built by subscribing to a particular partition and storing both the offset and the + * indexed data together. If this is done in a way that is atomic, it is often possible to have it be the case that even + * if a crash occurs that causes unsync'd data to be lost, whatever is left has the corresponding offset stored as well. + * This means that in this case the indexing process that comes back having lost recent updates just resumes indexing + * from what it has ensuring that no updates are lost. + *
+ * + * Each record comes with it's own offset, so to manage your own offset you just need to do the following: + *
    + *
  1. Configure enable.auto.commit=false + *
  2. Use the offset provided with each {@link ConsumerRecord} to save your position. + *
  3. On restart restore the position of the consumer using {@link #seek(TopicPartition, long)}. + *
+ * + * This type of usage is simplest when the partition assignment is also done manually (this would be likely in the + * search index use case described above). If the partition assignment is done automatically special care will also be + * needed to handle the case where partition assignments change. This can be handled using a special callback specified + * using rebalance.callback.class, which specifies an implementation of the interface + * {@link ConsumerRebalanceCallback}. When partitions are taken from a consumer the consumer will want to commit its + * offset for those partitions by implementing + * {@link ConsumerRebalanceCallback#onPartitionsRevoked(Consumer, Collection)}. When partitions are assigned to a + * consumer, the consumer will want to look up the offset for those new partitions an correctly initialize the consumer + * to that position by implementing {@link ConsumerRebalanceCallback#onPartitionsAssigned(Consumer, Collection)}. + *

+ * Another common use for {@link ConsumerRebalanceCallback} is to flush any caches the application maintains for + * partitions that are moved elsewhere. + * + *

Controlling The Consumer's Position

+ * + * In most use cases the consumer will simply consume records from beginning to end, periodically committing it's + * position (either automatically or manually). However Kafka allows the consumer to manually control it's position, + * moving forward or backwards in a partition at will. This means a consumer can re-consume older records, or skip to + * the most recent records without actually consuming the intermediate records. + *

+ * There are several instances where manually controlling the consumer's position can be useful. + *

+ * One case is for time-sensitive record processing it may make sense for a consumer that falls far enough behind to not + * attempt to catch up processing all records, but rather just skip to the most recent records. + *

+ * Another use case is for a system that maintains local state as described in the previous section. In such a system + * the consumer will want to initialize it's position on start-up to whatever is contained in the local store. Likewise + * if the local state is destroyed (say because the disk is lost) the state may be recreated on a new machine by + * reconsuming all the data and recreating the state (assuming that Kafka is retaining sufficient history). + * + * Kafka allows specifying the position using {@link #seek(TopicPartition, long)} to specify the new position. Special + * methods for seeking to the earliest and latest offset the server maintains are also available ( + * {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively). + * + * + *

Multi-threaded Processing

+ * + * The Kafka consumer is NOT thread-safe. All network I/O happens in the thread of the application + * making the call. It is the responsibility of the user to ensure that multi-threaded access + * is properly synchronized. Un-synchronized access will result in {@link ConcurrentModificationException}. + * + *

+ * The only exception to this rule is {@link #wakeup()}, which can safely be used from an external thread to + * interrupt an active operation. In this case, a {@link ConsumerWakeupException} will be thrown from the thread + * blocking on the operation. This can be used to shutdown the consumer from another thread. The following + * snippet shows the typical pattern: + * + *

+ * public class KafkaConsumerRunner implements Runnable {
+ *     private final AtomicBoolean closed = new AtomicBoolean(false);
+ *     private final KafkaConsumer consumer;
+ *
+ *     public void run() {
+ *         try {
+ *             consumer.subscribe("topic");
+ *             while (!closed.get()) {
+ *                 ConsumerRecords records = consumer.poll(10000);
+ *                 // Handle new records
+ *             }
+ *         } catch (ConsumerWakeupException e) {
+ *             // Ignore exception if closing
+ *             if (!closed.get()) throw e;
+ *         } finally {
+ *             consumer.close();
+ *         }
+ *     }
+ *
+ *     // Shutdown hook which can be called from a separate thread
+ *     public void shutdown() {
+ *         closed.set(true);
+ *         consumer.wakeup();
+ *     }
+ * }
+ * 
+ * + * Then in a separate thread, the consumer can be shutdown by setting the closed flag and waking up the consumer. + * + *
+ *     closed.set(true);
+ *     consumer.wakeup();
+ * 
+ * + *

+ * We have intentionally avoided implementing a particular threading model for processing. This leaves several + * options for implementing multi-threaded processing of records. + * + * + *

1. One Consumer Per Thread

+ * + * A simple option is to give each thread it's own consumer instance. Here are the pros and cons of this approach: + *
    + *
  • PRO: It is the easiest to implement + *
  • PRO: It is often the fastest as no inter-thread co-ordination is needed + *
  • PRO: It makes in-order processing on a per-partition basis very easy to implement (each thread just + * processes messages in the order it receives them). + *
  • CON: More consumers means more TCP connections to the cluster (one per thread). In general Kafka handles + * connections very efficiently so this is generally a small cost. + *
  • CON: Multiple consumers means more requests being sent to the server and slightly less batching of data + * which can cause some drop in I/O throughput. + *
  • CON: The number of total threads across all processes will be limited by the total number of partitions. + *
+ * + *

2. Decouple Consumption and Processing

+ * + * Another alternative is to have one or more consumer threads that do all data consumption and hands off + * {@link ConsumerRecords} instances to a blocking queue consumed by a pool of processor threads that actually handle + * the record processing. + * + * This option likewise has pros and cons: + *
    + *
  • PRO: This option allows independently scaling the number of consumers and processors. This makes it + * possible to have a single consumer that feeds many processor threads, avoiding any limitation on partitions. + *
  • CON: Guaranteeing order across the processors requires particular care as the threads will execute + * independently an earlier chunk of data may actually be processed after a later chunk of data just due to the luck of + * thread execution timing. For processing that has no ordering requirements this is not a problem. + *
  • CON: Manually committing the position becomes harder as it requires that all threads co-ordinate to ensure + * that processing is complete for that partition. + *
+ * + * There are many possible variations on this approach. For example each processor thread can have it's own queue, and + * the consumer threads can hash into these queues using the TopicPartition to ensure in-order consumption and simplify + * commit. + * + */ +public class KafkaConsumer implements Consumer { + + private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class); + private static final long NO_CURRENT_THREAD = -1L; + private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); + + private final Coordinator coordinator; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + private final Fetcher fetcher; + + private final Time time; + private final ConsumerNetworkClient client; + private final Metrics metrics; + private final SubscriptionState subscriptions; + private final Metadata metadata; + private final long retryBackoffMs; + private final boolean autoCommit; + private final long autoCommitIntervalMs; + private boolean closed = false; + + // currentThread holds the threadId of the current thread accessing KafkaConsumer + // and is used to prevent multi-threaded access + private final AtomicLong currentThread = new AtomicLong(NO_CURRENT_THREAD); + // refcount is used to allow reentrant access by the thread who has acquired currentThread + private final AtomicInteger refcount = new AtomicInteger(0); + + // TODO: This timeout controls how long we should wait before retrying a request. We should be able + // to leverage the work of KAFKA-2120 to get this value from configuration. + private long requestTimeoutMs = 5000L; + + /** + * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings + * are documented here. Values can be + * either strings or objects of the appropriate type (for example a numeric configuration would accept either the + * string "42" or the integer 42). + *

+ * Valid configuration strings are documented at {@link ConsumerConfig} + * + * @param configs The consumer configs + */ + public KafkaConsumer(Map configs) { + this(configs, null, null, null); + } + + /** + * A consumer is instantiated by providing a set of key-value pairs as configuration, a + * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}. + *

+ * Valid configuration strings are documented at {@link ConsumerConfig} + * + * @param configs The consumer configs + * @param callback A callback interface that the user can implement to manage customized offsets on the start and + * end of every rebalance operation. + * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. + * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. + */ + public KafkaConsumer(Map configs, + ConsumerRebalanceCallback callback, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)), + callback, + keyDeserializer, + valueDeserializer); + } + + /** + * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration. Valid + * configuration strings are documented at {@link ConsumerConfig} A consumer is instantiated by providing a + * {@link java.util.Properties} object as configuration. Valid configuration strings are documented at + * {@link ConsumerConfig} + */ + public KafkaConsumer(Properties properties) { + this(properties, null, null, null); + } + + /** + * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration and a + * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}. + *

+ * Valid configuration strings are documented at {@link ConsumerConfig} + * + * @param properties The consumer configuration properties + * @param callback A callback interface that the user can implement to manage customized offsets on the start and + * end of every rebalance operation. + * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. + * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. + */ + public KafkaConsumer(Properties properties, + ConsumerRebalanceCallback callback, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + this(new ConsumerConfig(ConsumerConfig.addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)), + callback, + keyDeserializer, + valueDeserializer); + } + + @SuppressWarnings("unchecked") + private KafkaConsumer(ConsumerConfig config, + ConsumerRebalanceCallback callback, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + try { + log.debug("Starting the Kafka consumer"); + if (callback == null) + callback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG, + ConsumerRebalanceCallback.class); + this.time = new SystemTime(); + this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); + this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); + + MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG)) + .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), + TimeUnit.MILLISECONDS); + String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG); + String jmxPrefix = "kafka.consumer"; + if (clientId.length() <= 0) + clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement(); + List reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, + MetricsReporter.class); + reporters.add(new JmxReporter(jmxPrefix)); + this.metrics = new Metrics(metricConfig, reporters, time); + this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); + this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG)); + List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); + this.metadata.update(Cluster.bootstrap(addresses), 0); + + String metricGrpPrefix = "consumer"; + Map metricsTags = new LinkedHashMap(); + metricsTags.put("client-id", clientId); + NetworkClient netClient = new NetworkClient( + new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags), + this.metadata, + clientId, + 100, // a fixed large enough value will suffice + config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG), + config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG), + config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG)); + this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs); + OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase()); + this.subscriptions = new SubscriptionState(offsetResetStrategy); + this.coordinator = new Coordinator(this.client, + config.getString(ConsumerConfig.GROUP_ID_CONFIG), + config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), + config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + this.subscriptions, + metrics, + metricGrpPrefix, + metricsTags, + this.time, + requestTimeoutMs, + retryBackoffMs, + wrapRebalanceCallback(callback)); + if (keyDeserializer == null) { + this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + Deserializer.class); + this.keyDeserializer.configure(config.originals(), true); + } else { + this.keyDeserializer = keyDeserializer; + } + if (valueDeserializer == null) { + this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + Deserializer.class); + this.valueDeserializer.configure(config.originals(), false); + } else { + this.valueDeserializer = valueDeserializer; + } + this.fetcher = new Fetcher(this.client, + config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG), + config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG), + config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG), + config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG), + this.keyDeserializer, + this.valueDeserializer, + this.metadata, + this.subscriptions, + metrics, + metricGrpPrefix, + metricsTags, + this.time, + this.retryBackoffMs); + + config.logUnused(); + + if (autoCommit) + scheduleAutoCommitTask(autoCommitIntervalMs); + + log.debug("Kafka consumer created"); + } catch (Throwable t) { + // call close methods if internal objects are already constructed + // this is to prevent resource leak. see KAFKA-2121 + close(true); + // now propagate the exception + throw new KafkaException("Failed to construct kafka consumer", t); + } + } + + /** + * The set of partitions currently assigned to this consumer. If subscription happened by directly subscribing to + * partitions using {@link #subscribe(TopicPartition...)} then this will simply return the list of partitions that + * were subscribed to. If subscription was done by specifying only the topic using {@link #subscribe(String...)} + * then this will give the set of topics currently assigned to the consumer (which may be none if the assignment + * hasn't happened yet, or the partitions are in the process of getting reassigned). + */ + public Set subscriptions() { + acquire(); + try { + return Collections.unmodifiableSet(this.subscriptions.assignedPartitions()); + } finally { + release(); + } + } + + /** + * Incrementally subscribes to the given list of topics and uses the consumer's group management functionality + *

+ * As part of group management, the consumer will keep track of the list of consumers that belong to a particular + * group and will trigger a rebalance operation if one of the following events trigger - + *

    + *
  • Number of partitions change for any of the subscribed list of topics + *
  • Topic is created or deleted + *
  • An existing member of the consumer group dies + *
  • A new member is added to an existing consumer group via the join API + *
+ * + * @param topics A variable list of topics that the consumer wants to subscribe to + */ + @Override + public void subscribe(String... topics) { + acquire(); + try { + log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", ")); + for (String topic : topics) + this.subscriptions.subscribe(topic); + metadata.addTopics(topics); + } finally { + release(); + } + } + + /** + * Incrementally subscribes to a specific topic partition and does not use the consumer's group management + * functionality. As such, there will be no rebalance operation triggered when group membership or cluster and topic + * metadata change. + *

+ * + * @param partitions Partitions to incrementally subscribe to + */ + @Override + public void subscribe(TopicPartition... partitions) { + acquire(); + try { + log.debug("Subscribed to partitions(s): {}", Utils.join(partitions, ", ")); + for (TopicPartition tp : partitions) { + this.subscriptions.subscribe(tp); + metadata.addTopics(tp.topic()); + } + } finally { + release(); + } + } + + /** + * Unsubscribe from the specific topics. This will trigger a rebalance operation and records for this topic will not + * be returned from the next {@link #poll(long) poll()} onwards + * + * @param topics Topics to unsubscribe from + */ + public void unsubscribe(String... topics) { + acquire(); + try { + log.debug("Unsubscribed from topic(s): {}", Utils.join(topics, ", ")); + // throw an exception if the topic was never subscribed to + for (String topic : topics) + this.subscriptions.unsubscribe(topic); + } finally { + release(); + } + } + + /** + * Unsubscribe from the specific topic partitions. records for these partitions will not be returned from the next + * {@link #poll(long) poll()} onwards + * + * @param partitions Partitions to unsubscribe from + */ + public void unsubscribe(TopicPartition... partitions) { + acquire(); + try { + log.debug("Unsubscribed from partitions(s): {}", Utils.join(partitions, ", ")); + // throw an exception if the partition was never subscribed to + for (TopicPartition partition : partitions) + this.subscriptions.unsubscribe(partition); + } finally { + release(); + } + } + + /** + * Fetches data for the topics or partitions specified using one of the subscribe APIs. It is an error to not have + * subscribed to any topics or partitions before polling for data. + *

+ * The offset used for fetching the data is governed by whether or not {@link #seek(TopicPartition, long)} is used. + * If {@link #seek(TopicPartition, long)} is used, it will use the specified offsets on startup and on every + * rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed + * offset using {@link #commit(Map, CommitType) commit(offsets, sync)} for the subscribed list of partitions. + * + * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, returns + * immediately with any records available now. Must not be negative. + * @return map of topic to records since the last fetch for the subscribed list of topics and partitions + * + * @throws NoOffsetForPartitionException If there is no stored offset for a subscribed partition and no automatic + * offset reset policy has been configured. + */ + @Override + public ConsumerRecords poll(long timeout) { + acquire(); + try { + if (timeout < 0) + throw new IllegalArgumentException("Timeout must not be negative"); + + // poll for new data until the timeout expires + long remaining = timeout; + while (remaining >= 0) { + long start = time.milliseconds(); + Map>> records = pollOnce(remaining); + long end = time.milliseconds(); + + if (!records.isEmpty()) { + // if data is available, then return it, but first send off the + // next round of fetches to enable pipelining while the user is + // handling the fetched records. + fetcher.initFetches(metadata.fetch()); + client.poll(0); + return new ConsumerRecords(records); + } + + remaining -= end - start; + + // nothing was available, so we should backoff before retrying + if (remaining > 0) { + Utils.sleep(min(remaining, retryBackoffMs)); + remaining -= time.milliseconds() - end; + } + } + + return ConsumerRecords.empty(); + } finally { + release(); + } + } + + /** + * Do one round of polling. In addition to checking for new data, this does any needed + * heart-beating, auto-commits, and offset updates. + * @param timeout The maximum time to block in the underlying poll + * @return The fetched records (may be empty) + */ + private Map>> pollOnce(long timeout) { + // TODO: Sub-requests should take into account the poll timeout (KAFKA-1894) + coordinator.ensureCoordinatorKnown(); + + // ensure we have partitions assigned if we expect to + if (subscriptions.partitionsAutoAssigned()) + coordinator.ensurePartitionAssignment(); + + // fetch positions if we have partitions we're subscribed to that we + // don't know the offset for + if (!subscriptions.hasAllFetchPositions()) + updateFetchPositions(this.subscriptions.missingFetchPositions()); + + // init any new fetches (won't resend pending fetches) + Cluster cluster = this.metadata.fetch(); + fetcher.initFetches(cluster); + client.poll(timeout); + return fetcher.fetchedRecords(); + } + + private void scheduleAutoCommitTask(final long interval) { + DelayedTask task = new DelayedTask() { + public void run(long now) { + commit(CommitType.ASYNC); + client.schedule(this, now + interval); + } + }; + client.schedule(task, time.milliseconds() + interval); + } + + /** + * Commits the specified offsets for the specified list of topics and partitions to Kafka. + *

+ * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every + * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + *

+ * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous + * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use + * {@link #commit(Map, CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC}) + * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown + * to the caller). + * + * @param offsets The list of offsets per partition that should be committed to Kafka. + * @param commitType Control whether the commit is blocking + */ + @Override + public void commit(final Map offsets, CommitType commitType) { + commit(offsets, commitType, null); + } + + /** + * Commits the specified offsets for the specified list of topics and partitions to Kafka. + *

+ * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every + * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + *

+ * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous + * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e. + * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In + * this case, the error is either passed to the callback (if provided) or thrown to the caller. + * + * @param offsets The list of offsets per partition that should be committed to Kafka. + * @param commitType Control whether the commit is blocking + * @param callback Callback to invoke when the commit completes + */ + @Override + public void commit(final Map offsets, CommitType commitType, ConsumerCommitCallback callback) { + acquire(); + try { + log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); + coordinator.commitOffsets(offsets, commitType, callback); + } finally { + release(); + } + } + + /** + * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions. + *

+ * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after + * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + *

+ * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous + * commit are either passed to the callback (if provided) or silently discarded. Synchronous commits (i.e. + * {@link CommitType#SYNC}) block until either the commit succeeds or an unrecoverable error is encountered. In + * this case, the error is either passed to the callback (if provided) or thrown to the caller. + * + * @param commitType Whether or not the commit should block until it is acknowledged. + * @param callback Callback to invoke when the commit completes + */ + @Override + public void commit(CommitType commitType, ConsumerCommitCallback callback) { + acquire(); + try { + // need defensive copy to ensure offsets are not removed before completion (e.g. in rebalance) + Map allConsumed = new HashMap(this.subscriptions.allConsumed()); + commit(allConsumed, commitType, callback); + } finally { + release(); + } + } + + /** + * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions. + *

+ * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after + * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + *

+ * Asynchronous commits (i.e. {@link CommitType#ASYNC} will not block. Any errors encountered during an asynchronous + * commit are silently discarded. If you need to determine the result of an asynchronous commit, you should use + * {@link #commit(CommitType, ConsumerCommitCallback)}. Synchronous commits (i.e. {@link CommitType#SYNC}) + * block until either the commit succeeds or an unrecoverable error is encountered (in which case it is thrown + * to the caller). + * + * @param commitType Whether or not the commit should block until it is acknowledged. + */ + @Override + public void commit(CommitType commitType) { + commit(commitType, null); + } + + /** + * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API + * is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that + * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets + */ + @Override + public void seek(TopicPartition partition, long offset) { + acquire(); + try { + log.debug("Seeking to offset {} for partition {}", offset, partition); + this.subscriptions.seek(partition, offset); + } finally { + release(); + } + } + + /** + * Seek to the first offset for each of the given partitions + */ + public void seekToBeginning(TopicPartition... partitions) { + acquire(); + try { + Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() + : Arrays.asList(partitions); + for (TopicPartition tp : parts) { + log.debug("Seeking to beginning of partition {}", tp); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); + } + } finally { + release(); + } + } + + /** + * Seek to the last offset for each of the given partitions + */ + public void seekToEnd(TopicPartition... partitions) { + acquire(); + try { + Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() + : Arrays.asList(partitions); + for (TopicPartition tp : parts) { + log.debug("Seeking to end of partition {}", tp); + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + } + } finally { + release(); + } + } + + /** + * Returns the offset of the next record that will be fetched (if a record with that offset exists). + * + * @param partition The partition to get the position for + * @return The offset + * @throws NoOffsetForPartitionException If a position hasn't been set for a given partition, and no reset policy is + * available. + */ + public long position(TopicPartition partition) { + acquire(); + try { + if (!this.subscriptions.assignedPartitions().contains(partition)) + throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer."); + Long offset = this.subscriptions.consumed(partition); + if (offset == null) { + updateFetchPositions(Collections.singleton(partition)); + return this.subscriptions.consumed(partition); + } else { + return offset; + } + } finally { + release(); + } + } + + /** + * Fetches the last committed offset for the given partition (whether the commit happened by this process or + * another). This offset will be used as the position for the consumer in the event of a failure. + *

+ * This call may block to do a remote call if the partition in question isn't assigned to this consumer or if the + * consumer hasn't yet initialized it's cache of committed offsets. + * + * @param partition The partition to check + * @return The last committed offset or null if no offset has been committed + * @throws NoOffsetForPartitionException If no offset has ever been committed by any process for the given + * partition. + */ + @Override + public long committed(TopicPartition partition) { + acquire(); + try { + Long committed; + if (subscriptions.assignedPartitions().contains(partition)) { + committed = this.subscriptions.committed(partition); + if (committed == null) { + coordinator.refreshCommittedOffsetsIfNeeded(); + committed = this.subscriptions.committed(partition); + } + } else { + Map offsets = coordinator.fetchCommittedOffsets(Collections.singleton(partition)); + committed = offsets.get(partition); + } + + if (committed == null) + throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); + + return committed; + } finally { + release(); + } + } + + /** + * Get the metrics kept by the consumer + */ + @Override + public Map metrics() { + return Collections.unmodifiableMap(this.metrics.metrics()); + } + + /** + * Get metadata about the partitions for a given topic. This method will issue a remote call to the server if it + * does not already have any metadata about the given topic. + * + * @param topic The topic to get partition metadata for + * @return The list of partitions + */ + @Override + public List partitionsFor(String topic) { + acquire(); + try { + Cluster cluster = this.metadata.fetch(); + List parts = cluster.partitionsForTopic(topic); + if (parts == null) { + metadata.add(topic); + client.awaitMetadataUpdate(); + parts = metadata.fetch().partitionsForTopic(topic); + } + return parts; + } finally { + release(); + } + } + + @Override + public void close() { + acquire(); + try { + if (closed) return; + close(false); + } finally { + release(); + } + } + + /** + * Wakeup the consumer. This method is thread-safe and is useful in particular to abort a long poll. + * The thread which is blocking in an operation will throw {@link ConsumerWakeupException}. + */ + @Override + public void wakeup() { + this.client.wakeup(); + } + + private void close(boolean swallowException) { + log.trace("Closing the Kafka consumer."); + AtomicReference firstException = new AtomicReference(); + this.closed = true; + ClientUtils.closeQuietly(metrics, "consumer metrics", firstException); + ClientUtils.closeQuietly(client, "consumer network client", firstException); + ClientUtils.closeQuietly(keyDeserializer, "consumer key deserializer", firstException); + ClientUtils.closeQuietly(valueDeserializer, "consumer value deserializer", firstException); + log.debug("The Kafka consumer has closed."); + if (firstException.get() != null && !swallowException) { + throw new KafkaException("Failed to close kafka consumer", firstException.get()); + } + } + + private Coordinator.RebalanceCallback wrapRebalanceCallback(final ConsumerRebalanceCallback callback) { + return new Coordinator.RebalanceCallback() { + @Override + public void onPartitionsAssigned(Collection partitions) { + callback.onPartitionsAssigned(KafkaConsumer.this, partitions); + } + + @Override + public void onPartitionsRevoked(Collection partitions) { + callback.onPartitionsRevoked(KafkaConsumer.this, partitions); + } + }; + } + + /** + * Set the fetch position to the committed position (if there is one) + * or reset it using the offset reset policy the user has configured. + * + * @param partitions The partitions that needs updating fetch positions + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is + * defined + */ + private void updateFetchPositions(Set partitions) { + // refresh commits for all assigned partitions + coordinator.refreshCommittedOffsetsIfNeeded(); + + // then do any offset lookups in case some positions are not known + fetcher.updateFetchPositions(partitions); + } + + /* + * Check that the consumer hasn't been closed. + */ + private void ensureNotClosed() { + if (this.closed) + throw new IllegalStateException("This consumer has already been closed."); + } + + /** + * Acquire the light lock protecting this consumer from multi-threaded access. Instead of blocking + * when the lock is not available, however, we just throw an exception (since multi-threaded usage is not + * supported). + * @throws IllegalStateException if the consumer has been closed + * @throws ConcurrentModificationException if another thread already has the lock + */ + private void acquire() { + ensureNotClosed(); + long threadId = Thread.currentThread().getId(); + if (threadId != currentThread.get() && !currentThread.compareAndSet(NO_CURRENT_THREAD, threadId)) + throw new ConcurrentModificationException("KafkaConsumer is not safe for multi-threaded access"); + refcount.incrementAndGet(); + } + + /** + * Release the light lock protecting the consumer from multi-threaded access. + */ + private void release() { + if (refcount.decrementAndGet() == 0) + currentThread.set(NO_CURRENT_THREAD); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java new file mode 100644 index 0000000000000..c14eed1e95f2e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -0,0 +1,200 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.MetricName; + +/** + * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. This class is not + * threadsafe + *

+ * The consumer runs in the user thread and multiplexes I/O over TCP connections to each of the brokers it needs to + * communicate with. Failure to close the consumer after use will leak these resources. + */ +public class MockConsumer implements Consumer { + + private final Map> partitions; + private final SubscriptionState subscriptions; + private Map>> records; + private boolean closed; + + public MockConsumer(OffsetResetStrategy offsetResetStrategy) { + this.subscriptions = new SubscriptionState(offsetResetStrategy); + this.partitions = new HashMap>(); + this.records = new HashMap>>(); + this.closed = false; + } + + @Override + public synchronized Set subscriptions() { + return this.subscriptions.assignedPartitions(); + } + + @Override + public synchronized void subscribe(String... topics) { + ensureNotClosed(); + for (String topic : topics) + this.subscriptions.subscribe(topic); + } + + @Override + public synchronized void subscribe(TopicPartition... partitions) { + ensureNotClosed(); + for (TopicPartition partition : partitions) + this.subscriptions.subscribe(partition); + } + + public synchronized void unsubscribe(String... topics) { + ensureNotClosed(); + for (String topic : topics) + this.subscriptions.unsubscribe(topic); + } + + public synchronized void unsubscribe(TopicPartition... partitions) { + ensureNotClosed(); + for (TopicPartition partition : partitions) + this.subscriptions.unsubscribe(partition); + } + + @Override + public synchronized ConsumerRecords poll(long timeout) { + ensureNotClosed(); + // update the consumed offset + for (Map.Entry>> entry : this.records.entrySet()) { + List> recs = entry.getValue(); + if (!recs.isEmpty()) + this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset()); + } + + ConsumerRecords copy = new ConsumerRecords(this.records); + this.records = new HashMap>>(); + return copy; + } + + public synchronized void addRecord(ConsumerRecord record) { + ensureNotClosed(); + TopicPartition tp = new TopicPartition(record.topic(), record.partition()); + this.subscriptions.assignedPartitions().add(tp); + List> recs = this.records.get(tp); + if (recs == null) { + recs = new ArrayList>(); + this.records.put(tp, recs); + } + recs.add(record); + } + + @Override + public synchronized void commit(Map offsets, CommitType commitType, ConsumerCommitCallback callback) { + ensureNotClosed(); + for (Entry entry : offsets.entrySet()) + subscriptions.committed(entry.getKey(), entry.getValue()); + if (callback != null) { + callback.onComplete(offsets, null); + } + } + + @Override + public synchronized void commit(Map offsets, CommitType commitType) { + commit(offsets, commitType, null); + } + + @Override + public synchronized void commit(CommitType commitType, ConsumerCommitCallback callback) { + ensureNotClosed(); + commit(this.subscriptions.allConsumed(), commitType, callback); + } + + @Override + public synchronized void commit(CommitType commitType) { + commit(commitType, null); + } + + @Override + public synchronized void seek(TopicPartition partition, long offset) { + ensureNotClosed(); + subscriptions.seek(partition, offset); + } + + @Override + public synchronized long committed(TopicPartition partition) { + ensureNotClosed(); + return subscriptions.committed(partition); + } + + @Override + public synchronized long position(TopicPartition partition) { + ensureNotClosed(); + return subscriptions.consumed(partition); + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + ensureNotClosed(); + throw new UnsupportedOperationException(); + } + + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + ensureNotClosed(); + throw new UnsupportedOperationException(); + } + + @Override + public Map metrics() { + ensureNotClosed(); + return Collections.emptyMap(); + } + + @Override + public synchronized List partitionsFor(String topic) { + ensureNotClosed(); + List parts = this.partitions.get(topic); + if (parts == null) + return Collections.emptyList(); + else + return parts; + } + + public synchronized void updatePartitions(String topic, List partitions) { + ensureNotClosed(); + this.partitions.put(topic, partitions); + } + + @Override + public synchronized void close() { + ensureNotClosed(); + this.closed = true; + } + + @Override + public void wakeup() { + + } + + private void ensureNotClosed() { + if (this.closed) + throw new IllegalStateException("This consumer has already been closed."); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java new file mode 100644 index 0000000000000..a21f97be5c2e4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java @@ -0,0 +1,29 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.KafkaException; + +/** + * Indicates that there is no stored offset and no defined offset reset policy + */ +public class NoOffsetForPartitionException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public NoOffsetForPartitionException(String message) { + super(message); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java new file mode 100644 index 0000000000000..542da7f7bf5c8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java @@ -0,0 +1,17 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +public enum OffsetResetStrategy { + LATEST, EARLIEST, NONE +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java new file mode 100644 index 0000000000000..9517d9d0cd480 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -0,0 +1,296 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.ConsumerWakeupException; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.utils.Time; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Higher level consumer access to the network layer with basic support for futures and + * task scheduling. NOT thread-safe! + * + * TODO: The current implementation is simplistic in that it provides a facility for queueing requests + * prior to delivery, but it makes no effort to retry requests which cannot be sent at the time + * {@link #poll(long)} is called. This makes the behavior of the queue predictable and easy to + * understand, but there are opportunities to provide timeout or retry capabilities in the future. + * How we do this may depend on KAFKA-2120, so for now, we retain the simplistic behavior. + */ +public class ConsumerNetworkClient implements Closeable { + private final KafkaClient client; + private final AtomicBoolean wakeup = new AtomicBoolean(false); + private final DelayedTaskQueue delayedTasks = new DelayedTaskQueue(); + private final Map> unsent = new HashMap>(); + private final Metadata metadata; + private final Time time; + private final long retryBackoffMs; + + public ConsumerNetworkClient(KafkaClient client, + Metadata metadata, + Time time, + long retryBackoffMs) { + this.client = client; + this.metadata = metadata; + this.time = time; + this.retryBackoffMs = retryBackoffMs; + } + + /** + * Schedule a new task to be executed at the given time. This is "best-effort" scheduling and + * should only be used for coarse synchronization. + * @param task The task to be scheduled + * @param at The time it should run + */ + public void schedule(DelayedTask task, long at) { + delayedTasks.add(task, at); + } + + /** + * Unschedule a task. This will remove all instances of the task from the task queue. + * This is a no-op if the task is not scheduled. + * @param task The task to be unscheduled. + */ + public void unschedule(DelayedTask task) { + delayedTasks.remove(task); + } + + /** + * Send a new request. Note that the request is not actually transmitted on the + * network until one of the {@link #poll(long)} variants is invoked. At this + * point the request will either be transmitted successfully or will fail. + * Use the returned future to obtain the result of the send. + * @param node The destination of the request + * @param api The Kafka API call + * @param request The request payload + * @return A future which indicates the result of the send. + */ + public RequestFuture send(Node node, + ApiKeys api, + AbstractRequest request) { + long now = time.milliseconds(); + RequestFutureCompletionHandler future = new RequestFutureCompletionHandler(); + RequestHeader header = client.nextRequestHeader(api); + RequestSend send = new RequestSend(node.idString(), header, request.toStruct()); + put(node, new ClientRequest(now, true, send, future)); + return future; + } + + private void put(Node node, ClientRequest request) { + List nodeUnsent = unsent.get(node); + if (nodeUnsent == null) { + nodeUnsent = new ArrayList(); + unsent.put(node, nodeUnsent); + } + nodeUnsent.add(request); + } + + public Node leastLoadedNode() { + return client.leastLoadedNode(time.milliseconds()); + } + + /** + * Block until the metadata has been refreshed. + */ + public void awaitMetadataUpdate() { + int version = this.metadata.requestUpdate(); + do { + poll(Long.MAX_VALUE); + } while (this.metadata.version() == version); + } + + /** + * Wakeup an active poll. This will cause the polling thread to throw an exception either + * on the current poll if one is active, or the next poll. + */ + public void wakeup() { + this.wakeup.set(true); + this.client.wakeup(); + } + + /** + * Block indefinitely until the given request future has finished. + * @param future The request future to await. + * @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread + */ + public void poll(RequestFuture future) { + while (!future.isDone()) + poll(Long.MAX_VALUE); + } + + /** + * Block until the provided request future request has finished or the timeout has expired. + * @param future The request future to wait for + * @param timeout The maximum duration (in ms) to wait for the request + * @return true if the future is done, false otherwise + * @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread + */ + public boolean poll(RequestFuture future, long timeout) { + long now = time.milliseconds(); + long deadline = now + timeout; + while (!future.isDone() && now < deadline) { + poll(deadline - now, now); + now = time.milliseconds(); + } + return future.isDone(); + } + + /** + * Poll for any network IO. All send requests will either be transmitted on the network + * or failed when this call completes. + * @param timeout The maximum time to wait for an IO event. + * @throws ConsumerWakeupException if {@link #wakeup()} is called from another thread + */ + public void poll(long timeout) { + poll(timeout, time.milliseconds()); + } + + private void poll(long timeout, long now) { + // send all the requests we can send now + pollUnsentRequests(now); + + // ensure we don't poll any longer than the deadline for + // the next scheduled task + timeout = Math.min(timeout, delayedTasks.nextTimeout(now)); + clientPoll(timeout, now); + + // execute scheduled tasks + now = time.milliseconds(); + delayedTasks.poll(now); + + // try again to send requests since buffer space may have been + // cleared or a connect finished in the poll + pollUnsentRequests(now); + + // fail all requests that couldn't be sent + clearUnsentRequests(now); + + } + + /** + * Block until all pending requests from the given node have finished. + * @param node The node to await requests from + */ + public void awaitPendingRequests(Node node) { + while (pendingRequestCount(node) > 0) + poll(retryBackoffMs); + } + + /** + * Get the count of pending requests to the given node. This includes both request that + * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission. + * @param node The node in question + * @return The number of pending requests + */ + public int pendingRequestCount(Node node) { + List pending = unsent.get(node); + int unsentCount = pending == null ? 0 : pending.size(); + return unsentCount + client.inFlightRequestCount(node.idString()); + } + + /** + * Get the total count of pending requests from all nodes. This includes both requests that + * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission. + * @return The total count of pending requests + */ + public int pendingRequestCount() { + int total = 0; + for (List requests: unsent.values()) + total += requests.size(); + return total + client.inFlightRequestCount(); + } + + private void pollUnsentRequests(long now) { + while (trySend(now)) + clientPoll(0, now); + } + + private void clearUnsentRequests(long now) { + // clear all unsent requests and fail their corresponding futures + for (Map.Entry> requestEntry: unsent.entrySet()) { + Iterator iterator = requestEntry.getValue().iterator(); + while (iterator.hasNext()) { + ClientRequest request = iterator.next(); + RequestFutureCompletionHandler handler = + (RequestFutureCompletionHandler) request.callback(); + handler.raise(SendFailedException.INSTANCE); + iterator.remove(); + } + } + unsent.clear(); + } + + private boolean trySend(long now) { + // send any requests that can be sent now + boolean requestsSent = false; + for (Map.Entry> requestEntry: unsent.entrySet()) { + Node node = requestEntry.getKey(); + Iterator iterator = requestEntry.getValue().iterator(); + while (iterator.hasNext()) { + ClientRequest request = iterator.next(); + if (client.ready(node, now)) { + client.send(request); + iterator.remove(); + requestsSent = true; + } else if (client.connectionFailed(node)) { + RequestFutureCompletionHandler handler = + (RequestFutureCompletionHandler) request.callback(); + handler.onComplete(new ClientResponse(request, now, true, null)); + iterator.remove(); + } + } + } + return requestsSent; + } + + private void clientPoll(long timeout, long now) { + client.poll(timeout, now); + if (wakeup.get()) { + clearUnsentRequests(now); + wakeup.set(false); + throw new ConsumerWakeupException(); + } + } + + @Override + public void close() throws IOException { + client.close(); + } + + public static class RequestFutureCompletionHandler + extends RequestFuture + implements RequestCompletionHandler { + + @Override + public void onComplete(ClientResponse response) { + complete(response); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java new file mode 100644 index 0000000000000..6026b23a5cb31 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -0,0 +1,791 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.consumer.CommitType; +import org.apache.kafka.clients.consumer.ConsumerCommitCallback; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Count; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ConsumerMetadataRequest; +import org.apache.kafka.common.requests.ConsumerMetadataResponse; +import org.apache.kafka.common.requests.HeartbeatRequest; +import org.apache.kafka.common.requests.HeartbeatResponse; +import org.apache.kafka.common.requests.JoinGroupRequest; +import org.apache.kafka.common.requests.JoinGroupResponse; +import org.apache.kafka.common.requests.OffsetCommitRequest; +import org.apache.kafka.common.requests.OffsetCommitResponse; +import org.apache.kafka.common.requests.OffsetFetchRequest; +import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** + * This class manages the coordination process with the consumer coordinator. + */ +public final class Coordinator { + + private static final Logger log = LoggerFactory.getLogger(Coordinator.class); + + private final ConsumerNetworkClient client; + private final Time time; + private final String groupId; + private final Heartbeat heartbeat; + private final HeartbeatTask heartbeatTask; + private final int sessionTimeoutMs; + private final String assignmentStrategy; + private final SubscriptionState subscriptions; + private final CoordinatorMetrics sensors; + private final long requestTimeoutMs; + private final long retryBackoffMs; + private final RebalanceCallback rebalanceCallback; + private Node consumerCoordinator; + private String consumerId; + private int generation; + + + /** + * Initialize the coordination manager. + */ + public Coordinator(ConsumerNetworkClient client, + String groupId, + int sessionTimeoutMs, + String assignmentStrategy, + SubscriptionState subscriptions, + Metrics metrics, + String metricGrpPrefix, + Map metricTags, + Time time, + long requestTimeoutMs, + long retryBackoffMs, + RebalanceCallback rebalanceCallback) { + + this.client = client; + this.time = time; + this.generation = -1; + this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + this.groupId = groupId; + this.consumerCoordinator = null; + this.subscriptions = subscriptions; + this.sessionTimeoutMs = sessionTimeoutMs; + this.assignmentStrategy = assignmentStrategy; + this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); + this.heartbeatTask = new HeartbeatTask(); + this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags); + this.requestTimeoutMs = requestTimeoutMs; + this.retryBackoffMs = retryBackoffMs; + this.rebalanceCallback = rebalanceCallback; + } + + /** + * Refresh the committed offsets for provided partitions. + */ + public void refreshCommittedOffsetsIfNeeded() { + if (subscriptions.refreshCommitsNeeded()) { + Map offsets = fetchCommittedOffsets(subscriptions.assignedPartitions()); + for (Map.Entry entry : offsets.entrySet()) { + TopicPartition tp = entry.getKey(); + this.subscriptions.committed(tp, entry.getValue()); + } + this.subscriptions.commitsRefreshed(); + } + } + + /** + * Fetch the current committed offsets from the coordinator for a set of partitions. + * @param partitions The partitions to fetch offsets for + * @return A map from partition to the committed offset + */ + public Map fetchCommittedOffsets(Set partitions) { + while (true) { + ensureCoordinatorKnown(); + ensurePartitionAssignment(); + + // contact coordinator to fetch committed offsets + RequestFuture> future = sendOffsetFetchRequest(partitions); + client.poll(future); + + if (future.succeeded()) + return future.value(); + + if (!future.isRetriable()) + throw future.exception(); + + Utils.sleep(retryBackoffMs); + } + } + + /** + * Ensure that we have a valid partition assignment from the coordinator. + */ + public void ensurePartitionAssignment() { + if (!subscriptions.partitionAssignmentNeeded()) + return; + + // execute the user's callback before rebalance + log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); + try { + Set revoked = new HashSet(subscriptions.assignedPartitions()); + rebalanceCallback.onPartitionsRevoked(revoked); + } catch (Exception e) { + log.error("User provided callback " + this.rebalanceCallback.getClass().getName() + + " failed on partition revocation: ", e); + } + + reassignPartitions(); + + // execute the user's callback after rebalance + log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); + try { + Set assigned = new HashSet(subscriptions.assignedPartitions()); + rebalanceCallback.onPartitionsAssigned(assigned); + } catch (Exception e) { + log.error("User provided callback " + this.rebalanceCallback.getClass().getName() + + " failed on partition assignment: ", e); + } + } + + private void reassignPartitions() { + while (subscriptions.partitionAssignmentNeeded()) { + ensureCoordinatorKnown(); + + // ensure that there are no pending requests to the coordinator. This is important + // in particular to avoid resending a pending JoinGroup request. + if (client.pendingRequestCount(this.consumerCoordinator) > 0) { + client.awaitPendingRequests(this.consumerCoordinator); + continue; + } + + RequestFuture future = sendJoinGroupRequest(); + client.poll(future); + + if (future.failed()) { + if (!future.isRetriable()) + throw future.exception(); + Utils.sleep(retryBackoffMs); + } + } + } + + /** + * Block until the coordinator for this group is known. + */ + public void ensureCoordinatorKnown() { + while (coordinatorUnknown()) { + RequestFuture future = sendConsumerMetadataRequest(); + client.poll(future, requestTimeoutMs); + + if (future.failed()) + client.awaitMetadataUpdate(); + } + } + + /** + * Commit offsets. This call blocks (regardless of commitType) until the coordinator + * can receive the commit request. Once the request has been made, however, only the + * synchronous commits will wait for a successful response from the coordinator. + * @param offsets Offsets to commit. + * @param commitType Commit policy + * @param callback Callback to be executed when the commit request finishes + */ + public void commitOffsets(Map offsets, CommitType commitType, ConsumerCommitCallback callback) { + if (commitType == CommitType.ASYNC) + commitOffsetsAsync(offsets, callback); + else + commitOffsetsSync(offsets, callback); + } + + private class HeartbeatTask implements DelayedTask { + + public void reset() { + // start or restart the heartbeat task to be executed at the next chance + long now = time.milliseconds(); + heartbeat.resetSessionTimeout(now); + client.unschedule(this); + client.schedule(this, now); + } + + @Override + public void run(final long now) { + if (!subscriptions.partitionsAutoAssigned() || + subscriptions.partitionAssignmentNeeded() || + coordinatorUnknown()) + // no need to send if we're not using auto-assignment or if we are + // awaiting a rebalance + return; + + if (heartbeat.sessionTimeoutExpired(now)) { + // we haven't received a successful heartbeat in one session interval + // so mark the coordinator dead + coordinatorDead(); + return; + } + + if (!heartbeat.shouldHeartbeat(now)) { + // we don't need to heartbeat now, so reschedule for when we do + client.schedule(this, now + heartbeat.timeToNextHeartbeat(now)); + } else { + heartbeat.sentHeartbeat(now); + RequestFuture future = sendHeartbeatRequest(); + future.addListener(new RequestFutureListener() { + @Override + public void onSuccess(Void value) { + long now = time.milliseconds(); + heartbeat.receiveHeartbeat(now); + long nextHeartbeatTime = now + heartbeat.timeToNextHeartbeat(now); + client.schedule(HeartbeatTask.this, nextHeartbeatTime); + } + + @Override + public void onFailure(RuntimeException e) { + client.schedule(HeartbeatTask.this, time.milliseconds() + retryBackoffMs); + } + }); + } + } + } + + /** + * Send a request to get a new partition assignment. This is a non-blocking call which sends + * a JoinGroup request to the coordinator (if it is available). The returned future must + * be polled to see if the request completed successfully. + * @return A request future whose completion indicates the result of the JoinGroup request. + */ + private RequestFuture sendJoinGroupRequest() { + if (coordinatorUnknown()) + return RequestFuture.coordinatorNotAvailable(); + + // send a join group request to the coordinator + List subscribedTopics = new ArrayList(subscriptions.subscribedTopics()); + log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); + + JoinGroupRequest request = new JoinGroupRequest(groupId, + this.sessionTimeoutMs, + subscribedTopics, + this.consumerId, + this.assignmentStrategy); + + // create the request for the coordinator + log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id()); + return client.send(consumerCoordinator, ApiKeys.JOIN_GROUP, request) + .compose(new JoinGroupResponseHandler()); + } + + private class JoinGroupResponseHandler extends CoordinatorResponseHandler { + + @Override + public JoinGroupResponse parse(ClientResponse response) { + return new JoinGroupResponse(response.responseBody()); + } + + @Override + public void handle(JoinGroupResponse joinResponse, RequestFuture future) { + // process the response + short errorCode = joinResponse.errorCode(); + + if (errorCode == Errors.NONE.code()) { + Coordinator.this.consumerId = joinResponse.consumerId(); + Coordinator.this.generation = joinResponse.generationId(); + + // set the flag to refresh last committed offsets + subscriptions.needRefreshCommits(); + + log.debug("Joined group: {}", joinResponse.toStruct()); + + // record re-assignment time + sensors.partitionReassignments.record(response.requestLatencyMs()); + + // update partition assignment + subscriptions.changePartitionAssignment(joinResponse.assignedPartitions()); + heartbeatTask.reset(); + future.complete(null); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) { + // reset the consumer id and retry immediately + Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.", + groupId); + future.raise(Errors.UNKNOWN_CONSUMER_ID); + } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + // re-discover the coordinator and retry with backoff + coordinatorDead(); + log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", + groupId); + future.raise(Errors.forCode(errorCode)); + } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code() + || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code() + || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) { + // log the error and re-throw the exception + Errors error = Errors.forCode(errorCode); + log.error("Attempt to join group {} failed due to: {}", + groupId, error.exception().getMessage()); + future.raise(error); + } else { + // unexpected error, throw the exception + future.raise(new KafkaException("Unexpected error in join group response: " + + Errors.forCode(joinResponse.errorCode()).exception().getMessage())); + } + } + } + + private void commitOffsetsAsync(final Map offsets, final ConsumerCommitCallback callback) { + this.subscriptions.needRefreshCommits(); + RequestFuture future = sendOffsetCommitRequest(offsets); + if (callback != null) { + future.addListener(new RequestFutureListener() { + @Override + public void onSuccess(Void value) { + callback.onComplete(offsets, null); + } + + @Override + public void onFailure(RuntimeException e) { + callback.onComplete(offsets, e); + } + }); + } + } + + private void commitOffsetsSync(Map offsets, ConsumerCommitCallback callback) { + while (true) { + ensureCoordinatorKnown(); + ensurePartitionAssignment(); + + RequestFuture future = sendOffsetCommitRequest(offsets); + client.poll(future); + + if (future.succeeded()) { + if (callback != null) + callback.onComplete(offsets, null); + return; + } + + if (!future.isRetriable()) { + if (callback == null) + throw future.exception(); + else + callback.onComplete(offsets, future.exception()); + return; + } + + Utils.sleep(retryBackoffMs); + } + } + + /** + * Commit offsets for the specified list of topics and partitions. This is a non-blocking call + * which returns a request future that can be polled in the case of a synchronous commit or ignored in the + * asynchronous case. + * + * @param offsets The list of offsets per partition that should be committed. + * @return A request future whose value indicates whether the commit was successful or not + */ + private RequestFuture sendOffsetCommitRequest(final Map offsets) { + if (coordinatorUnknown()) + return RequestFuture.coordinatorNotAvailable(); + + if (offsets.isEmpty()) + return RequestFuture.voidSuccess(); + + // create the offset commit request + Map offsetData; + offsetData = new HashMap(offsets.size()); + for (Map.Entry entry : offsets.entrySet()) + offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), "")); + OffsetCommitRequest req = new OffsetCommitRequest(this.groupId, + this.generation, + this.consumerId, + OffsetCommitRequest.DEFAULT_RETENTION_TIME, + offsetData); + + return client.send(consumerCoordinator, ApiKeys.OFFSET_COMMIT, req) + .compose(new OffsetCommitResponseHandler(offsets)); + } + + + private class OffsetCommitResponseHandler extends CoordinatorResponseHandler { + + private final Map offsets; + + public OffsetCommitResponseHandler(Map offsets) { + this.offsets = offsets; + } + + @Override + public OffsetCommitResponse parse(ClientResponse response) { + return new OffsetCommitResponse(response.responseBody()); + } + + @Override + public void handle(OffsetCommitResponse commitResponse, RequestFuture future) { + sensors.commitLatency.record(response.requestLatencyMs()); + for (Map.Entry entry : commitResponse.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + long offset = this.offsets.get(tp); + short errorCode = entry.getValue(); + if (errorCode == Errors.NONE.code()) { + log.debug("Committed offset {} for partition {}", offset, tp); + subscriptions.committed(tp, offset); + } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + future.raise(Errors.forCode(errorCode)); + return; + } else if (errorCode == Errors.OFFSET_METADATA_TOO_LARGE.code() + || errorCode == Errors.INVALID_COMMIT_OFFSET_SIZE.code()) { + // do not need to throw the exception but just log the error + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code() + || errorCode == Errors.ILLEGAL_GENERATION.code()) { + // need to re-join group + subscriptions.needReassignment(); + future.raise(Errors.forCode(errorCode)); + return; + } else { + // do not need to throw the exception but just log the error + future.raise(Errors.forCode(errorCode)); + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } + } + + future.complete(null); + } + } + + /** + * Fetch the committed offsets for a set of partitions. This is a non-blocking call. The + * returned future can be polled to get the actual offsets returned from the broker. + * + * @param partitions The set of partitions to get offsets for. + * @return A request future containing the committed offsets. + */ + private RequestFuture> sendOffsetFetchRequest(Set partitions) { + if (coordinatorUnknown()) + return RequestFuture.coordinatorNotAvailable(); + + log.debug("Fetching committed offsets for partitions: {}", Utils.join(partitions, ", ")); + // construct the request + OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); + + // send the request with a callback + return client.send(consumerCoordinator, ApiKeys.OFFSET_FETCH, request) + .compose(new OffsetFetchResponseHandler()); + } + + private class OffsetFetchResponseHandler extends CoordinatorResponseHandler> { + + @Override + public OffsetFetchResponse parse(ClientResponse response) { + return new OffsetFetchResponse(response.responseBody()); + } + + @Override + public void handle(OffsetFetchResponse response, RequestFuture> future) { + Map offsets = new HashMap(response.responseData().size()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + OffsetFetchResponse.PartitionData data = entry.getValue(); + if (data.hasError()) { + log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) + .exception() + .getMessage()); + if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { + // just retry + future.raise(Errors.OFFSET_LOAD_IN_PROGRESS); + } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + // re-discover the coordinator and retry + coordinatorDead(); + future.raise(Errors.NOT_COORDINATOR_FOR_CONSUMER); + } else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code() + || data.errorCode == Errors.ILLEGAL_GENERATION.code()) { + // need to re-join group + subscriptions.needReassignment(); + future.raise(Errors.forCode(data.errorCode)); + } else { + future.raise(new KafkaException("Unexpected error in fetch offset response: " + + Errors.forCode(data.errorCode).exception().getMessage())); + } + return; + } else if (data.offset >= 0) { + // record the position with the offset (-1 indicates no committed offset to fetch) + offsets.put(tp, data.offset); + } else { + log.debug("No committed offset for partition " + tp); + } + } + + future.complete(offsets); + } + } + + /** + * Send a heartbeat request now (visible only for testing). + */ + public RequestFuture sendHeartbeatRequest() { + HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); + return client.send(consumerCoordinator, ApiKeys.HEARTBEAT, req) + .compose(new HeartbeatCompletionHandler()); + } + + public boolean coordinatorUnknown() { + return this.consumerCoordinator == null; + } + + /** + * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to + * one of the brokers. The returned future should be polled to get the result of the request. + * @return A request future which indicates the completion of the metadata request + */ + private RequestFuture sendConsumerMetadataRequest() { + // initiate the consumer metadata request + // find a node to ask about the coordinator + Node node = this.client.leastLoadedNode(); + if (node == null) { + // TODO: If there are no brokers left, perhaps we should use the bootstrap set + // from configuration? + return RequestFuture.noBrokersAvailable(); + } else { + // create a consumer metadata request + log.debug("Issuing consumer metadata request to broker {}", node.id()); + ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId); + return client.send(node, ApiKeys.CONSUMER_METADATA, metadataRequest) + .compose(new RequestFutureAdapter() { + @Override + public void onSuccess(ClientResponse response, RequestFuture future) { + handleConsumerMetadataResponse(response, future); + } + }); + } + } + + private void handleConsumerMetadataResponse(ClientResponse resp, RequestFuture future) { + log.debug("Consumer metadata response {}", resp); + + // parse the response to get the coordinator info if it is not disconnected, + // otherwise we need to request metadata update + if (resp.wasDisconnected()) { + future.raise(new DisconnectException()); + } else if (!coordinatorUnknown()) { + // We already found the coordinator, so ignore the request + future.complete(null); + } else { + ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody()); + // use MAX_VALUE - node.id as the coordinator id to mimic separate connections + // for the coordinator in the underlying network client layer + // TODO: this needs to be better handled in KAFKA-1935 + if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) { + this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), + consumerMetadataResponse.node().host(), + consumerMetadataResponse.node().port()); + heartbeatTask.reset(); + future.complete(null); + } else { + future.raise(Errors.forCode(consumerMetadataResponse.errorCode())); + } + } + } + + /** + * Mark the current coordinator as dead. + */ + private void coordinatorDead() { + if (this.consumerCoordinator != null) { + log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id()); + this.consumerCoordinator = null; + } + } + + private class HeartbeatCompletionHandler extends CoordinatorResponseHandler { + @Override + public HeartbeatResponse parse(ClientResponse response) { + return new HeartbeatResponse(response.responseBody()); + } + + @Override + public void handle(HeartbeatResponse heartbeatResponse, RequestFuture future) { + sensors.heartbeatLatency.record(response.requestLatencyMs()); + short error = heartbeatResponse.errorCode(); + if (error == Errors.NONE.code()) { + log.debug("Received successful heartbeat response."); + future.complete(null); + } else if (error == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || error == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); + coordinatorDead(); + future.raise(Errors.forCode(error)); + } else if (error == Errors.ILLEGAL_GENERATION.code()) { + log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group."); + subscriptions.needReassignment(); + future.raise(Errors.ILLEGAL_GENERATION); + } else if (error == Errors.UNKNOWN_CONSUMER_ID.code()) { + log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group."); + consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID; + subscriptions.needReassignment(); + future.raise(Errors.UNKNOWN_CONSUMER_ID); + } else { + future.raise(new KafkaException("Unexpected error in heartbeat response: " + + Errors.forCode(error).exception().getMessage())); + } + } + } + + private abstract class CoordinatorResponseHandler + extends RequestFutureAdapter { + protected ClientResponse response; + + public abstract R parse(ClientResponse response); + + public abstract void handle(R response, RequestFuture future); + + @Override + public void onSuccess(ClientResponse clientResponse, RequestFuture future) { + this.response = clientResponse; + + if (clientResponse.wasDisconnected()) { + int correlation = response.request().request().header().correlationId(); + log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected", + response.request(), + correlation, + response.request().request().destination()); + + // mark the coordinator as dead + coordinatorDead(); + future.raise(new DisconnectException()); + return; + } + + R response = parse(clientResponse); + handle(response, future); + } + + @Override + public void onFailure(RuntimeException e, RequestFuture future) { + if (e instanceof DisconnectException) { + log.debug("Coordinator request failed", e); + coordinatorDead(); + } + future.raise(e); + } + } + + public interface RebalanceCallback { + void onPartitionsAssigned(Collection partitions); + void onPartitionsRevoked(Collection partitions); + } + + private class CoordinatorMetrics { + public final Metrics metrics; + public final String metricGrpName; + + public final Sensor commitLatency; + public final Sensor heartbeatLatency; + public final Sensor partitionReassignments; + + public CoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map tags) { + this.metrics = metrics; + this.metricGrpName = metricGrpPrefix + "-coordinator-metrics"; + + this.commitLatency = metrics.sensor("commit-latency"); + this.commitLatency.add(new MetricName("commit-latency-avg", + this.metricGrpName, + "The average time taken for a commit request", + tags), new Avg()); + this.commitLatency.add(new MetricName("commit-latency-max", + this.metricGrpName, + "The max time taken for a commit request", + tags), new Max()); + this.commitLatency.add(new MetricName("commit-rate", + this.metricGrpName, + "The number of commit calls per second", + tags), new Rate(new Count())); + + this.heartbeatLatency = metrics.sensor("heartbeat-latency"); + this.heartbeatLatency.add(new MetricName("heartbeat-response-time-max", + this.metricGrpName, + "The max time taken to receive a response to a hearbeat request", + tags), new Max()); + this.heartbeatLatency.add(new MetricName("heartbeat-rate", + this.metricGrpName, + "The average number of heartbeats per second", + tags), new Rate(new Count())); + + this.partitionReassignments = metrics.sensor("reassignment-latency"); + this.partitionReassignments.add(new MetricName("reassignment-time-avg", + this.metricGrpName, + "The average time taken for a partition reassignment", + tags), new Avg()); + this.partitionReassignments.add(new MetricName("reassignment-time-max", + this.metricGrpName, + "The max time taken for a partition reassignment", + tags), new Avg()); + this.partitionReassignments.add(new MetricName("reassignment-rate", + this.metricGrpName, + "The number of partition reassignments per second", + tags), new Rate(new Count())); + + Measurable numParts = + new Measurable() { + public double measure(MetricConfig config, long now) { + return subscriptions.assignedPartitions().size(); + } + }; + metrics.addMetric(new MetricName("assigned-partitions", + this.metricGrpName, + "The number of partitions currently assigned to this consumer", + tags), + numParts); + + Measurable lastHeartbeat = + new Measurable() { + public double measure(MetricConfig config, long now) { + return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS); + } + }; + metrics.addMetric(new MetricName("last-heartbeat-seconds-ago", + this.metricGrpName, + "The number of seconds since the last controller heartbeat", + tags), + lastHeartbeat); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java new file mode 100644 index 0000000000000..61663f8ba8cc6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTask.java @@ -0,0 +1,24 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + + +public interface DelayedTask { + + /** + * Execute the task. + * @param now current time in milliseconds + */ + void run(long now); +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java new file mode 100644 index 0000000000000..61cab20af4a1b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.java @@ -0,0 +1,96 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import java.util.Iterator; +import java.util.PriorityQueue; + +/** + * Tracks a set of tasks to be executed after a delay. + */ +public class DelayedTaskQueue { + + private PriorityQueue tasks; + + public DelayedTaskQueue() { + tasks = new PriorityQueue(); + } + + /** + * Schedule a task for execution in the future. + * + * @param task the task to execute + * @param at the time at which to + */ + public void add(DelayedTask task, long at) { + tasks.add(new Entry(task, at)); + } + + /** + * Remove a task from the queue if it is present + * @param task the task to be removed + * @returns true if a task was removed as a result of this call + */ + public boolean remove(DelayedTask task) { + boolean wasRemoved = false; + Iterator iterator = tasks.iterator(); + while (iterator.hasNext()) { + Entry entry = iterator.next(); + if (entry.task.equals(task)) { + iterator.remove(); + wasRemoved = true; + } + } + return wasRemoved; + } + + /** + * Get amount of time in milliseconds until the next event. Returns Long.MAX_VALUE if no tasks are scheduled. + * + * @return the remaining time in milliseconds + */ + public long nextTimeout(long now) { + if (tasks.isEmpty()) + return Long.MAX_VALUE; + else + return Math.max(tasks.peek().timeout - now, 0); + } + + /** + * Run any ready tasks. + * + * @param now the current time + */ + public void poll(long now) { + while (!tasks.isEmpty() && tasks.peek().timeout <= now) { + Entry entry = tasks.poll(); + entry.task.run(now); + } + } + + private static class Entry implements Comparable { + DelayedTask task; + long timeout; + + public Entry(DelayedTask task, long timeout) { + this.task = task; + this.timeout = timeout; + } + + @Override + public int compareTo(Entry entry) { + return Long.compare(timeout, entry.timeout); + } + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java new file mode 100644 index 0000000000000..d595c1cb07909 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -0,0 +1,498 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.errors.InvalidMetadataException; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Count; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.LogEntry; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.requests.FetchRequest; +import org.apache.kafka.common.requests.FetchResponse; +import org.apache.kafka.common.requests.ListOffsetRequest; +import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; + + +/** + * This class manage the fetching process with the brokers. + */ +public class Fetcher { + private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; + private static final long LATEST_OFFSET_TIMESTAMP = -1L; + + private static final Logger log = LoggerFactory.getLogger(Fetcher.class); + + private final ConsumerNetworkClient client; + private final Time time; + private final int minBytes; + private final int maxWaitMs; + private final int fetchSize; + private final long retryBackoffMs; + private final boolean checkCrcs; + private final Metadata metadata; + private final FetchManagerMetrics sensors; + private final SubscriptionState subscriptions; + private final List> records; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + + public Fetcher(ConsumerNetworkClient client, + int minBytes, + int maxWaitMs, + int fetchSize, + boolean checkCrcs, + Deserializer keyDeserializer, + Deserializer valueDeserializer, + Metadata metadata, + SubscriptionState subscriptions, + Metrics metrics, + String metricGrpPrefix, + Map metricTags, + Time time, + long retryBackoffMs) { + + this.time = time; + this.client = client; + this.metadata = metadata; + this.subscriptions = subscriptions; + this.minBytes = minBytes; + this.maxWaitMs = maxWaitMs; + this.fetchSize = fetchSize; + this.checkCrcs = checkCrcs; + + this.keyDeserializer = keyDeserializer; + this.valueDeserializer = valueDeserializer; + + this.records = new LinkedList>(); + + this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags); + this.retryBackoffMs = retryBackoffMs; + } + + /** + * Set-up a fetch request for any node that we have assigned partitions for which doesn't have one. + * + * @param cluster The current cluster metadata + */ + public void initFetches(Cluster cluster) { + for (Map.Entry fetchEntry: createFetchRequests(cluster).entrySet()) { + final FetchRequest fetch = fetchEntry.getValue(); + client.send(fetchEntry.getKey(), ApiKeys.FETCH, fetch) + .addListener(new RequestFutureListener() { + @Override + public void onSuccess(ClientResponse response) { + handleFetchResponse(response, fetch); + } + + @Override + public void onFailure(RuntimeException e) { + log.debug("Fetch failed", e); + } + }); + } + } + + /** + * Update the fetch positions for the provided partitions. + * @param partitions + */ + public void updateFetchPositions(Set partitions) { + // reset the fetch position to the committed position + for (TopicPartition tp : partitions) { + // skip if we already have a fetch position + if (subscriptions.fetched(tp) != null) + continue; + + // TODO: If there are several offsets to reset, we could submit offset requests in parallel + if (subscriptions.isOffsetResetNeeded(tp)) { + resetOffset(tp); + } else if (subscriptions.committed(tp) == null) { + // there's no committed position, so we need to reset with the default strategy + subscriptions.needOffsetReset(tp); + resetOffset(tp); + } else { + log.debug("Resetting offset for partition {} to the committed offset {}", + tp, subscriptions.committed(tp)); + subscriptions.seek(tp, subscriptions.committed(tp)); + } + } + } + + /** + * Reset offsets for the given partition using the offset reset strategy. + * + * @param partition The given partition that needs reset offset + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined + */ + private void resetOffset(TopicPartition partition) { + OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); + final long timestamp; + if (strategy == OffsetResetStrategy.EARLIEST) + timestamp = EARLIEST_OFFSET_TIMESTAMP; + else if (strategy == OffsetResetStrategy.LATEST) + timestamp = LATEST_OFFSET_TIMESTAMP; + else + throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); + + log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase()); + long offset = listOffset(partition, timestamp); + this.subscriptions.seek(partition, offset); + } + + /** + * Fetch a single offset before the given timestamp for the partition. + * + * @param partition The partition that needs fetching offset. + * @param timestamp The timestamp for fetching offset. + * @return The offset of the message that is published before the given timestamp + */ + private long listOffset(TopicPartition partition, long timestamp) { + while (true) { + RequestFuture future = sendListOffsetRequest(partition, timestamp); + client.poll(future); + + if (future.succeeded()) + return future.value(); + + if (!future.isRetriable()) + throw future.exception(); + + if (future.exception() instanceof InvalidMetadataException) + client.awaitMetadataUpdate(); + else + Utils.sleep(retryBackoffMs); + } + } + + /** + * Return the fetched records, empty the record buffer and update the consumed position. + * + * @return The fetched records per partition + */ + public Map>> fetchedRecords() { + if (this.subscriptions.partitionAssignmentNeeded()) { + return Collections.emptyMap(); + } else { + Map>> drained = new HashMap>>(); + for (PartitionRecords part : this.records) { + Long consumed = subscriptions.consumed(part.partition); + if (this.subscriptions.assignedPartitions().contains(part.partition) + && (consumed == null || part.fetchOffset == consumed)) { + List> records = drained.get(part.partition); + if (records == null) { + records = part.records; + drained.put(part.partition, records); + } else { + records.addAll(part.records); + } + subscriptions.consumed(part.partition, part.records.get(part.records.size() - 1).offset() + 1); + } else { + // these records aren't next in line based on the last consumed position, ignore them + // they must be from an obsolete request + log.debug("Ignoring fetched records for {} at offset {}", part.partition, part.fetchOffset); + } + } + this.records.clear(); + return drained; + } + } + + /** + * Fetch a single offset before the given timestamp for the partition. + * + * @param topicPartition The partition that needs fetching offset. + * @param timestamp The timestamp for fetching offset. + * @return A response which can be polled to obtain the corresponding offset. + */ + private RequestFuture sendListOffsetRequest(final TopicPartition topicPartition, long timestamp) { + Map partitions = new HashMap(1); + partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1)); + PartitionInfo info = metadata.fetch().partition(topicPartition); + if (info == null) { + metadata.add(topicPartition.topic()); + log.debug("Partition {} is unknown for fetching offset, wait for metadata refresh", topicPartition); + return RequestFuture.staleMetadata(); + } else if (info.leader() == null) { + log.debug("Leader for partition {} unavailable for fetching offset, wait for metadata refresh", topicPartition); + return RequestFuture.leaderNotAvailable(); + } else { + Node node = info.leader(); + ListOffsetRequest request = new ListOffsetRequest(-1, partitions); + return client.send(node, ApiKeys.LIST_OFFSETS, request) + .compose(new RequestFutureAdapter() { + @Override + public void onSuccess(ClientResponse response, RequestFuture future) { + handleListOffsetResponse(topicPartition, response, future); + } + }); + } + } + + /** + * Callback for the response of the list offset call above. + * @param topicPartition The partition that was fetched + * @param clientResponse The response from the server. + */ + private void handleListOffsetResponse(TopicPartition topicPartition, + ClientResponse clientResponse, + RequestFuture future) { + if (clientResponse.wasDisconnected()) { + future.raise(new DisconnectException()); + } else { + ListOffsetResponse lor = new ListOffsetResponse(clientResponse.responseBody()); + short errorCode = lor.responseData().get(topicPartition).errorCode; + if (errorCode == Errors.NONE.code()) { + List offsets = lor.responseData().get(topicPartition).offsets; + if (offsets.size() != 1) + throw new IllegalStateException("This should not happen."); + long offset = offsets.get(0); + log.debug("Fetched offset {} for partition {}", offset, topicPartition); + + future.complete(offset); + } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() + || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { + log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", + topicPartition); + future.raise(Errors.forCode(errorCode)); + } else { + log.error("Attempt to fetch offsets for partition {} failed due to: {}", + topicPartition, Errors.forCode(errorCode).exception().getMessage()); + future.raise(new StaleMetadataException()); + } + } + } + + /** + * Create fetch requests for all nodes for which we have assigned partitions + * that have no existing requests in flight. + */ + private Map createFetchRequests(Cluster cluster) { + // create the fetch info + Map> fetchable = new HashMap>(); + for (TopicPartition partition : subscriptions.assignedPartitions()) { + Node node = cluster.leaderFor(partition); + if (node == null) { + metadata.requestUpdate(); + } else if (this.client.pendingRequestCount(node) == 0) { + // if there is a leader and no in-flight requests, issue a new fetch + Map fetch = fetchable.get(node); + if (fetch == null) { + fetch = new HashMap(); + fetchable.put(node, fetch); + } + long offset = this.subscriptions.fetched(partition); + fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize)); + } + } + + // create the fetches + Map requests = new HashMap(); + for (Map.Entry> entry : fetchable.entrySet()) { + Node node = entry.getKey(); + FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); + requests.put(node, fetch); + } + return requests; + } + + /** + * The callback for fetch completion + */ + private void handleFetchResponse(ClientResponse resp, FetchRequest request) { + if (resp.wasDisconnected()) { + int correlation = resp.request().request().header().correlationId(); + log.debug("Cancelled fetch request {} with correlation id {} due to node {} being disconnected", + resp.request(), correlation, resp.request().request().destination()); + } else { + int totalBytes = 0; + int totalCount = 0; + FetchResponse response = new FetchResponse(resp.responseBody()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + FetchResponse.PartitionData partition = entry.getValue(); + if (!subscriptions.assignedPartitions().contains(tp)) { + log.debug("Ignoring fetched data for partition {} which is no longer assigned.", tp); + } else if (partition.errorCode == Errors.NONE.code()) { + int bytes = 0; + ByteBuffer buffer = partition.recordSet; + MemoryRecords records = MemoryRecords.readableRecords(buffer); + long fetchOffset = request.fetchData().get(tp).offset; + List> parsed = new ArrayList>(); + for (LogEntry logEntry : records) { + parsed.add(parseRecord(tp, logEntry)); + bytes += logEntry.size(); + } + if (parsed.size() > 0) { + ConsumerRecord record = parsed.get(parsed.size() - 1); + this.subscriptions.fetched(tp, record.offset() + 1); + this.records.add(new PartitionRecords(fetchOffset, tp, parsed)); + this.sensors.recordsFetchLag.record(partition.highWatermark - record.offset()); + } + this.sensors.recordTopicFetchMetrics(tp.topic(), bytes, parsed.size()); + totalBytes += bytes; + totalCount += parsed.size(); + } else if (partition.errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() + || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { + this.metadata.requestUpdate(); + } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { + // TODO: this could be optimized by grouping all out-of-range partitions + log.info("Fetch offset {} is out of range, resetting offset", subscriptions.fetched(tp)); + subscriptions.needOffsetReset(tp); + } else if (partition.errorCode == Errors.UNKNOWN.code()) { + log.warn("Unknown error fetching data for topic-partition {}", tp); + } else { + throw new IllegalStateException("Unexpected error code " + partition.errorCode + " while fetching data"); + } + } + this.sensors.bytesFetched.record(totalBytes); + this.sensors.recordsFetched.record(totalCount); + } + this.sensors.fetchLatency.record(resp.requestLatencyMs()); + } + + /** + * Parse the record entry, deserializing the key / value fields if necessary + */ + private ConsumerRecord parseRecord(TopicPartition partition, LogEntry logEntry) { + if (this.checkCrcs) + logEntry.record().ensureValid(); + + long offset = logEntry.offset(); + ByteBuffer keyBytes = logEntry.record().key(); + K key = keyBytes == null ? null : this.keyDeserializer.deserialize(partition.topic(), Utils.toArray(keyBytes)); + ByteBuffer valueBytes = logEntry.record().value(); + V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), Utils.toArray(valueBytes)); + + return new ConsumerRecord(partition.topic(), partition.partition(), offset, key, value); + } + + private static class PartitionRecords { + public long fetchOffset; + public TopicPartition partition; + public List> records; + + public PartitionRecords(long fetchOffset, TopicPartition partition, List> records) { + this.fetchOffset = fetchOffset; + this.partition = partition; + this.records = records; + } + } + + private class FetchManagerMetrics { + public final Metrics metrics; + public final String metricGrpName; + + public final Sensor bytesFetched; + public final Sensor recordsFetched; + public final Sensor fetchLatency; + public final Sensor recordsFetchLag; + + + public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix, Map tags) { + this.metrics = metrics; + this.metricGrpName = metricGrpPrefix + "-fetch-manager-metrics"; + + this.bytesFetched = metrics.sensor("bytes-fetched"); + this.bytesFetched.add(new MetricName("fetch-size-avg", + this.metricGrpName, + "The average number of bytes fetched per request", + tags), new Avg()); + this.bytesFetched.add(new MetricName("fetch-size-max", + this.metricGrpName, + "The maximum number of bytes fetched per request", + tags), new Max()); + this.bytesFetched.add(new MetricName("bytes-consumed-rate", + this.metricGrpName, + "The average number of bytes consumed per second", + tags), new Rate()); + + this.recordsFetched = metrics.sensor("records-fetched"); + this.recordsFetched.add(new MetricName("records-per-request-avg", + this.metricGrpName, + "The average number of records in each request", + tags), new Avg()); + this.recordsFetched.add(new MetricName("records-consumed-rate", + this.metricGrpName, + "The average number of records consumed per second", + tags), new Rate()); + + this.fetchLatency = metrics.sensor("fetch-latency"); + this.fetchLatency.add(new MetricName("fetch-latency-avg", + this.metricGrpName, + "The average time taken for a fetch request.", + tags), new Avg()); + this.fetchLatency.add(new MetricName("fetch-latency-max", + this.metricGrpName, + "The max time taken for any fetch request.", + tags), new Max()); + this.fetchLatency.add(new MetricName("fetch-rate", + this.metricGrpName, + "The number of fetch requests per second.", + tags), new Rate(new Count())); + + this.recordsFetchLag = metrics.sensor("records-lag"); + this.recordsFetchLag.add(new MetricName("records-lag-max", + this.metricGrpName, + "The maximum lag in terms of number of records for any partition in this window", + tags), new Max()); + } + + public void recordTopicFetchMetrics(String topic, int bytes, int records) { + // record bytes fetched + String name = "topic." + topic + ".bytes-fetched"; + Sensor bytesFetched = this.metrics.getSensor(name); + if (bytesFetched == null) + bytesFetched = this.metrics.sensor(name); + bytesFetched.record(bytes); + + // record records fetched + name = "topic." + topic + ".records-fetched"; + Sensor recordsFetched = this.metrics.getSensor(name); + if (recordsFetched == null) + recordsFetched = this.metrics.sensor(name); + recordsFetched.record(records); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java new file mode 100644 index 0000000000000..6da89368a1fff --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -0,0 +1,74 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +/** + * A helper class for managing the heartbeat to the coordinator + */ +public final class Heartbeat { + + /* The number of heartbeats to attempt to complete per session timeout interval. + * so, e.g., with a session timeout of 3 seconds we would attempt a heartbeat + * once per second. + */ + public final static int HEARTBEATS_PER_SESSION_INTERVAL = 3; + + private final long timeout; + private long lastHeartbeatSend; + private long lastHeartbeatReceive; + private long lastSessionReset; + + public Heartbeat(long timeout, long now) { + this.timeout = timeout; + this.lastSessionReset = now; + } + + public void sentHeartbeat(long now) { + this.lastHeartbeatSend = now; + } + + public void receiveHeartbeat(long now) { + this.lastHeartbeatReceive = now; + } + + public boolean shouldHeartbeat(long now) { + return timeToNextHeartbeat(now) == 0; + } + + public long lastHeartbeatSend() { + return this.lastHeartbeatSend; + } + + public long timeToNextHeartbeat(long now) { + long timeSinceLastHeartbeat = now - Math.max(lastHeartbeatSend, lastSessionReset); + + long hbInterval = timeout / HEARTBEATS_PER_SESSION_INTERVAL; + if (timeSinceLastHeartbeat > hbInterval) + return 0; + else + return hbInterval - timeSinceLastHeartbeat; + } + + public boolean sessionTimeoutExpired(long now) { + return now - Math.max(lastSessionReset, lastHeartbeatReceive) > timeout; + } + + public long interval() { + return timeout / HEARTBEATS_PER_SESSION_INTERVAL; + } + + public void resetSessionTimeout(long now) { + this.lastSessionReset = now; + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java new file mode 100644 index 0000000000000..0ec6017b345db --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java @@ -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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.errors.InvalidMetadataException; + +/** + * No brokers were available to complete a request. + */ +public class NoAvailableBrokersException extends InvalidMetadataException { + private static final long serialVersionUID = 1L; + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java new file mode 100644 index 0000000000000..c06ab3a372a54 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java @@ -0,0 +1,30 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; +import org.apache.kafka.common.TopicPartition; + +public class NoOpConsumerRebalanceCallback implements ConsumerRebalanceCallback { + + @Override + public void onPartitionsAssigned(Consumer consumer, Collection partitions) {} + + @Override + public void onPartitionsRevoked(Consumer consumer, Collection partitions) {} + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java new file mode 100644 index 0000000000000..5f0025144a6af --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java @@ -0,0 +1,202 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.protocol.Errors; + +import java.util.ArrayList; +import java.util.List; + +/** + * Result of an asynchronous request from {@link ConsumerNetworkClient}. Use {@link ConsumerNetworkClient#poll(long)} + * (and variants) to finish a request future. Use {@link #isDone()} to check if the future is complete, and + * {@link #succeeded()} to check if the request completed successfully. Typical usage might look like this: + * + *

+ *     RequestFuture future = client.send(api, request);
+ *     client.poll(future);
+ *
+ *     if (future.succeeded()) {
+ *         ClientResponse response = future.value();
+ *         // Handle response
+ *     } else {
+ *         throw future.exception();
+ *     }
+ * 
+ * + * @param Return type of the result (Can be Void if there is no response) + */ +public class RequestFuture { + + private boolean isDone = false; + private T value; + private RuntimeException exception; + private List> listeners = new ArrayList>(); + + + /** + * Check whether the response is ready to be handled + * @return true if the response is ready, false otherwise + */ + public boolean isDone() { + return isDone; + } + + /** + * Get the value corresponding to this request (only available if the request succeeded) + * @return the value if it exists or null + */ + public T value() { + return value; + } + + /** + * Check if the request succeeded; + * @return true if the request completed and was successful + */ + public boolean succeeded() { + return isDone && exception == null; + } + + /** + * Check if the request failed. + * @return true if the request completed with a failure + */ + public boolean failed() { + return isDone && exception != null; + } + + /** + * Check if the request is retriable (convenience method for checking if + * the exception is an instance of {@link RetriableException}. + * @return true if it is retriable, false otherwise + */ + public boolean isRetriable() { + return exception instanceof RetriableException; + } + + /** + * Get the exception from a failed result (only available if the request failed) + * @return The exception if it exists or null + */ + public RuntimeException exception() { + return exception; + } + + /** + * Complete the request successfully. After this call, {@link #succeeded()} will return true + * and the value can be obtained through {@link #value()}. + * @param value corresponding value (or null if there is none) + */ + public void complete(T value) { + this.value = value; + this.isDone = true; + fireSuccess(); + } + + /** + * Raise an exception. The request will be marked as failed, and the caller can either + * handle the exception or throw it. + * @param e corresponding exception to be passed to caller + */ + public void raise(RuntimeException e) { + this.exception = e; + this.isDone = true; + fireFailure(); + } + + /** + * Raise an error. The request will be marked as failed. + * @param error corresponding error to be passed to caller + */ + public void raise(Errors error) { + raise(error.exception()); + } + + private void fireSuccess() { + for (RequestFutureListener listener: listeners) + listener.onSuccess(value); + } + + private void fireFailure() { + for (RequestFutureListener listener: listeners) + listener.onFailure(exception); + } + + /** + * Add a listener which will be notified when the future completes + * @param listener + */ + public void addListener(RequestFutureListener listener) { + if (isDone) { + if (exception != null) + listener.onFailure(exception); + else + listener.onSuccess(value); + } else { + this.listeners.add(listener); + } + } + + /** + * Convert from a request future of one type to another type + * @param adapter The adapter which does the conversion + * @param The type of the future adapted to + * @return The new future + */ + public RequestFuture compose(final RequestFutureAdapter adapter) { + final RequestFuture adapted = new RequestFuture(); + addListener(new RequestFutureListener() { + @Override + public void onSuccess(T value) { + adapter.onSuccess(value, adapted); + } + + @Override + public void onFailure(RuntimeException e) { + adapter.onFailure(e, adapted); + } + }); + return adapted; + } + + public static RequestFuture failure(RuntimeException e) { + RequestFuture future = new RequestFuture(); + future.raise(e); + return future; + } + + public static RequestFuture voidSuccess() { + RequestFuture future = new RequestFuture(); + future.complete(null); + return future; + } + + public static RequestFuture coordinatorNotAvailable() { + return failure(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception()); + } + + public static RequestFuture leaderNotAvailable() { + return failure(Errors.LEADER_NOT_AVAILABLE.exception()); + } + + public static RequestFuture noBrokersAvailable() { + return failure(new NoAvailableBrokersException()); + } + + public static RequestFuture staleMetadata() { + return failure(new StaleMetadataException()); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java new file mode 100644 index 0000000000000..cc5322ff0b57a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +/** + * Adapt from a request future of one type to another. + * + * @param Type to adapt from + * @param Type to adapt to + */ +public abstract class RequestFutureAdapter { + + public abstract void onSuccess(F value, RequestFuture future); + + public void onFailure(RuntimeException e, RequestFuture future) { + future.raise(e); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java new file mode 100644 index 0000000000000..b39261b70b51f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java @@ -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 org.apache.kafka.clients.consumer.internals; + +/** + * Listener interface to hook into RequestFuture completion. + */ +public interface RequestFutureListener { + + void onSuccess(T value); + + void onFailure(RuntimeException e); +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java new file mode 100644 index 0000000000000..3312a2c5e6b74 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java @@ -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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.errors.RetriableException; + +/** + * Exception used in {@link ConsumerNetworkClient} to indicate the failure + * to transmit a request to the networking layer. This could be either because + * the client is still connecting to the given host or its send buffer is full. + */ +public class SendFailedException extends RetriableException { + public static final SendFailedException INSTANCE = new SendFailedException(); + + private static final long serialVersionUID = 1L; + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java new file mode 100644 index 0000000000000..09114cb33745b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.errors.InvalidMetadataException; + +/** + * Thrown when metadata is old and needs to be refreshed. + */ +public class StaleMetadataException extends InvalidMetadataException { + private static final long serialVersionUID = 1L; +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java new file mode 100644 index 0000000000000..4d9a425201115 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -0,0 +1,233 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A class for tracking the topics, partitions, and offsets for the consumer + */ +public class SubscriptionState { + + /* the list of topics the user has requested */ + private final Set subscribedTopics; + + /* the list of partitions the user has requested */ + private final Set subscribedPartitions; + + /* the list of partitions currently assigned */ + private final Set assignedPartitions; + + /* the offset exposed to the user */ + private final Map consumed; + + /* the current point we have fetched up to */ + private final Map fetched; + + /* the last committed offset for each partition */ + private final Map committed; + + /* do we need to request a partition assignment from the coordinator? */ + private boolean needsPartitionAssignment; + + /* do we need to request the latest committed offsets from the coordinator? */ + private boolean needsFetchCommittedOffsets; + + /* Partitions that need to be reset before fetching */ + private Map resetPartitions; + + /* Default offset reset strategy */ + private OffsetResetStrategy offsetResetStrategy; + + public SubscriptionState(OffsetResetStrategy offsetResetStrategy) { + this.offsetResetStrategy = offsetResetStrategy; + this.subscribedTopics = new HashSet(); + this.subscribedPartitions = new HashSet(); + this.assignedPartitions = new HashSet(); + this.consumed = new HashMap(); + this.fetched = new HashMap(); + this.committed = new HashMap(); + this.needsPartitionAssignment = false; + this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up + this.resetPartitions = new HashMap(); + } + + public void subscribe(String topic) { + if (this.subscribedPartitions.size() > 0) + throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive"); + if (!this.subscribedTopics.contains(topic)) { + this.subscribedTopics.add(topic); + this.needsPartitionAssignment = true; + } + } + + public void unsubscribe(String topic) { + if (!this.subscribedTopics.contains(topic)) + throw new IllegalStateException("Topic " + topic + " was never subscribed to."); + this.subscribedTopics.remove(topic); + this.needsPartitionAssignment = true; + for (TopicPartition tp: assignedPartitions()) + if (topic.equals(tp.topic())) + clearPartition(tp); + } + + public void needReassignment() { + this.needsPartitionAssignment = true; + } + + public void subscribe(TopicPartition tp) { + if (this.subscribedTopics.size() > 0) + throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive"); + this.subscribedPartitions.add(tp); + this.assignedPartitions.add(tp); + } + + public void unsubscribe(TopicPartition partition) { + if (!subscribedPartitions.contains(partition)) + throw new IllegalStateException("Partition " + partition + " was never subscribed to."); + subscribedPartitions.remove(partition); + clearPartition(partition); + } + + private void clearPartition(TopicPartition tp) { + this.assignedPartitions.remove(tp); + this.committed.remove(tp); + this.fetched.remove(tp); + this.consumed.remove(tp); + this.resetPartitions.remove(tp); + } + + public void clearAssignment() { + this.assignedPartitions.clear(); + this.committed.clear(); + this.fetched.clear(); + this.consumed.clear(); + this.needsPartitionAssignment = !subscribedTopics().isEmpty(); + } + + public Set subscribedTopics() { + return this.subscribedTopics; + } + + public Long fetched(TopicPartition tp) { + return this.fetched.get(tp); + } + + public void fetched(TopicPartition tp, long offset) { + if (!this.assignedPartitions.contains(tp)) + throw new IllegalArgumentException("Can't change the fetch position for a partition you are not currently subscribed to."); + this.fetched.put(tp, offset); + } + + public void committed(TopicPartition tp, long offset) { + this.committed.put(tp, offset); + } + + public Long committed(TopicPartition tp) { + return this.committed.get(tp); + } + + public void needRefreshCommits() { + this.needsFetchCommittedOffsets = true; + } + + public boolean refreshCommitsNeeded() { + return this.needsFetchCommittedOffsets; + } + + public void commitsRefreshed() { + this.needsFetchCommittedOffsets = false; + } + + public void seek(TopicPartition tp, long offset) { + fetched(tp, offset); + consumed(tp, offset); + resetPartitions.remove(tp); + } + + public Set assignedPartitions() { + return this.assignedPartitions; + } + + public boolean partitionsAutoAssigned() { + return !this.subscribedTopics.isEmpty(); + } + + public void consumed(TopicPartition tp, long offset) { + if (!this.assignedPartitions.contains(tp)) + throw new IllegalArgumentException("Can't change the consumed position for a partition you are not currently subscribed to."); + this.consumed.put(tp, offset); + } + + public Long consumed(TopicPartition partition) { + return this.consumed.get(partition); + } + + public Map allConsumed() { + return this.consumed; + } + + public void needOffsetReset(TopicPartition partition, OffsetResetStrategy offsetResetStrategy) { + this.resetPartitions.put(partition, offsetResetStrategy); + this.fetched.remove(partition); + this.consumed.remove(partition); + } + + public void needOffsetReset(TopicPartition partition) { + needOffsetReset(partition, offsetResetStrategy); + } + + public boolean isOffsetResetNeeded(TopicPartition partition) { + return resetPartitions.containsKey(partition); + } + + public boolean isOffsetResetNeeded() { + return !resetPartitions.isEmpty(); + } + + public OffsetResetStrategy resetStrategy(TopicPartition partition) { + return resetPartitions.get(partition); + } + + public boolean hasAllFetchPositions() { + return this.fetched.size() >= this.assignedPartitions.size(); + } + + public Set missingFetchPositions() { + Set copy = new HashSet(this.assignedPartitions); + copy.removeAll(this.fetched.keySet()); + return copy; + } + + public boolean partitionAssignmentNeeded() { + return this.needsPartitionAssignment; + } + + public void changePartitionAssignment(List assignments) { + for (TopicPartition tp : assignments) + if (!this.subscribedTopics.contains(tp.topic())) + throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic."); + this.clearAssignment(); + this.assignedPartitions.addAll(assignments); + this.needsPartitionAssignment = false; + } + + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java b/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java index b89aa582f64e8..7caefc33603ea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java @@ -29,6 +29,25 @@ public interface Callback { * @param metadata The metadata for the record that was sent (i.e. the partition and offset). Null if an error * occurred. * @param exception The exception thrown during processing of this record. Null if no error occurred. + * Possible thrown exceptions include: + * + * Non-Retriable exceptions (fatal, the message will never be sent): + * + * InvalidTopicException + * OffsetMetadataTooLargeException + * RecordBatchTooLargeException + * RecordTooLargeException + * UnknownServerException + * + * Retriable exceptions (transient, may be covered by increasing #.retries): + * + * CorruptRecordException + * InvalidMetadataException + * NotEnoughReplicasAfterAppendException + * NotEnoughReplicasException + * OffsetOutOfRangeException + * TimeoutException + * UnknownTopicOrPartitionException */ public void onCompletion(RecordMetadata metadata, Exception exception); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 4b2f55650945a..03b8dd23df63a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1,34 +1,28 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * 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 + * 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. + * 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 org.apache.kafka.clients.producer; import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Properties; +import java.util.*; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; -import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; -import org.apache.kafka.clients.producer.internals.Metadata; -import org.apache.kafka.clients.producer.internals.Partitioner; +import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.Sender; import org.apache.kafka.common.Cluster; @@ -37,28 +31,96 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.RecordTooLargeException; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.SystemTime; - +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A Kafka client that publishes records to the Kafka cluster. *

- * The producer is thread safe and should generally be shared among all threads for best performance. + * The producer is thread safe and sharing a single producer instance across threads will generally be faster than + * having multiple instances. + *

+ * Here is a simple example of using the producer to send records with strings containing sequential numbers as the key/value + * pairs. + *

+ * {@code
+ * Properties props = new Properties();
+ * props.put("bootstrap.servers", "localhost:4242");
+ * props.put("acks", "all");
+ * props.put("retries", 0);
+ * props.put("batch.size", 16384);
+ * props.put("linger.ms", 1);
+ * props.put("buffer.memory", 33554432);
+ * props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+ * props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+ *
+ * Producer producer = new KafkaProducer(props);
+ * for(int i = 0; i < 100; i++)
+ *     producer.send(new ProducerRecord("my-topic", Integer.toString(i), Integer.toString(i)));
+ *
+ * producer.close();
+ * }
+ *

+ * The producer consists of a pool of buffer space that holds records that haven't yet been transmitted to the server + * as well as a background I/O thread that is responsible for turning these records into requests and transmitting them + * to the cluster. Failure to close the producer after use will leak these resources. + *

+ * The {@link #send(ProducerRecord) send()} method is asynchronous. When called it adds the record to a buffer of pending record sends + * and immediately returns. This allows the producer to batch together individual records for efficiency. *

- * The producer manages a single background thread that does I/O as well as a TCP connection to each of the brokers it - * needs to communicate with. Failure to close the producer after use will leak these resources. + * The acks config controls the criteria under which requests are considered complete. The "all" setting + * we have specified will result in blocking on the full commit of the record, the slowest but most durable setting. + *

+ * If the request fails, the producer can automatically retry, though since we have specified retries + * as 0 it won't. Enabling retries also opens up the possibility of duplicates (see the documentation on + * message delivery semantics for details). + *

+ * The producer maintains buffers of unsent records for each partition. These buffers are of a size specified by + * the batch.size config. Making this larger can result in more batching, but requires more memory (since we will + * generally have one of these buffers for each active partition). + *

+ * By default a buffer is available to send immediately even if there is additional unused space in the buffer. However if you + * want to reduce the number of requests you can set linger.ms to something greater than 0. This will + * instruct the producer to wait up to that number of milliseconds before sending a request in hope that more records will + * arrive to fill up the same batch. This is analogous to Nagle's algorithm in TCP. For example, in the code snippet above, + * likely all 100 records would be sent in a single request since we set our linger time to 1 millisecond. However this setting + * would add 1 millisecond of latency to our request waiting for more records to arrive if we didn't fill up the buffer. Note that + * records that arrive close together in time will generally batch together even with linger.ms=0 so under heavy load + * batching will occur regardless of the linger configuration; however setting this to something larger than 0 can lead to fewer, more + * efficient requests when not under maximal load at the cost of a small amount of latency. + *

+ * The buffer.memory controls the total amount of memory available to the producer for buffering. If records + * are sent faster than they can be transmitted to the server then this buffer space will be exhausted. When the buffer space is + * exhausted additional send calls will block. For uses where you want to avoid any blocking you can set block.on.buffer.full=false which + * will cause the send call to result in an exception. + *

+ * The key.serializer and value.serializer instruct how to turn the key and value objects the user provides with + * their ProducerRecord into bytes. You can use the included {@link org.apache.kafka.common.serialization.ByteArraySerializer} or + * {@link org.apache.kafka.common.serialization.StringSerializer} for simple string or byte types. */ -public class KafkaProducer implements Producer { +public class KafkaProducer implements Producer { + + private static final Logger log = LoggerFactory.getLogger(KafkaProducer.class); + private static final AtomicInteger PRODUCER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); private final Partitioner partitioner; private final int maxRequestSize; @@ -69,82 +131,165 @@ public class KafkaProducer implements Producer { private final Sender sender; private final Metrics metrics; private final Thread ioThread; + private final CompressionType compressionType; + private final Sensor errors; + private final Time time; + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final ProducerConfig producerConfig; /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings - * are documented here. Values can be + * are documented here. Values can be * either strings or Objects of the appropriate type (for example a numeric configuration would accept either the * string "42" or the integer 42). + * @param configs The producer configs + * */ public KafkaProducer(Map configs) { - this(new ProducerConfig(configs)); + this(new ProducerConfig(configs), null, null); + } + + /** + * A producer is instantiated by providing a set of key-value pairs as configuration, a key and a value {@link Serializer}. + * Valid configuration strings are documented here. + * Values can be either strings or Objects of the appropriate type (for example a numeric configuration would accept + * either the string "42" or the integer 42). + * @param configs The producer configs + * @param keySerializer The serializer for key that implements {@link Serializer}. The configure() method won't be + * called in the producer when the serializer is passed in directly. + * @param valueSerializer The serializer for value that implements {@link Serializer}. The configure() method won't + * be called in the producer when the serializer is passed in directly. + */ + public KafkaProducer(Map configs, Serializer keySerializer, Serializer valueSerializer) { + this(new ProducerConfig(ProducerConfig.addSerializerToConfig(configs, keySerializer, valueSerializer)), + keySerializer, valueSerializer); } /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings - * are documented here. + * are documented here. + * @param properties The producer configs */ public KafkaProducer(Properties properties) { - this(new ProducerConfig(properties)); + this(new ProducerConfig(properties), null, null); } - private KafkaProducer(ProducerConfig config) { - this.metrics = new Metrics(new MetricConfig(), - Collections.singletonList((MetricsReporter) new JmxReporter("kafka.producer.")), - new SystemTime()); - this.partitioner = new Partitioner(); - this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); - this.metadata = new Metadata(); - this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); - this.totalMemorySize = config.getLong(ProducerConfig.TOTAL_BUFFER_MEMORY_CONFIG); - this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.MAX_PARTITION_SIZE_CONFIG), - this.totalMemorySize, - config.getLong(ProducerConfig.LINGER_MS_CONFIG), - config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL), - metrics, - new SystemTime()); - List addresses = parseAndValidateAddresses(config.getList(ProducerConfig.BROKER_LIST_CONFIG)); - this.metadata.update(Cluster.bootstrap(addresses), System.currentTimeMillis()); - this.sender = new Sender(new Selector(), - this.metadata, - this.accumulator, - config.getString(ProducerConfig.CLIENT_ID_CONFIG), - config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG), - config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG), - (short) config.getInt(ProducerConfig.REQUIRED_ACKS_CONFIG), - config.getInt(ProducerConfig.REQUEST_TIMEOUT_CONFIG), - new SystemTime()); - this.ioThread = new KafkaThread("kafka-network-thread", this.sender, true); - this.ioThread.start(); + /** + * A producer is instantiated by providing a set of key-value pairs as configuration, a key and a value {@link Serializer}. + * Valid configuration strings are documented here. + * @param properties The producer configs + * @param keySerializer The serializer for key that implements {@link Serializer}. The configure() method won't be + * called in the producer when the serializer is passed in directly. + * @param valueSerializer The serializer for value that implements {@link Serializer}. The configure() method won't + * be called in the producer when the serializer is passed in directly. + */ + public KafkaProducer(Properties properties, Serializer keySerializer, Serializer valueSerializer) { + this(new ProducerConfig(ProducerConfig.addSerializerToConfig(properties, keySerializer, valueSerializer)), + keySerializer, valueSerializer); } - private static List parseAndValidateAddresses(List urls) { - List addresses = new ArrayList(); - for (String url : urls) { - if (url != null && url.length() > 0) { - String[] pieces = url.split(":"); - if (pieces.length != 2) - throw new ConfigException("Invalid url in metadata.broker.list: " + url); - try { - InetSocketAddress address = new InetSocketAddress(pieces[0], Integer.parseInt(pieces[1])); - if (address.isUnresolved()) - throw new ConfigException("DNS resolution failed for metadata bootstrap url: " + url); - addresses.add(address); - } catch (NumberFormatException e) { - throw new ConfigException("Invalid port in metadata.broker.list: " + url); - } + @SuppressWarnings("unchecked") + private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serializer valueSerializer) { + try { + log.trace("Starting the Kafka producer"); + this.producerConfig = config; + this.time = new SystemTime(); + MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG)) + .timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), + TimeUnit.MILLISECONDS); + String clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG); + if (clientId.length() <= 0) + clientId = "producer-" + PRODUCER_CLIENT_ID_SEQUENCE.getAndIncrement(); + String jmxPrefix = "kafka.producer"; + List reporters = config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, + MetricsReporter.class); + reporters.add(new JmxReporter(jmxPrefix)); + this.metrics = new Metrics(metricConfig, reporters, time); + this.partitioner = config.getConfiguredInstance(ProducerConfig.PARTITIONER_CLASS_CONFIG, Partitioner.class); + long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG); + this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); + this.metadata = new Metadata(retryBackoffMs, config.getLong(ProducerConfig.METADATA_MAX_AGE_CONFIG)); + this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); + this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG); + this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG)); + Map metricTags = new LinkedHashMap(); + metricTags.put("client-id", clientId); + this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG), + this.totalMemorySize, + this.compressionType, + config.getLong(ProducerConfig.LINGER_MS_CONFIG), + retryBackoffMs, + config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG), + metrics, + time, + metricTags); + List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); + + NetworkClient client = new NetworkClient( + new Selector(config.getLong(ProducerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "producer", metricTags), + this.metadata, + clientId, + config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), + config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG), + config.getInt(ProducerConfig.SEND_BUFFER_CONFIG), + config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG)); + this.sender = new Sender(client, + this.metadata, + this.accumulator, + config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG), + (short) parseAcks(config.getString(ProducerConfig.ACKS_CONFIG)), + config.getInt(ProducerConfig.RETRIES_CONFIG), + config.getInt(ProducerConfig.TIMEOUT_CONFIG), + this.metrics, + new SystemTime(), + clientId); + String ioThreadName = "kafka-producer-network-thread" + (clientId.length() > 0 ? " | " + clientId : ""); + this.ioThread = new KafkaThread(ioThreadName, this.sender, true); + this.ioThread.start(); + + this.errors = this.metrics.sensor("errors"); + + if (keySerializer == null) { + this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + Serializer.class); + this.keySerializer.configure(config.originals(), true); + } else { + this.keySerializer = keySerializer; + } + if (valueSerializer == null) { + this.valueSerializer = config.getConfiguredInstance(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + Serializer.class); + this.valueSerializer.configure(config.originals(), false); + } else { + this.valueSerializer = valueSerializer; } + config.logUnused(); + log.debug("Kafka producer started"); + } catch (Throwable t) { + // call close methods if internal objects are already constructed + // this is to prevent resource leak. see KAFKA-2121 + close(0, TimeUnit.MILLISECONDS, true); + // now propagate the exception + throw new KafkaException("Failed to construct kafka producer", t); + } + } + + private static int parseAcks(String acksString) { + try { + return acksString.trim().toLowerCase().equals("all") ? -1 : Integer.parseInt(acksString.trim()); + } catch (NumberFormatException e) { + throw new ConfigException("Invalid configuration value for 'acks': " + acksString); } - if (addresses.size() < 1) - throw new ConfigException("No bootstrap urls given in metadata.broker.list."); - return addresses; } /** - * Asynchronously send a record to a topic. Equivalent to {@link #send(ProducerRecord, Callback) send(record, null)} + * Asynchronously send a record to a topic. Equivalent to send(record, null). + * See {@link #send(ProducerRecord, Callback)} for details. */ @Override - public Future send(ProducerRecord record) { + public Future send(ProducerRecord record) { return send(record, null); } @@ -160,109 +305,325 @@ public Future send(ProducerRecord record) { *

* Since the send call is asynchronous it returns a {@link java.util.concurrent.Future Future} for the * {@link RecordMetadata} that will be assigned to this record. Invoking {@link java.util.concurrent.Future#get() - * get()} on this future will result in the metadata for the record or throw any exception that occurred while - * sending the record. + * get()} on this future will block until the associated request completes and then return the metadata for the record + * or throw any exception that occurred while sending the record. *

- * If you want to simulate a simple blocking call you can do the following: - * + * If you want to simulate a simple blocking call you can call the get() method immediately: + * *

-     *   producer.send(new ProducerRecord("the-topic", "key, "value")).get();
-     * 
+ * {@code + * byte[] key = "key".getBytes(); + * byte[] value = "value".getBytes(); + * ProducerRecord record = new ProducerRecord("my-topic", key, value) + * producer.send(record).get(); + * } *

- * Those desiring fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that + * Fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that * will be invoked when the request is complete. - * + * *

-     *   ProducerRecord record = new ProducerRecord("the-topic", "key, "value");
-     *   producer.send(myRecord,
-     *                 new Callback() {
-     *                     public void onCompletion(RecordMetadata metadata, Exception e) {
-     *                         if(e != null)
-     *                             e.printStackTrace();
-     *                         System.out.println("The offset of the record we just sent is: " + metadata.offset());
-     *                     }
-     *                 });
+     * {@code
+     * ProducerRecord record = new ProducerRecord("the-topic", key, value);
+     * producer.send(myRecord,
+     *               new Callback() {
+     *                   public void onCompletion(RecordMetadata metadata, Exception e) {
+     *                       if(e != null)
+     *                           e.printStackTrace();
+     *                       System.out.println("The offset of the record we just sent is: " + metadata.offset());
+     *                   }
+     *               });
+     * }
      * 
- * + * * Callbacks for records being sent to the same partition are guaranteed to execute in order. That is, in the * following example callback1 is guaranteed to execute before callback2: - * + * *
-     * producer.send(new ProducerRecord(topic, partition, key, value), callback1);
-     * producer.send(new ProducerRecord(topic, partition, key2, value2), callback2);
+     * {@code
+     * producer.send(new ProducerRecord(topic, partition, key1, value1), callback1);
+     * producer.send(new ProducerRecord(topic, partition, key2, value2), callback2);
+     * }
      * 
*

* Note that callbacks will generally execute in the I/O thread of the producer and so should be reasonably fast or * they will delay the sending of messages from other threads. If you want to execute blocking or computationally * expensive callbacks it is recommended to use your own {@link java.util.concurrent.Executor} in the callback body * to parallelize processing. - *

- * The producer manages a buffer of records waiting to be sent. This buffer has a hard limit on it's size, which is - * controlled by the configuration total.memory.bytes. If send() is called faster than the - * I/O thread can transfer data to the brokers the buffer will eventually run out of space. The default behavior in - * this case is to block the send call until the I/O thread catches up and more buffer space is available. However - * in cases where non-blocking usage is desired the setting block.on.buffer.full=false will cause the - * producer to instead throw an exception when buffer memory is exhausted. - * + * * @param record The record to send * @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null * indicates no callback) + * + * @throws InterruptException If the thread is interrupted while blocked + * @throws SerializationException If the key or value are not valid objects given the configured serializers + * @throws BufferExhaustedException If block.on.buffer.full=false and the buffer is full. + * */ @Override - public Future send(ProducerRecord record, Callback callback) { + public Future send(ProducerRecord record, Callback callback) { try { - Cluster cluster = metadata.fetch(record.topic(), this.metadataFetchTimeoutMs); - int partition = partitioner.partition(record, cluster); - ensureValidSize(record.key(), record.value()); + // first make sure the metadata for the topic is available + waitOnMetadata(record.topic(), this.metadataFetchTimeoutMs); + byte[] serializedKey; + try { + serializedKey = keySerializer.serialize(record.topic(), record.key()); + } catch (ClassCastException cce) { + throw new SerializationException("Can't convert key of class " + record.key().getClass().getName() + + " to class " + producerConfig.getClass(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).getName() + + " specified in key.serializer"); + } + byte[] serializedValue; + try { + serializedValue = valueSerializer.serialize(record.topic(), record.value()); + } catch (ClassCastException cce) { + throw new SerializationException("Can't convert value of class " + record.value().getClass().getName() + + " to class " + producerConfig.getClass(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName() + + " specified in value.serializer"); + } + int partition = partition(record, serializedKey, serializedValue, metadata.fetch()); + int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); + ensureValidRecordSize(serializedSize); TopicPartition tp = new TopicPartition(record.topic(), partition); - FutureRecordMetadata future = accumulator.append(tp, record.key(), record.value(), CompressionType.NONE, callback); - this.sender.wakeup(); - return future; - } catch (Exception e) { + log.trace("Sending record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition); + RecordAccumulator.RecordAppendResult result = accumulator.append(tp, serializedKey, serializedValue, callback); + if (result.batchIsFull || result.newBatchCreated) { + log.trace("Waking up the sender since topic {} partition {} is either full or getting a new batch", record.topic(), partition); + this.sender.wakeup(); + } + return result.future; + // handling exceptions and record the errors; + // for API exceptions return them in the future, + // for other exceptions throw directly + } catch (ApiException e) { + log.debug("Exception occurred during message send:", e); if (callback != null) callback.onCompletion(null, e); + this.errors.record(); return new FutureFailure(e); + } catch (InterruptedException e) { + this.errors.record(); + throw new InterruptException(e); + } catch (BufferExhaustedException e) { + this.errors.record(); + this.metrics.sensor("buffer-exhausted-records").record(); + throw e; + } catch (KafkaException e) { + this.errors.record(); + throw e; } } /** - * Check that this key-value pair will have a serialized size small enough + * Wait for cluster metadata including partitions for the given topic to be available. + * @param topic The topic we want metadata for + * @param maxWaitMs The maximum time in ms for waiting on the metadata + */ + private void waitOnMetadata(String topic, long maxWaitMs) throws InterruptedException { + // add topic to metadata topic list if it is not there already. + if (!this.metadata.containsTopic(topic)) + this.metadata.add(topic); + + if (metadata.fetch().partitionsForTopic(topic) != null) { + return; + } else { + long begin = time.milliseconds(); + long remainingWaitMs = maxWaitMs; + while (metadata.fetch().partitionsForTopic(topic) == null) { + log.trace("Requesting metadata update for topic {}.", topic); + int version = metadata.requestUpdate(); + sender.wakeup(); + metadata.awaitUpdate(version, remainingWaitMs); + long elapsed = time.milliseconds() - begin; + if (elapsed >= maxWaitMs) + throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms."); + remainingWaitMs = maxWaitMs - elapsed; + } + } + } + + /** + * Validate that the record size isn't too large + */ + private void ensureValidRecordSize(int size) { + if (size > this.maxRequestSize) + throw new RecordTooLargeException("The message is " + size + + " bytes when serialized which is larger than the maximum request size you have configured with the " + + ProducerConfig.MAX_REQUEST_SIZE_CONFIG + + " configuration."); + if (size > this.totalMemorySize) + throw new RecordTooLargeException("The message is " + size + + " bytes when serialized which is larger than the total memory buffer you have configured with the " + + ProducerConfig.BUFFER_MEMORY_CONFIG + + " configuration."); + } + + /** + * Invoking this method makes all buffered records immediately available to send (even if linger.ms is + * greater than 0) and blocks on the completion of the requests associated with these records. The post-condition + * of flush() is that any previously sent record will have completed (e.g. Future.isDone() == true). + * A request is considered completed when it is successfully acknowledged + * according to the acks configuration you have specified or else it results in an error. + *

+ * Other threads can continue sending records while one thread is blocked waiting for a flush call to complete, + * however no guarantee is made about the completion of records sent after the flush call begins. + *

+ * This method can be useful when consuming from some input system and producing into Kafka. The flush() call + * gives a convenient way to ensure all previously sent messages have actually completed. + *

+ * This example shows how to consume from one Kafka topic and produce to another Kafka topic: + *

+     * {@code
+     * for(ConsumerRecord record: consumer.poll(100))
+     *     producer.send(new ProducerRecord("my-topic", record.key(), record.value());
+     * producer.flush();
+     * consumer.commit();
+     * }
+     * 
+ * + * Note that the above example may drop records if the produce request fails. If we want to ensure that this does not occur + * we need to set retries=<large_number> in our config. + * + * @throws InterruptException If the thread is interrupted while blocked */ - private void ensureValidSize(byte[] key, byte[] value) { - int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(key, value); - if (serializedSize > this.maxRequestSize) - throw new RecordTooLargeException("The message is " + serializedSize - + " bytes when serialized which is larger than the maximum request size you have configured with the " - + ProducerConfig.MAX_REQUEST_SIZE_CONFIG - + " configuration."); - if (serializedSize > this.totalMemorySize) - throw new RecordTooLargeException("The message is " + serializedSize - + " bytes when serialized which is larger than the total memory buffer you have configured with the " - + ProducerConfig.TOTAL_BUFFER_MEMORY_CONFIG - + " configuration."); + @Override + public void flush() { + log.trace("Flushing accumulated records in producer."); + this.accumulator.beginFlush(); + this.sender.wakeup(); + try { + this.accumulator.awaitFlushCompletion(); + } catch (InterruptedException e) { + throw new InterruptException("Flush interrupted.", e); + } } + /** + * Get the partition metadata for the give topic. This can be used for custom partitioning. + * @throws InterruptException If the thread is interrupted while blocked + */ + @Override public List partitionsFor(String topic) { - return this.metadata.fetch(topic, this.metadataFetchTimeoutMs).partitionsFor(topic); + try { + waitOnMetadata(topic, this.metadataFetchTimeoutMs); + } catch (InterruptedException e) { + throw new InterruptException(e); + } + return this.metadata.fetch().partitionsForTopic(topic); } + /** + * Get the full set of internal metrics maintained by the producer. + */ @Override - public Map metrics() { + public Map metrics() { return Collections.unmodifiableMap(this.metrics.metrics()); } /** - * Close this producer. This method blocks until all in-flight requests complete. + * Close this producer. This method blocks until all previously sent requests complete. + * This method is equivalent to close(Long.MAX_VALUE, TimeUnit.MILLISECONDS). + *

+ * If close() is called from {@link Callback}, a warning message will be logged and close(0, TimeUnit.MILLISECONDS) + * will be called instead. We do this because the sender thread would otherwise try to join itself and + * block forever. + *

+ * + * @throws InterruptException If the thread is interrupted while blocked */ @Override public void close() { - this.sender.initiateClose(); - try { - this.ioThread.join(); - } catch (InterruptedException e) { - throw new KafkaException(e); + close(Long.MAX_VALUE, TimeUnit.MILLISECONDS); + } + + /** + * This method waits up to timeout for the producer to complete the sending of all incomplete requests. + *

+ * If the producer is unable to complete all requests before the timeout expires, this method will fail + * any unsent and unacknowledged records immediately. + *

+ * If invoked from within a {@link Callback} this method will not block and will be equivalent to + * close(0, TimeUnit.MILLISECONDS). This is done since no further sending will happen while + * blocking the I/O thread of the producer. + * + * @param timeout The maximum time to wait for producer to complete any pending requests. The value should be + * non-negative. Specifying a timeout of zero means do not wait for pending send requests to complete. + * @param timeUnit The time unit for the timeout + * @throws InterruptException If the thread is interrupted while blocked + * @throws IllegalArgumentException If the timeout is negative. + */ + @Override + public void close(long timeout, TimeUnit timeUnit) { + close(timeout, timeUnit, false); + } + + private void close(long timeout, TimeUnit timeUnit, boolean swallowException) { + if (timeout < 0) + throw new IllegalArgumentException("The timeout cannot be negative."); + + log.info("Closing the Kafka producer with timeoutMillis = {} ms.", timeUnit.toMillis(timeout)); + // this will keep track of the first encountered exception + AtomicReference firstException = new AtomicReference(); + boolean invokedFromCallback = Thread.currentThread() == this.ioThread; + if (timeout > 0) { + if (invokedFromCallback) { + log.warn("Overriding close timeout {} ms to 0 ms in order to prevent useless blocking due to self-join. " + + "This means you have incorrectly invoked close with a non-zero timeout from the producer call-back.", timeout); + } else { + // Try to close gracefully. + if (this.sender != null) + this.sender.initiateClose(); + if (this.ioThread != null) { + try { + this.ioThread.join(timeUnit.toMillis(timeout)); + } catch (InterruptedException t) { + firstException.compareAndSet(null, t); + log.error("Interrupted while joining ioThread", t); + } + } + } + } + + if (this.sender != null && this.ioThread != null && this.ioThread.isAlive()) { + log.info("Proceeding to force close the producer since pending requests could not be completed " + + "within timeout {} ms.", timeout); + this.sender.forceClose(); + // Only join the sender thread when not calling from callback. + if (!invokedFromCallback) { + try { + this.ioThread.join(); + } catch (InterruptedException e) { + firstException.compareAndSet(null, e); + } + } + } + + ClientUtils.closeQuietly(metrics, "producer metrics", firstException); + ClientUtils.closeQuietly(keySerializer, "producer keySerializer", firstException); + ClientUtils.closeQuietly(valueSerializer, "producer valueSerializer", firstException); + log.debug("The Kafka producer has closed."); + if (firstException.get() != null && !swallowException) + throw new KafkaException("Failed to close kafka producer", firstException.get()); + } + + /** + * computes partition for given record. + * if the record has partition returns the value otherwise + * calls configured partitioner class to compute the partition. + */ + private int partition(ProducerRecord record, byte[] serializedKey , byte[] serializedValue, Cluster cluster) { + Integer partition = record.partition(); + if (partition != null) { + List partitions = cluster.partitionsForTopic(record.topic()); + int numPartitions = partitions.size(); + // they have given us a partition, use it + if (partition < 0 || partition >= numPartitions) + throw new IllegalArgumentException("Invalid partition given with record: " + partition + + " is not in the range [0..." + + numPartitions + + "]."); + return partition; } - this.metrics.close(); + return this.partitioner.partition(record.topic(), record.key(), serializedKey, record.value(), serializedValue, cluster); } private static class FutureFailure implements Future { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index f43da80580f5a..36e7ffa2a0a0b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -24,14 +24,13 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; -import org.apache.kafka.clients.producer.internals.Partitioner; +import org.apache.kafka.clients.producer.internals.DefaultPartitioner; import org.apache.kafka.clients.producer.internals.ProduceRequestResult; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.Metric; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.*; +import org.apache.kafka.common.serialization.Serializer; /** @@ -40,48 +39,56 @@ * By default this mock will synchronously complete each send call successfully. However it can be configured to allow * the user to control the completion of the call and supply an optional error for the producer to throw. */ -public class MockProducer implements Producer { +public class MockProducer implements Producer { private final Cluster cluster; - private final Partitioner partitioner = new Partitioner(); - private final List sent; + private final Partitioner partitioner; + private final List> sent; private final Deque completions; private boolean autoComplete; private Map offsets; + private final Serializer keySerializer; + private final Serializer valueSerializer; /** * Create a mock producer - * + * * @param cluster The cluster holding metadata for this producer * @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise * the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after * {@link #send(ProducerRecord) send()} to complete the call and unblock the @{link * java.util.concurrent.Future Future<RecordMetadata>} that is returned. + * @param partitioner The partition strategy + * @param keySerializer The serializer for key that implements {@link Serializer}. + * @param valueSerializer The serializer for value that implements {@link Serializer}. */ - public MockProducer(Cluster cluster, boolean autoComplete) { + public MockProducer(Cluster cluster, boolean autoComplete, Partitioner partitioner, Serializer keySerializer, Serializer valueSerializer) { this.cluster = cluster; this.autoComplete = autoComplete; + this.partitioner = partitioner; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; this.offsets = new HashMap(); - this.sent = new ArrayList(); + this.sent = new ArrayList>(); this.completions = new ArrayDeque(); } /** - * Create a new mock producer with invented metadata the given autoComplete setting. - * - * Equivalent to {@link #MockProducer(Cluster, boolean) new MockProducer(null, autoComplete)} + * Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers + * + * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)} */ - public MockProducer(boolean autoComplete) { - this(Cluster.empty(), autoComplete); + public MockProducer(boolean autoComplete, Serializer keySerializer, Serializer valueSerializer) { + this(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer); } /** - * Create a new auto completing mock producer - * - * Equivalent to {@link #MockProducer(boolean) new MockProducer(true)} + * Create a new mock producer with invented metadata the given autoComplete setting, partitioner and key\value serializers + * + * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer)} */ - public MockProducer() { - this(true); + public MockProducer(boolean autoComplete, Partitioner partitioner, Serializer keySerializer, Serializer valueSerializer) { + this(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer); } /** @@ -90,25 +97,25 @@ public MockProducer() { * @see #history() */ @Override - public synchronized Future send(ProducerRecord record) { + public synchronized Future send(ProducerRecord record) { return send(record, null); } /** * Adds the record to the list of sent records. - * + * * @see #history() */ @Override - public synchronized Future send(ProducerRecord record, Callback callback) { + public synchronized Future send(ProducerRecord record, Callback callback) { int partition = 0; - if (this.cluster.partitionsFor(record.topic()) != null) - partition = partitioner.partition(record, this.cluster); + if (this.cluster.partitionsForTopic(record.topic()) != null) + partition = partition(record, this.cluster); ProduceRequestResult result = new ProduceRequestResult(); FutureRecordMetadata future = new FutureRecordMetadata(result, 0); TopicPartition topicPartition = new TopicPartition(record.topic(), partition); long offset = nextOffset(topicPartition); - Completion completion = new Completion(topicPartition, offset, new RecordMetadata(topicPartition, offset), result, callback); + Completion completion = new Completion(topicPartition, offset, new RecordMetadata(topicPartition, 0, offset), result, callback); this.sent.add(record); if (autoComplete) completion.complete(null); @@ -132,11 +139,16 @@ private long nextOffset(TopicPartition tp) { } } + public synchronized void flush() { + while (!this.completions.isEmpty()) + completeNext(); + } + public List partitionsFor(String topic) { - return this.cluster.partitionsFor(topic); + return this.cluster.partitionsForTopic(topic); } - public Map metrics() { + public Map metrics() { return Collections.emptyMap(); } @@ -144,11 +156,15 @@ public Map metrics() { public void close() { } + @Override + public void close(long timeout, TimeUnit timeUnit) { + } + /** * Get the list of sent records since the last call to {@link #clear()} */ - public synchronized List history() { - return new ArrayList(this.sent); + public synchronized List> history() { + return new ArrayList>(this.sent); } /** @@ -161,7 +177,7 @@ public synchronized void clear() { /** * Complete the earliest uncompleted call successfully. - * + * * @return true if there was an uncompleted call to complete */ public synchronized boolean completeNext() { @@ -170,7 +186,7 @@ public synchronized boolean completeNext() { /** * Complete the earliest uncompleted call with the given error. - * + * * @return true if there was an uncompleted call to complete */ public synchronized boolean errorNext(RuntimeException e) { @@ -183,6 +199,28 @@ public synchronized boolean errorNext(RuntimeException e) { } } + /** + * computes partition for given record. + */ + private int partition(ProducerRecord record, Cluster cluster) { + Integer partition = record.partition(); + String topic = record.topic(); + if (partition != null) { + List partitions = cluster.partitionsForTopic(topic); + int numPartitions = partitions.size(); + // they have given us a partition, use it + if (partition < 0 || partition >= numPartitions) + throw new IllegalArgumentException("Invalid partition given with record: " + partition + + " is not in the range [0..." + + numPartitions + + "]."); + return partition; + } + byte[] keyBytes = keySerializer.serialize(topic, record.key()); + byte[] valueBytes = valueSerializer.serialize(topic, record.value()); + return this.partitioner.partition(topic, record.key(), keyBytes, record.value(), valueBytes, cluster); + } + private static class Completion { private final long offset; private final RecordMetadata metadata; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java new file mode 100644 index 0000000000000..383619d9d02bc --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java @@ -0,0 +1,46 @@ +/** + * 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 org.apache.kafka.clients.producer; + +import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.Cluster; + +/** + * Partitioner Interface + */ + +public interface Partitioner extends Configurable { + + /** + * Compute the partition for the given record. + * + * @param topic The topic name + * @param key The key to partition on (or null if no key) + * @param keyBytes The serialized key to partition on( or null if no key) + * @param value The value to partition on or null + * @param valueBytes The serialized value to partition on or null + * @param cluster The current cluster metadata + */ + public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster); + + /** + * This is called when partitioner is closed. + */ + public void close(); + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index 36e8398416036..d4a5d391bf23f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -20,18 +20,19 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import org.apache.kafka.common.Metric; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.MetricName; /** * The interface for the {@link KafkaProducer} - * * @see KafkaProducer * @see MockProducer */ -public interface Producer extends Closeable { +public interface Producer extends Closeable { /** * Send the given record asynchronously and return a future which will eventually contain the response information. @@ -39,12 +40,17 @@ public interface Producer extends Closeable { * @param record The record to send * @return A future which will eventually contain the response information */ - public Future send(ProducerRecord record); + public Future send(ProducerRecord record); /** * Send a record and invoke the given callback when the record has been acknowledged by the server */ - public Future send(ProducerRecord record, Callback callback); + public Future send(ProducerRecord record, Callback callback); + + /** + * Flush any accumulated records from the producer. Blocks until all sends are complete. + */ + public void flush(); /** * Get a list of partitions for the given topic for custom partition assignment. The partition metadata will change @@ -55,11 +61,17 @@ public interface Producer extends Closeable { /** * Return a map of metrics maintained by the producer */ - public Map metrics(); + public Map metrics(); /** * Close this producer */ public void close(); + /** + * Tries to close the producer cleanly within the specified timeout. If the close does not complete within the + * timeout, fail any pending send requests and force close the producer. + */ + public void close(long timeout, TimeUnit unit); + } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 502af5cd555dd..aa264202f2724 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -1,147 +1,263 @@ /** - * 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 + * 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 + * 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. + * 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 org.apache.kafka.clients.producer; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Range.between; +import static org.apache.kafka.common.config.ConfigDef.ValidString.in; +import java.util.HashMap; import java.util.Map; +import java.util.Properties; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; - +import org.apache.kafka.common.serialization.Serializer; /** - * The producer configuration keys + * Configuration for the Kafka Producer. Documentation for these configurations can be found in the Kafka documentation */ public class ProducerConfig extends AbstractConfig { - private static final ConfigDef config; - - /** - * A list of URLs to use for establishing the initial connection to the cluster. This list should be in the form - * host1:port1,host2:port2,.... These urls are just used for the initial connection to discover the - * full cluster membership (which may change dynamically) so this list need not contain the full set of servers (you - * may want more than one, though, in case a server is down). + /* + * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS THESE ARE PART OF THE PUBLIC API AND + * CHANGE WILL BREAK USER CODE. */ - public static final String BROKER_LIST_CONFIG = "metadata.broker.list"; - /** - * The amount of time to block waiting to fetch metadata about a topic the first time a record is sent to that - * topic. - */ + private static final ConfigDef CONFIG; + + /** bootstrap.servers */ + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + + /** metadata.fetch.timeout.ms */ public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms"; + private static final String METADATA_FETCH_TIMEOUT_DOC = "The first time data is sent to a topic we must fetch metadata about that topic to know which servers host the topic's partitions. This " + + "fetch to succeed before throwing an exception back to the client."; - /** - * The buffer size allocated for a partition. When records are received which are smaller than this size the - * producer will attempt to optimistically group them together until this size is reached. - */ - public static final String MAX_PARTITION_SIZE_CONFIG = "max.partition.bytes"; + /** metadata.max.age.ms */ + public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; + private static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; - /** - * The total memory used by the producer to buffer records waiting to be sent to the server. If records are sent - * faster than they can be delivered to the server the producer will either block or throw an exception based on the - * preference specified by {@link #BLOCK_ON_BUFFER_FULL}. - */ - public static final String TOTAL_BUFFER_MEMORY_CONFIG = "total.memory.bytes"; + /** batch.size */ + public static final String BATCH_SIZE_CONFIG = "batch.size"; + private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + " to the same partition. This helps performance on both the client and the server. This configuration controls the " + + "default batch size in bytes. " + + "

" + + "No attempt will be made to batch records larger than this size. " + + "

" + + "Requests sent to brokers will contain multiple batches, one for each partition with data available to be sent. " + + "

" + + "A small batch size will make batching less common and may reduce throughput (a batch size of zero will disable " + + "batching entirely). A very large batch size may use memory a bit more wastefully as we will always allocate a " + + "buffer of the specified batch size in anticipation of additional records."; - /** - * The number of acknowledgments the producer requires from the server before considering a request complete. - */ - public static final String REQUIRED_ACKS_CONFIG = "request.required.acks"; + /** buffer.memory */ + public static final String BUFFER_MEMORY_CONFIG = "buffer.memory"; + private static final String BUFFER_MEMORY_DOC = "The total bytes of memory the producer can use to buffer records waiting to be sent to the server. If records are " + "sent faster than they can be delivered to the server the producer will either block or throw an exception based " + + "on the preference specified by block.on.buffer.full. " + + "

" + + "This setting should correspond roughly to the total memory the producer will use, but is not a hard bound since " + + "not all memory the producer uses is used for buffering. Some additional memory will be used for compression (if " + + "compression is enabled) as well as for maintaining in-flight requests."; - /** - * The maximum amount of time the server will wait for acknowledgments from followers to meet the acknowledgment - * requirements the producer has specified. If the requested number of acknowledgments are not met an error will be - * returned. - */ - public static final String REQUEST_TIMEOUT_CONFIG = "request.timeout.ms"; - - /** - * The producer groups together any records that arrive in between request sends. Normally this occurs only under - * load when records arrive faster than they can be sent out. However the client can reduce the number of requests - * and increase throughput by adding a small amount of artificial delay to force more records to batch together. - * This setting gives an upper bound on this delay. If we get {@link #MAX_PARTITION_SIZE_CONFIG} worth of records - * for a partition it will be sent immediately regardless of this setting, however if we have fewer than this many - * bytes accumulated for this partition we will "linger" for the specified time waiting for more records to show up. - * This setting defaults to 0. - */ + /** acks */ + public static final String ACKS_CONFIG = "acks"; + private static final String ACKS_DOC = "The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the " + + " durability of records that are sent. The following settings are common: " + + "

    " + + "
  • acks=0 If set to zero then the producer will not wait for any acknowledgment from the" + + " server at all. The record will be immediately added to the socket buffer and considered sent. No guarantee can be" + + " made that the server has received the record in this case, and the retries configuration will not" + + " take effect (as the client won't generally know of any failures). The offset given back for each record will" + + " always be set to -1." + + "
  • acks=1 This will mean the leader will write the record to its local log but will respond" + + " without awaiting full acknowledgement from all followers. In this case should the leader fail immediately after" + + " acknowledging the record but before the followers have replicated it then the record will be lost." + + "
  • acks=all This means the leader will wait for the full set of in-sync replicas to" + + " acknowledge the record. This guarantees that the record will not be lost as long as at least one in-sync replica" + + " remains alive. This is the strongest available guarantee."; + + /** timeout.ms */ + public static final String TIMEOUT_CONFIG = "timeout.ms"; + private static final String TIMEOUT_DOC = "The configuration controls the maximum amount of time the server will wait for acknowledgments from followers to " + "meet the acknowledgment requirements the producer has specified with the acks configuration. If the " + + "requested number of acknowledgments are not met when the timeout elapses an error will be returned. This timeout " + + "is measured on the server side and does not include the network latency of the request."; + + /** linger.ms */ public static final String LINGER_MS_CONFIG = "linger.ms"; + private static final String LINGER_MS_DOC = "The producer groups together any records that arrive in between request transmissions into a single batched request. " + "Normally this occurs only under load when records arrive faster than they can be sent out. However in some circumstances the client may want to " + + "reduce the number of requests even under moderate load. This setting accomplishes this by adding a small amount " + + "of artificial delay—that is, rather than immediately sending out a record the producer will wait for up to " + + "the given delay to allow other records to be sent so that the sends can be batched together. This can be thought " + + "of as analogous to Nagle's algorithm in TCP. This setting gives the upper bound on the delay for batching: once " + + "we get batch.size worth of records for a partition it will be sent immediately regardless of this " + + "setting, however if we have fewer than this many bytes accumulated for this partition we will 'linger' for the " + + "specified time waiting for more records to show up. This setting defaults to 0 (i.e. no delay). Setting linger.ms=5, " + + "for example, would have the effect of reducing the number of requests sent but would add up to 5ms of latency to records sent in the absense of load."; - /** - * Force a refresh of the cluster metadata after this period of time. This ensures that changes to the number of - * partitions or other settings will by taken up by producers without restart. - */ - public static final String METADATA_REFRESH_MS_CONFIG = "topic.metadata.refresh.interval.ms"; + /** client.id */ + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; - /** - * The id string to pass to the server when making requests. The purpose of this is to be able to track the source - * of requests beyond just ip/port by allowing a logical application name to be included. - */ - public static final String CLIENT_ID_CONFIG = "client.id"; + /** send.buffer.bytes */ + public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; - /** - * The size of the TCP send buffer to use when sending data - */ - public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes"; + /** receive.buffer.bytes */ + public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; - /** - * The maximum size of a request. This is also effectively a cap on the maximum record size. Note that the server - * has its own cap on record size which may be different from this. - */ + /** max.request.size */ public static final String MAX_REQUEST_SIZE_CONFIG = "max.request.size"; + private static final String MAX_REQUEST_SIZE_DOC = "The maximum size of a request. This is also effectively a cap on the maximum record size. Note that the server " + "has its own cap on record size which may be different from this. This setting will limit the number of record " + + "batches the producer will send in a single request to avoid sending huge requests."; - /** - * The amount of time to wait before attempting to reconnect to a given host. This avoids repeated connecting to a - * host in a tight loop. - */ - public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; + /** reconnect.backoff.ms */ + public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; - /** - * When our memory buffer is exhausted we must either stop accepting new records (block) or throw errors. By default - * this setting is true and we block, however users who want to guarantee we never block can turn this into an - * error. - */ - public static final String BLOCK_ON_BUFFER_FULL = "block.on.buffer.full"; + /** block.on.buffer.full */ + public static final String BLOCK_ON_BUFFER_FULL_CONFIG = "block.on.buffer.full"; + private static final String BLOCK_ON_BUFFER_FULL_DOC = "When our memory buffer is exhausted we must either stop accepting new records (block) or throw errors. By default " + "this setting is true and we block, however in some scenarios blocking is not desirable and it is better to " + + "immediately give an error. Setting this to false will accomplish that: the producer will throw a BufferExhaustedException if a recrord is sent and the buffer space is full."; + + /** retries */ + public static final String RETRIES_CONFIG = "retries"; + private static final String RETRIES_DOC = "Setting a value greater than zero will cause the client to resend any record whose send fails with a potentially transient error." + " Note that this retry is no different than if the client resent the record upon receiving the " + + "error. Allowing retries will potentially change the ordering of records because if two records are " + + "sent to a single partition, and the first fails and is retried but the second succeeds, then the second record " + + "may appear first."; + + /** retry.backoff.ms */ + public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; + + /** compression.type */ + public static final String COMPRESSION_TYPE_CONFIG = "compression.type"; + private static final String COMPRESSION_TYPE_DOC = "The compression type for all data generated by the producer. The default is none (i.e. no compression). Valid " + " values are none, gzip, snappy, or lz4. " + + "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression)."; + + /** metrics.sample.window.ms */ + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; + + /** metrics.num.samples */ + public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; + + /** metric.reporters */ + public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG; + + /** max.in.flight.requests.per.connection */ + public static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = "max.in.flight.requests.per.connection"; + private static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC = "The maximum number of unacknowledged requests the client will send on a single connection before blocking." + + " Note that if this setting is set to be greater than 1 and there are failed sends, there is a risk of" + + " message re-ordering due to retries (i.e., if retries are enabled)."; + + /** key.serializer */ + public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; + private static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; - public static final String ENABLE_JMX = "enable.jmx"; + /** value.serializer */ + public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; + private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; + + /** connections.max.idle.ms */ + public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; + + /** partitioner.class */ + public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class"; + private static final String PARTITIONER_CLASS_DOC = "Partitioner class that implements the Partitioner interface."; static { - /* TODO: add docs */ - config = new ConfigDef().define(BROKER_LIST_CONFIG, Type.LIST, "blah blah") - .define(METADATA_FETCH_TIMEOUT_CONFIG, Type.LONG, 60 * 1000, atLeast(0), "blah blah") - .define(MAX_PARTITION_SIZE_CONFIG, Type.INT, 16384, atLeast(0), "blah blah") - .define(TOTAL_BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), "blah blah") - /* TODO: should be a string to handle acks=in-sync */ - .define(REQUIRED_ACKS_CONFIG, Type.INT, 1, between(-1, Short.MAX_VALUE), "blah blah") - .define(REQUEST_TIMEOUT_CONFIG, Type.INT, 30 * 1000, atLeast(0), "blah blah") - .define(LINGER_MS_CONFIG, Type.LONG, 0, atLeast(0L), "blah blah") - .define(METADATA_REFRESH_MS_CONFIG, Type.LONG, 10 * 60 * 1000, atLeast(-1L), "blah blah") - .define(CLIENT_ID_CONFIG, Type.STRING, "", "blah blah") - .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), "blah blah") - .define(MAX_REQUEST_SIZE_CONFIG, Type.INT, 1 * 1024 * 1024, atLeast(0), "blah blah") - .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), "blah blah") - .define(BLOCK_ON_BUFFER_FULL, Type.BOOLEAN, true, "blah blah") - .define(ENABLE_JMX, Type.BOOLEAN, true, ""); + CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) + .define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC) + .define(RETRIES_CONFIG, Type.INT, 0, between(0, Integer.MAX_VALUE), Importance.HIGH, RETRIES_DOC) + .define(ACKS_CONFIG, + Type.STRING, + "1", + in("all", "-1", "0", "1"), + Importance.HIGH, + ACKS_DOC) + .define(COMPRESSION_TYPE_CONFIG, Type.STRING, "none", Importance.HIGH, COMPRESSION_TYPE_DOC) + .define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC) + .define(TIMEOUT_CONFIG, Type.INT, 30 * 1000, atLeast(0), Importance.MEDIUM, TIMEOUT_DOC) + .define(LINGER_MS_CONFIG, Type.LONG, 0, atLeast(0L), Importance.MEDIUM, LINGER_MS_DOC) + .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CommonClientConfigs.CLIENT_ID_DOC) + .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.MEDIUM, CommonClientConfigs.SEND_BUFFER_DOC) + .define(RECEIVE_BUFFER_CONFIG, Type.INT, 32 * 1024, atLeast(0), Importance.MEDIUM, CommonClientConfigs.RECEIVE_BUFFER_DOC) + .define(MAX_REQUEST_SIZE_CONFIG, + Type.INT, + 1 * 1024 * 1024, + atLeast(0), + Importance.MEDIUM, + MAX_REQUEST_SIZE_DOC) + .define(BLOCK_ON_BUFFER_FULL_CONFIG, Type.BOOLEAN, true, Importance.LOW, BLOCK_ON_BUFFER_FULL_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 50L, atLeast(0L), Importance.LOW, CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, Type.LONG, 100L, atLeast(0L), Importance.LOW, CommonClientConfigs.RETRY_BACKOFF_MS_DOC) + .define(METADATA_FETCH_TIMEOUT_CONFIG, + Type.LONG, + 60 * 1000, + atLeast(0), + Importance.LOW, + METADATA_FETCH_TIMEOUT_DOC) + .define(METADATA_MAX_AGE_CONFIG, Type.LONG, 5 * 60 * 1000, atLeast(0), Importance.LOW, METADATA_MAX_AGE_DOC) + .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, + Type.LONG, + 30000, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) + .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) + .define(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, + Type.INT, + 5, + atLeast(1), + Importance.LOW, + MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC) + .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) + .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC) + /* default is set to be a bit lower than the server default (10 min), to avoid both client and server closing connection at same time */ + .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, Type.LONG, 9 * 60 * 1000, Importance.MEDIUM, CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) + .define(PARTITIONER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.producer.internals.DefaultPartitioner", Importance.MEDIUM, PARTITIONER_CLASS_DOC); + } + + public static Map addSerializerToConfig(Map configs, + Serializer keySerializer, Serializer valueSerializer) { + Map newConfigs = new HashMap(); + newConfigs.putAll(configs); + if (keySerializer != null) + newConfigs.put(KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass()); + if (valueSerializer != null) + newConfigs.put(VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass()); + return newConfigs; + } + + public static Properties addSerializerToConfig(Properties properties, + Serializer keySerializer, Serializer valueSerializer) { + Properties newProperties = new Properties(); + newProperties.putAll(properties); + if (keySerializer != null) + newProperties.put(KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass().getName()); + if (valueSerializer != null) + newProperties.put(VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass().getName()); + return newProperties; + } + + ProducerConfig(Map props) { + super(CONFIG, props); } - ProducerConfig(Map props) { - super(config, props); + public static void main(String[] args) { + System.out.println(CONFIG.toHtmlTable()); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java index 034bf33385fe3..75cd51e6d9a8a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.clients.producer; @@ -24,12 +20,12 @@ * specified but a key is present a partition will be chosen using a hash of the key. If neither key nor partition is * present a partition will be assigned in a round-robin fashion. */ -public final class ProducerRecord { +public final class ProducerRecord { private final String topic; private final Integer partition; - private final byte[] key; - private final byte[] value; + private final K key; + private final V value; /** * Creates a record to be sent to a specified topic and partition @@ -39,7 +35,7 @@ public final class ProducerRecord { * @param key The key that will be included in the record * @param value The record contents */ - public ProducerRecord(String topic, Integer partition, byte[] key, byte[] value) { + public ProducerRecord(String topic, Integer partition, K key, V value) { if (topic == null) throw new IllegalArgumentException("Topic cannot be null"); this.topic = topic; @@ -55,7 +51,7 @@ public ProducerRecord(String topic, Integer partition, byte[] key, byte[] value) * @param key The key that will be included in the record * @param value The record contents */ - public ProducerRecord(String topic, byte[] key, byte[] value) { + public ProducerRecord(String topic, K key, V value) { this(topic, null, key, value); } @@ -65,7 +61,7 @@ public ProducerRecord(String topic, byte[] key, byte[] value) { * @param topic The topic this record should be sent to * @param value The record contents */ - public ProducerRecord(String topic, byte[] value) { + public ProducerRecord(String topic, V value) { this(topic, null, value); } @@ -79,14 +75,14 @@ public String topic() { /** * The key (or null if no key is specified) */ - public byte[] key() { + public K key() { return key; } /** * @return The value */ - public byte[] value() { + public V value() { return value; } @@ -97,4 +93,40 @@ public Integer partition() { return partition; } + @Override + public String toString() { + String key = this.key == null ? "null" : this.key.toString(); + String value = this.value == null ? "null" : this.value.toString(); + return "ProducerRecord(topic=" + topic + ", partition=" + partition + ", key=" + key + ", value=" + value; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + else if (!(o instanceof ProducerRecord)) + return false; + + ProducerRecord that = (ProducerRecord) o; + + if (key != null ? !key.equals(that.key) : that.key != null) + return false; + else if (partition != null ? !partition.equals(that.partition) : that.partition != null) + return false; + else if (topic != null ? !topic.equals(that.topic) : that.topic != null) + return false; + else if (value != null ? !value.equals(that.value) : that.value != null) + return false; + + return true; + } + + @Override + public int hashCode() { + int result = topic != null ? topic.hashCode() : 0; + result = 31 * result + (partition != null ? partition.hashCode() : 0); + result = 31 * result + (key != null ? key.hashCode() : 0); + result = 31 * result + (value != null ? value.hashCode() : 0); + return result; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java index 8c776980ef1f5..8015f0da397e2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java @@ -26,12 +26,18 @@ public final class RecordMetadata { private final long offset; private final TopicPartition topicPartition; - public RecordMetadata(TopicPartition topicPartition, long offset) { + private RecordMetadata(TopicPartition topicPartition, long offset) { super(); this.offset = offset; this.topicPartition = topicPartition; } + public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset) { + // ignore the relativeOffset if the base offset is -1, + // since this indicates the offset is unknown + this(topicPartition, baseOffset == -1 ? baseOffset : baseOffset + relativeOffset); + } + /** * The offset of the record in the topic/partition. */ diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java index 5bed60730ea10..4cb1e50d6c4ed 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java @@ -19,10 +19,17 @@ import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.Deque; +import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; import org.apache.kafka.clients.producer.BufferExhaustedException; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.utils.Time; /** @@ -44,6 +51,9 @@ public final class BufferPool { private final Deque free; private final Deque waiters; private long availableMemory; + private final Metrics metrics; + private final Time time; + private final Sensor waitTime; /** * Create a new buffer pool @@ -53,8 +63,12 @@ public final class BufferPool { * @param blockOnExhaustion This controls the behavior when the buffer pool is out of memory. If true the * {@link #allocate(int)} call will block and wait for memory to be returned to the pool. If false * {@link #allocate(int)} will throw an exception if the buffer is out of memory. + * @param metrics instance of Metrics + * @param time time instance + * @param metricGrpName logical group name for metrics + * @param metricTags additional key/val attributes for metrics */ - public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion) { + public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion, Metrics metrics, Time time , String metricGrpName , Map metricTags) { this.poolableSize = poolableSize; this.blockOnExhaustion = blockOnExhaustion; this.lock = new ReentrantLock(); @@ -62,15 +76,24 @@ public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion) { this.waiters = new ArrayDeque(); this.totalMemory = memory; this.availableMemory = memory; + this.metrics = metrics; + this.time = time; + this.waitTime = this.metrics.sensor("bufferpool-wait-time"); + MetricName metricName = new MetricName("bufferpool-wait-ratio", + metricGrpName, + "The fraction of time an appender waits for space allocation.", + metricTags); + this.waitTime.add(metricName, new Rate(TimeUnit.NANOSECONDS)); } /** - * Allocate a buffer of the given size + * Allocate a buffer of the given size. This method blocks if there is not enough memory and the buffer pool + * is configured with blocking mode. * * @param size The buffer size to allocate in bytes * @return The buffer * @throws InterruptedException If the thread is interrupted while blocked - * @throws IllegalArgument if size is larger than the total memory controlled by the pool (and hence we would block + * @throws IllegalArgumentException if size is larger than the total memory controlled by the pool (and hence we would block * forever) * @throws BufferExhaustedException if the pool is in non-blocking mode and size exceeds the free memory in the pool */ @@ -110,7 +133,11 @@ public ByteBuffer allocate(int size) throws InterruptedException { // loop over and over until we have a buffer or have reserved // enough memory to allocate one while (accumulated < size) { + long startWait = time.nanoseconds(); moreMemory.await(); + long endWait = time.nanoseconds(); + this.waitTime.record(endWait - startWait, time.milliseconds()); + // check if we can satisfy this request from the free list, // otherwise allocate memory if (accumulated == 0 && size == this.poolableSize && !this.free.isEmpty()) { @@ -166,28 +193,31 @@ private void freeUp(int size) { * Return buffers to the pool. If they are of the poolable size add them to the free list, otherwise just mark the * memory as free. * - * @param buffers The buffers to return + * @param buffer The buffer to return + * @param size The size of the buffer to mark as deallocated, note that this maybe smaller than buffer.capacity + * since the buffer may re-allocate itself during in-place compression */ - public void deallocate(ByteBuffer... buffers) { + public void deallocate(ByteBuffer buffer, int size) { lock.lock(); try { - for (int i = 0; i < buffers.length; i++) { - int size = buffers[i].capacity(); - if (size == this.poolableSize) { - buffers[i].clear(); - this.free.add(buffers[i]); - } else { - this.availableMemory += size; - } - Condition moreMem = this.waiters.peekFirst(); - if (moreMem != null) - moreMem.signal(); + if (size == this.poolableSize && size == buffer.capacity()) { + buffer.clear(); + this.free.add(buffer); + } else { + this.availableMemory += size; } + Condition moreMem = this.waiters.peekFirst(); + if (moreMem != null) + moreMem.signal(); } finally { lock.unlock(); } } + public void deallocate(ByteBuffer buffer) { + deallocate(buffer, buffer.capacity()); + } + /** * the total free memory both unallocated and in the free list */ diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java new file mode 100644 index 0000000000000..f81c496b8f361 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java @@ -0,0 +1,89 @@ +/** + * 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 org.apache.kafka.clients.producer.internals; + +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.kafka.clients.producer.Partitioner; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.utils.Utils; + +/** + * The default partitioning strategy: + *
      + *
    • If a partition is specified in the record, use it + *
    • If no partition is specified but a key is present choose a partition based on a hash of the key + *
    • If no partition or key is present choose a partition in a round-robin fashion + */ +public class DefaultPartitioner implements Partitioner { + + private final AtomicInteger counter = new AtomicInteger(new Random().nextInt()); + + /** + * A cheap way to deterministically convert a number to a positive value. When the input is + * positive, the original value is returned. When the input number is negative, the returned + * positive value is the original value bit AND against 0x7fffffff which is not its absolutely + * value. + * + * Note: changing this method in the future will possibly cause partition selection not to be + * compatible with the existing messages already placed on a partition. + * + * @param number a given number + * @return a positive number. + */ + private static int toPositive(int number) { + return number & 0x7fffffff; + } + + public void configure(Map configs) {} + + /** + * Compute the partition for the given record. + * + * @param topic The topic name + * @param key The key to partition on (or null if no key) + * @param keyBytes serialized key to partition on (or null if no key) + * @param value The value to partition on or null + * @param valueBytes serialized value to partition on or null + * @param cluster The current cluster metadata + */ + public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster) { + List partitions = cluster.partitionsForTopic(topic); + int numPartitions = partitions.size(); + if (keyBytes == null) { + int nextValue = counter.getAndIncrement(); + List availablePartitions = cluster.availablePartitionsForTopic(topic); + if (availablePartitions.size() > 0) { + int part = DefaultPartitioner.toPositive(nextValue) % availablePartitions.size(); + return availablePartitions.get(part).partition(); + } else { + // no partitions are available, give a non-available partition + return DefaultPartitioner.toPositive(nextValue) % numPartitions; + } + } else { + // hash the keyBytes to choose a partition + return DefaultPartitioner.toPositive(Utils.murmur2(keyBytes)) % numPartitions; + } + } + + public void close() {} + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java new file mode 100644 index 0000000000000..747e29fadbcf6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java @@ -0,0 +1,51 @@ +/** + * 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 org.apache.kafka.clients.producer.internals; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ErrorLoggingCallback implements Callback { + private static final Logger log = LoggerFactory.getLogger(ErrorLoggingCallback.class); + private String topic; + private byte[] key; + private byte[] value; + private int valueLength; + private boolean logAsString; + + public ErrorLoggingCallback(String topic, byte[] key, byte[] value, boolean logAsString) { + this.topic = topic; + this.key = key; + + if (logAsString) { + this.value = value; + } + + this.valueLength = value == null ? -1 : value.length; + this.logAsString = logAsString; + } + + public void onCompletion(RecordMetadata metadata, Exception e) { + if (e != null) { + String keyString = (key == null) ? "null" : + logAsString ? new String(key) : key.length + " bytes"; + String valueString = (valueLength == -1) ? "null" : + logAsString ? new String(value) : valueLength + " bytes"; + log.error("Error when sending message to topic {} with key: {}, value: {} with error: {}", + topic, keyString, valueString, e.getMessage()); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java index 22d4c79bc06fb..e2d9ca87ad01b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.clients.producer.internals; @@ -23,7 +19,6 @@ import org.apache.kafka.clients.producer.RecordMetadata; - /** * The future result of a record send */ @@ -56,13 +51,17 @@ public RecordMetadata get(long timeout, TimeUnit unit) throws InterruptedExcepti return valueOrError(); } - private RecordMetadata valueOrError() throws ExecutionException { + RecordMetadata valueOrError() throws ExecutionException { if (this.result.error() != null) throw new ExecutionException(this.result.error()); else - return new RecordMetadata(result.topicPartition(), this.result.baseOffset() + this.relativeOffset); + return value(); } - + + RecordMetadata value() { + return new RecordMetadata(result.topicPartition(), this.result.baseOffset(), this.relativeOffset); + } + public long relativeOffset() { return this.relativeOffset; } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java deleted file mode 100644 index 52d30a86d0439..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java +++ /dev/null @@ -1,137 +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 org.apache.kafka.clients.producer.internals; - -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.errors.TimeoutException; - - -/** - * A class encapsulating some of the logic around metadata. - *

      - * This class is shared by the client thread (for partitioning) and the background sender thread. - * - * Metadata is maintained for only a subset of topics, which can be added to over time. When we request metdata for a - * topic we don't have any metadata for it will trigger a metadata update. - */ -public final class Metadata { - - private final long refreshBackoffMs; - private final long metadataExpireMs; - private long lastRefresh; - private Cluster cluster; - private boolean forceUpdate; - private final Set topics; - - /** - * Create a metadata instance with reasonable defaults - */ - public Metadata() { - this(100L, 60 * 60 * 1000L); - } - - /** - * Create a new Metadata instance - * @param refreshBackoffMs The minimum amount of time that must expire between metadata refreshes to avoid busy - * polling - * @param metadataExpireMs The maximum amount of time that metadata can be retained without refresh - */ - public Metadata(long refreshBackoffMs, long metadataExpireMs) { - this.refreshBackoffMs = refreshBackoffMs; - this.metadataExpireMs = metadataExpireMs; - this.lastRefresh = 0L; - this.cluster = Cluster.empty(); - this.forceUpdate = false; - this.topics = new HashSet(); - } - - /** - * Get the current cluster info without blocking - */ - public synchronized Cluster fetch() { - return this.cluster; - } - - /** - * Fetch cluster metadata including partitions for the given topic. If there is no metadata for the given topic, - * block waiting for an update. - * @param topic The topic we want metadata for - * @param maxWaitMs The maximum amount of time to block waiting for metadata - */ - public synchronized Cluster fetch(String topic, long maxWaitMs) { - List partitions = null; - do { - partitions = cluster.partitionsFor(topic); - if (partitions == null) { - long begin = System.currentTimeMillis(); - topics.add(topic); - forceUpdate = true; - try { - wait(maxWaitMs); - } catch (InterruptedException e) { /* this is fine, just try again */ - } - long ellapsed = System.currentTimeMillis() - begin; - if (ellapsed > maxWaitMs) - throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms."); - } else { - return cluster; - } - } while (true); - } - - /** - * Does the current cluster info need to be updated? An update is needed if it has been at least refreshBackoffMs - * since our last update and either (1) an update has been requested or (2) the current metadata has expired (more - * than metadataExpireMs has passed since the last refresh) - */ - public synchronized boolean needsUpdate(long now) { - long msSinceLastUpdate = now - this.lastRefresh; - boolean updateAllowed = msSinceLastUpdate >= this.refreshBackoffMs; - boolean updateNeeded = this.forceUpdate || msSinceLastUpdate >= this.metadataExpireMs; - return updateAllowed && updateNeeded; - } - - /** - * Force an update of the current cluster info - */ - public synchronized void forceUpdate() { - this.forceUpdate = true; - } - - /** - * Get the list of topics we are currently maintaining metadata for - */ - public synchronized Set topics() { - return new HashSet(this.topics); - } - - /** - * Update the cluster metadata - */ - public synchronized void update(Cluster cluster, long now) { - this.forceUpdate = false; - this.lastRefresh = now; - this.cluster = cluster; - notifyAll(); - } - -} diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java deleted file mode 100644 index fbb732a575221..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java +++ /dev/null @@ -1,72 +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 org.apache.kafka.clients.producer.internals; - -import java.util.List; -import java.util.Random; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.utils.Utils; - - -/** - * The default partitioning strategy: - *

        - *
      • If a partition is specified in the record, use it - *
      • If no partition is specified but a key is present choose a partition based on a hash of the key - *
      • If no partition or key is present choose a partition in a round-robin fashion - */ -public class Partitioner { - - private final AtomicInteger counter = new AtomicInteger(new Random().nextInt()); - - /** - * Compute the partition for the given record. - * - * @param record The record being sent - * @param numPartitions The total number of partitions for the given topic - */ - public int partition(ProducerRecord record, Cluster cluster) { - List partitions = cluster.partitionsFor(record.topic()); - int numPartitions = partitions.size(); - if (record.partition() != null) { - // they have given us a partition, use it - if (record.partition() < 0 || record.partition() >= numPartitions) - throw new IllegalArgumentException("Invalid partition given with record: " + record.partition() - + " is not in the range [0..." - + numPartitions - + "]."); - return record.partition(); - } else if (record.key() == null) { - // choose the next available node in a round-robin fashion - for (int i = 0; i < numPartitions; i++) { - int partition = Utils.abs(counter.getAndIncrement()) % numPartitions; - if (partitions.get(partition).leader() != null) - return partition; - } - // no partitions are available, give a non-available partition - return Utils.abs(counter.getAndIncrement()) % numPartitions; - } else { - // hash the key to choose a partition - return Utils.abs(Utils.murmur2(record.key())) % numPartitions; - } - } - -} diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java index b70ece71fdbd1..8e5855d9f41e2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java @@ -19,7 +19,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index be8a4a399b8b4..a152bd7697dca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -1,36 +1,28 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * 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 org.apache.kafka.clients.producer.internals; -import java.nio.ByteBuffer; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Deque; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentMap; - import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -38,123 +30,197 @@ import org.apache.kafka.common.utils.CopyOnWriteMap; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Deque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; /** - * This class acts as a queue that accumulates records into {@link org.apache.kafka.common.record.MemoryRecords} instances to be - * sent to the server. + * This class acts as a queue that accumulates records into {@link org.apache.kafka.common.record.MemoryRecords} + * instances to be sent to the server. *

        * The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless * this behavior is explicitly disabled. */ public final class RecordAccumulator { + private static final Logger log = LoggerFactory.getLogger(RecordAccumulator.class); + private volatile boolean closed; private int drainIndex; + private final AtomicInteger flushesInProgress; + private final AtomicInteger appendsInProgress; private final int batchSize; + private final CompressionType compression; private final long lingerMs; - private final ConcurrentMap> batches; + private final long retryBackoffMs; private final BufferPool free; private final Time time; + private final ConcurrentMap> batches; + private final IncompleteRecordBatches incomplete; + /** * Create a new record accumulator * * @param batchSize The size to use when allocating {@link org.apache.kafka.common.record.MemoryRecords} instances * @param totalSize The maximum memory the record accumulator can use. + * @param compression The compression codec for the records * @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for * sending. This allows time for more records to arrive. Setting a non-zero lingerMs will trade off some * latency for potentially better throughput due to more batching (and hence fewer, larger requests). + * @param retryBackoffMs An artificial delay time to retry the produce request upon receiving an error. This avoids + * exhausting all retries in a short period of time. * @param blockOnBufferFull If true block when we are out of memory; if false throw an exception when we are out of * memory * @param metrics The metrics * @param time The time instance to use + * @param metricTags additional key/value attributes of the metric */ - public RecordAccumulator(int batchSize, long totalSize, long lingerMs, boolean blockOnBufferFull, Metrics metrics, Time time) { + public RecordAccumulator(int batchSize, + long totalSize, + CompressionType compression, + long lingerMs, + long retryBackoffMs, + boolean blockOnBufferFull, + Metrics metrics, + Time time, + Map metricTags) { this.drainIndex = 0; this.closed = false; + this.flushesInProgress = new AtomicInteger(0); + this.appendsInProgress = new AtomicInteger(0); this.batchSize = batchSize; + this.compression = compression; this.lingerMs = lingerMs; + this.retryBackoffMs = retryBackoffMs; this.batches = new CopyOnWriteMap>(); - this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull); + String metricGrpName = "producer-metrics"; + this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull, metrics, time , metricGrpName , metricTags); + this.incomplete = new IncompleteRecordBatches(); this.time = time; - registerMetrics(metrics); - } - - private void registerMetrics(Metrics metrics) { - metrics.addMetric("blocked_threads", - "The number of user threads blocked waiting for buffer memory to enqueue their records", - new Measurable() { - public double measure(MetricConfig config, long now) { - return free.queued(); - } - }); - metrics.addMetric("buffer_total_bytes", - "The total amount of buffer memory that is available (not currently used for buffering records).", - new Measurable() { - public double measure(MetricConfig config, long now) { - return free.totalMemory(); - } - }); - metrics.addMetric("buffer_available_bytes", - "The total amount of buffer memory that is available (not currently used for buffering records).", - new Measurable() { - public double measure(MetricConfig config, long now) { - return free.availableMemory(); - } - }); + registerMetrics(metrics, metricGrpName, metricTags); + } + + private void registerMetrics(Metrics metrics, String metricGrpName, Map metricTags) { + MetricName metricName = new MetricName("waiting-threads", metricGrpName, "The number of user threads blocked waiting for buffer memory to enqueue their records", metricTags); + Measurable waitingThreads = new Measurable() { + public double measure(MetricConfig config, long now) { + return free.queued(); + } + }; + metrics.addMetric(metricName, waitingThreads); + + metricName = new MetricName("buffer-total-bytes", metricGrpName, "The maximum amount of buffer memory the client can use (whether or not it is currently used).", metricTags); + Measurable totalBytes = new Measurable() { + public double measure(MetricConfig config, long now) { + return free.totalMemory(); + } + }; + metrics.addMetric(metricName, totalBytes); + + metricName = new MetricName("buffer-available-bytes", metricGrpName, "The total amount of buffer memory that is not being used (either unallocated or in the free list).", metricTags); + Measurable availableBytes = new Measurable() { + public double measure(MetricConfig config, long now) { + return free.availableMemory(); + } + }; + metrics.addMetric(metricName, availableBytes); + + Sensor bufferExhaustedRecordSensor = metrics.sensor("buffer-exhausted-records"); + metricName = new MetricName("buffer-exhausted-rate", metricGrpName, "The average per-second number of record sends that are dropped due to buffer exhaustion", metricTags); + bufferExhaustedRecordSensor.add(metricName, new Rate()); } /** - * Add a record to the accumulator. + * Add a record to the accumulator, return the append result *

        - * This method will block if sufficient memory isn't available for the record unless blocking has been disabled. - * + * The append result will contain the future metadata, and flag for whether the appended batch is full or a new batch is created + *

        + * * @param tp The topic/partition to which this record is being sent * @param key The key for the record * @param value The value for the record - * @param compression The compression codec for the record * @param callback The user-supplied callback to execute when the request is complete */ - public FutureRecordMetadata append(TopicPartition tp, byte[] key, byte[] value, CompressionType compression, Callback callback) throws InterruptedException { - if (closed) - throw new IllegalStateException("Cannot send after the producer is closed."); - // check if we have an in-progress batch - Deque dq = dequeFor(tp); - synchronized (dq) { - RecordBatch batch = dq.peekLast(); - if (batch != null) { - FutureRecordMetadata future = batch.tryAppend(key, value, compression, callback); - if (future != null) - return future; + public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Callback callback) throws InterruptedException { + // We keep track of the number of appending thread to make sure we do not miss batches in + // abortIncompleteBatches(). + appendsInProgress.incrementAndGet(); + try { + if (closed) + throw new IllegalStateException("Cannot send after the producer is closed."); + // check if we have an in-progress batch + Deque dq = dequeFor(tp); + synchronized (dq) { + RecordBatch last = dq.peekLast(); + if (last != null) { + FutureRecordMetadata future = last.tryAppend(key, value, callback); + if (future != null) + return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false); + } } - } - // we don't have an in-progress record batch try to allocate a new batch - int size = Math.max(this.batchSize, Records.LOG_OVERHEAD + Record.recordSize(key, value)); - ByteBuffer buffer = free.allocate(size); - synchronized (dq) { - RecordBatch first = dq.peekLast(); - if (first != null) { - FutureRecordMetadata future = first.tryAppend(key, value, compression, callback); - if (future != null) { - // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen - // often... - free.deallocate(buffer); - return future; + // we don't have an in-progress record batch try to allocate a new batch + int size = Math.max(this.batchSize, Records.LOG_OVERHEAD + Record.recordSize(key, value)); + log.trace("Allocating a new {} byte message buffer for topic {} partition {}", size, tp.topic(), tp.partition()); + ByteBuffer buffer = free.allocate(size); + synchronized (dq) { + // Need to check if producer is closed again after grabbing the dequeue lock. + if (closed) + throw new IllegalStateException("Cannot send after the producer is closed."); + RecordBatch last = dq.peekLast(); + if (last != null) { + FutureRecordMetadata future = last.tryAppend(key, value, callback); + if (future != null) { + // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often... + free.deallocate(buffer); + return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false); + } } + MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize); + RecordBatch batch = new RecordBatch(tp, records, time.milliseconds()); + FutureRecordMetadata future = Utils.notNull(batch.tryAppend(key, value, callback)); + + dq.addLast(batch); + incomplete.add(batch); + return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true); } - RecordBatch batch = new RecordBatch(tp, new MemoryRecords(buffer), time.milliseconds()); - FutureRecordMetadata future = Utils.notNull(batch.tryAppend(key, value, compression, callback)); - dq.addLast(batch); - return future; + } finally { + appendsInProgress.decrementAndGet(); + } + } + + /** + * Re-enqueue the given record batch in the accumulator to retry + */ + public void reenqueue(RecordBatch batch, long now) { + batch.attempts++; + batch.lastAttemptMs = now; + Deque deque = dequeFor(batch.topicPartition); + synchronized (deque) { + deque.addFirst(batch); } } /** - * Get a list of topic-partitions which are ready to be sent. + * Get a list of nodes whose partitions are ready to be sent, and the earliest time at which any non-sendable + * partition will be ready; Also return the flag for whether there are any unknown leaders for the accumulated + * partition batches. *

        - * A partition is ready if ANY of the following are true: + * A destination node is ready to send data if ANY one of its partition is not backing off the send and ANY of the + * following are true : *

          *
        1. The record set is full *
        2. The record set has sat in the accumulator for at least lingerMs milliseconds @@ -163,57 +229,112 @@ public FutureRecordMetadata append(TopicPartition tp, byte[] key, byte[] value, *
        3. The accumulator has been closed *
        */ - public List ready(long now) { - List ready = new ArrayList(); + public ReadyCheckResult ready(Cluster cluster, long nowMs) { + Set readyNodes = new HashSet(); + long nextReadyCheckDelayMs = Long.MAX_VALUE; + boolean unknownLeadersExist = false; + boolean exhausted = this.free.queued() > 0; for (Map.Entry> entry : this.batches.entrySet()) { + TopicPartition part = entry.getKey(); Deque deque = entry.getValue(); - synchronized (deque) { - RecordBatch batch = deque.peekFirst(); - if (batch != null) { - boolean full = deque.size() > 1 || !batch.records.buffer().hasRemaining(); - boolean expired = now - batch.created >= lingerMs; - if (full | expired | exhausted | closed) - ready.add(batch.topicPartition); + + Node leader = cluster.leaderFor(part); + if (leader == null) { + unknownLeadersExist = true; + } else if (!readyNodes.contains(leader)) { + synchronized (deque) { + RecordBatch batch = deque.peekFirst(); + if (batch != null) { + boolean backingOff = batch.attempts > 0 && batch.lastAttemptMs + retryBackoffMs > nowMs; + long waitedTimeMs = nowMs - batch.lastAttemptMs; + long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs; + long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0); + boolean full = deque.size() > 1 || batch.records.isFull(); + boolean expired = waitedTimeMs >= timeToWaitMs; + boolean sendable = full || expired || exhausted || closed || flushInProgress(); + if (sendable && !backingOff) { + readyNodes.add(leader); + } else { + // Note that this results in a conservative estimate since an un-sendable partition may have + // a leader that will later be found to have sendable data. However, this is good enough + // since we'll just wake up and then sleep again for the remaining time. + nextReadyCheckDelayMs = Math.min(timeLeftMs, nextReadyCheckDelayMs); + } + } } } } - return ready; + + return new ReadyCheckResult(readyNodes, nextReadyCheckDelayMs, unknownLeadersExist); } /** - * Drain all the data for the given topic-partitions that will fit within the specified size. This method attempts - * to avoid choosing the same topic-partitions over and over. + * @return Whether there is any unsent record in the accumulator. + */ + public boolean hasUnsent() { + for (Map.Entry> entry : this.batches.entrySet()) { + Deque deque = entry.getValue(); + synchronized (deque) { + if (deque.size() > 0) + return true; + } + } + return false; + } + + /** + * Drain all the data for the given nodes and collate them into a list of batches that will fit within the specified + * size on a per-node basis. This method attempts to avoid choosing the same topic-node over and over. * - * @param partitions The list of partitions to drain + * @param cluster The current cluster metadata + * @param nodes The list of node to drain * @param maxSize The maximum number of bytes to drain - * @return A list of {@link RecordBatch} for partitions specified with total size less than the requested maxSize. - * TODO: There may be a starvation issue due to iteration order - */ - public List drain(List partitions, int maxSize) { - if (partitions.isEmpty()) - return Collections.emptyList(); - int size = 0; - List ready = new ArrayList(); - /* to make starvation less likely this loop doesn't start at 0 */ - int start = drainIndex = drainIndex % partitions.size(); - do { - TopicPartition tp = partitions.get(drainIndex); - Deque deque = dequeFor(tp); - if (deque != null) { - synchronized (deque) { - if (size + deque.peekFirst().records.sizeInBytes() > maxSize) { - return ready; - } else { - RecordBatch batch = deque.pollFirst(); - size += batch.records.sizeInBytes(); - ready.add(batch); + * @param now The current unix time in milliseconds + * @return A list of {@link RecordBatch} for each node specified with total size less than the requested maxSize. + */ + public Map> drain(Cluster cluster, Set nodes, int maxSize, long now) { + if (nodes.isEmpty()) + return Collections.emptyMap(); + + Map> batches = new HashMap>(); + for (Node node : nodes) { + int size = 0; + List parts = cluster.partitionsForNode(node.id()); + List ready = new ArrayList(); + /* to make starvation less likely this loop doesn't start at 0 */ + int start = drainIndex = drainIndex % parts.size(); + do { + PartitionInfo part = parts.get(drainIndex); + Deque deque = dequeFor(new TopicPartition(part.topic(), part.partition())); + if (deque != null) { + synchronized (deque) { + RecordBatch first = deque.peekFirst(); + if (first != null) { + boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now; + // Only drain the batch if it is not during backoff period. + if (!backoff) { + if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) { + // there is a rare case that a single batch size is larger than the request size due + // to compression; in this case we will still eventually send this batch in a single + // request + break; + } else { + RecordBatch batch = deque.pollFirst(); + batch.records.close(); + size += batch.records.sizeInBytes(); + ready.add(batch); + batch.drainedMs = now; + } + } + } } } - } - this.drainIndex = (this.drainIndex + 1) % partitions.size(); - } while (start != drainIndex); - return ready; + this.drainIndex = (this.drainIndex + 1) % parts.size(); + } while (start != drainIndex); + batches.put(node.id(), ready); + } + return batches; } /** @@ -229,16 +350,75 @@ private Deque dequeFor(TopicPartition tp) { } /** - * Deallocate the list of record batches - */ - public void deallocate(Collection batches) { - ByteBuffer[] buffers = new ByteBuffer[batches.size()]; - int i = 0; - for (RecordBatch batch : batches) { - buffers[i] = batch.records.buffer(); - i++; + * Deallocate the record batch + */ + public void deallocate(RecordBatch batch) { + incomplete.remove(batch); + free.deallocate(batch.records.buffer(), batch.records.capacity()); + } + + /** + * Are there any threads currently waiting on a flush? + */ + private boolean flushInProgress() { + return flushesInProgress.get() > 0; + } + + /** + * Initiate the flushing of data from the accumulator...this makes all requests immediately ready + */ + public void beginFlush() { + this.flushesInProgress.getAndIncrement(); + } + + /** + * Are there any threads currently appending messages? + */ + private boolean appendsInProgress() { + return appendsInProgress.get() > 0; + } + + /** + * Mark all partitions as ready to send and block until the send is complete + */ + public void awaitFlushCompletion() throws InterruptedException { + for (RecordBatch batch: this.incomplete.all()) + batch.produceFuture.await(); + this.flushesInProgress.decrementAndGet(); + } + + /** + * This function is only called when sender is closed forcefully. It will fail all the + * incomplete batches and return. + */ + public void abortIncompleteBatches() { + // We need to keep aborting the incomplete batch until no thread is trying to append to + // 1. Avoid losing batches. + // 2. Free up memory in case appending threads are blocked on buffer full. + // This is a tight loop but should be able to get through very quickly. + do { + abortBatches(); + } while (appendsInProgress()); + // After this point, no thread will append any messages because they will see the close + // flag set. We need to do the last abort after no thread was appending in case the there was a new + // batch appended by the last appending thread. + abortBatches(); + this.batches.clear(); + } + + /** + * Go through incomplete batches and abort them. + */ + private void abortBatches() { + for (RecordBatch batch : incomplete.all()) { + Deque dq = dequeFor(batch.topicPartition); + // Close the batch before aborting + synchronized (dq) { + batch.records.close(); + } + batch.done(-1L, new IllegalStateException("Producer is closed forcefully.")); + deallocate(batch); } - free.deallocate(buffers); } /** @@ -248,4 +428,64 @@ public void close() { this.closed = true; } + /* + * Metadata about a record just appended to the record accumulator + */ + public final static class RecordAppendResult { + public final FutureRecordMetadata future; + public final boolean batchIsFull; + public final boolean newBatchCreated; + + public RecordAppendResult(FutureRecordMetadata future, boolean batchIsFull, boolean newBatchCreated) { + this.future = future; + this.batchIsFull = batchIsFull; + this.newBatchCreated = newBatchCreated; + } + } + + /* + * The set of nodes that have at least one complete record batch in the accumulator + */ + public final static class ReadyCheckResult { + public final Set readyNodes; + public final long nextReadyCheckDelayMs; + public final boolean unknownLeadersExist; + + public ReadyCheckResult(Set readyNodes, long nextReadyCheckDelayMs, boolean unknownLeadersExist) { + this.readyNodes = readyNodes; + this.nextReadyCheckDelayMs = nextReadyCheckDelayMs; + this.unknownLeadersExist = unknownLeadersExist; + } + } + + /* + * A threadsafe helper class to hold RecordBatches that haven't been ack'd yet + */ + private final static class IncompleteRecordBatches { + private final Set incomplete; + + public IncompleteRecordBatches() { + this.incomplete = new HashSet(); + } + + public void add(RecordBatch batch) { + synchronized (incomplete) { + this.incomplete.add(batch); + } + } + + public void remove(RecordBatch batch) { + synchronized (incomplete) { + boolean removed = this.incomplete.remove(batch); + if (!removed) + throw new IllegalStateException("Remove from the incomplete set failed. This should be impossible."); + } + } + + public Iterable all() { + synchronized (incomplete) { + return new ArrayList(this.incomplete); + } + } + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java index 7a440a3dd29c7..06182db1c3a5d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.clients.producer.internals; @@ -22,9 +18,10 @@ import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; - +import org.apache.kafka.common.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A batch of records that is or will be sent. @@ -32,15 +29,23 @@ * This class is not thread safe and external synchronization must be used when modifying it */ public final class RecordBatch { + + private static final Logger log = LoggerFactory.getLogger(RecordBatch.class); + public int recordCount = 0; - public final long created; + public int maxRecordSize = 0; + public volatile int attempts = 0; + public final long createdMs; + public long drainedMs; + public long lastAttemptMs; public final MemoryRecords records; public final TopicPartition topicPartition; - private final ProduceRequestResult produceFuture; + public final ProduceRequestResult produceFuture; private final List thunks; public RecordBatch(TopicPartition tp, MemoryRecords records, long now) { - this.created = now; + this.createdMs = now; + this.lastAttemptMs = now; this.records = records; this.topicPartition = tp; this.produceFuture = new ProduceRequestResult(); @@ -52,14 +57,15 @@ public RecordBatch(TopicPartition tp, MemoryRecords records, long now) { * * @return The RecordSend corresponding to this record or null if there isn't sufficient room. */ - public FutureRecordMetadata tryAppend(byte[] key, byte[] value, CompressionType compression, Callback callback) { + public FutureRecordMetadata tryAppend(byte[] key, byte[] value, Callback callback) { if (!this.records.hasRoomFor(key, value)) { return null; } else { - this.records.append(0L, key, value, compression); + this.records.append(0L, key, value); + this.maxRecordSize = Math.max(this.maxRecordSize, Record.recordSize(key, value)); FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount); if (callback != null) - thunks.add(new Thunk(callback, this.recordCount)); + thunks.add(new Thunk(callback, future)); this.recordCount++; return future; } @@ -68,36 +74,46 @@ public FutureRecordMetadata tryAppend(byte[] key, byte[] value, CompressionType /** * Complete the request * - * @param offset The offset - * @param errorCode The error code or 0 if no error + * @param baseOffset The base offset of the messages assigned by the server + * @param exception The exception that occurred (or null if the request was successful) */ - public void done(long offset, RuntimeException exception) { - this.produceFuture.done(topicPartition, offset, exception); + public void done(long baseOffset, RuntimeException exception) { + log.trace("Produced messages to topic-partition {} with base offset offset {} and error: {}.", + topicPartition, + baseOffset, + exception); // execute callbacks for (int i = 0; i < this.thunks.size(); i++) { try { Thunk thunk = this.thunks.get(i); - if (exception == null) - thunk.callback.onCompletion(new RecordMetadata(topicPartition, this.produceFuture.baseOffset() + thunk.relativeOffset), - null); - else + if (exception == null) { + RecordMetadata metadata = new RecordMetadata(this.topicPartition, baseOffset, thunk.future.relativeOffset()); + thunk.callback.onCompletion(metadata, null); + } else { thunk.callback.onCompletion(null, exception); + } } catch (Exception e) { - e.printStackTrace(); + log.error("Error executing user-provided callback on message for topic-partition {}:", topicPartition, e); } } + this.produceFuture.done(topicPartition, baseOffset, exception); } /** - * A callback and the associated RecordSend argument to pass to it. + * A callback and the associated FutureRecordMetadata argument to pass to it. */ final private static class Thunk { final Callback callback; - final long relativeOffset; + final FutureRecordMetadata future; - public Thunk(Callback callback, long relativeOffset) { + public Thunk(Callback callback, FutureRecordMetadata future) { this.callback = callback; - this.relativeOffset = relativeOffset; + this.future = future; } } + + @Override + public String toString() { + return "RecordBatch(topicPartition=" + topicPartition + ", recordCount=" + recordCount + ")"; + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index b274e5e4376b6..0baf16e55046a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -1,49 +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. + * 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 org.apache.kafka.clients.producer.internals; -import java.io.IOException; -import java.net.InetSocketAddress; import java.nio.ByteBuffer; -import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Collections; -import java.util.Deque; import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Set; +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.NetworkException; -import org.apache.kafka.common.network.NetworkReceive; -import org.apache.kafka.common.network.NetworkSend; -import org.apache.kafka.common.network.Selectable; +import org.apache.kafka.common.errors.InvalidMetadataException; +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.ProduceRequest; +import org.apache.kafka.common.requests.ProduceResponse; import org.apache.kafka.common.requests.RequestSend; -import org.apache.kafka.common.requests.ResponseHeader; import org.apache.kafka.common.utils.Time; - +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * The background thread that handles the sending of produce requests to the Kafka cluster. This thread makes metadata @@ -51,135 +54,159 @@ */ public class Sender implements Runnable { - private final Map nodeState; + private static final Logger log = LoggerFactory.getLogger(Sender.class); + + /* the state of each nodes connection */ + private final KafkaClient client; + + /* the record accumulator that batches records */ private final RecordAccumulator accumulator; - private final Selectable selector; - private final String clientId; + + /* the metadata for the client */ + private final Metadata metadata; + + /* the maximum request size to attempt to send to the server */ private final int maxRequestSize; - private final long reconnectBackoffMs; + + /* the number of acknowledgements to request from the server */ private final short acks; + + /* the max time in ms for the server to wait for acknowlegements */ private final int requestTimeout; - private final InFlightRequests inFlightRequests; - private final Metadata metadata; + + /* the number of times to retry a failed request before giving up */ + private final int retries; + + /* the clock instance used for getting the time */ private final Time time; - private int correlation; - private boolean metadataFetchInProgress; + + /* true while the sender thread is still running */ private volatile boolean running; - public Sender(Selectable selector, + /* true when the caller wants to ignore all unsent/inflight messages and force close. */ + private volatile boolean forceClose; + + /* metrics */ + private final SenderMetrics sensors; + + /* param clientId of the client */ + private String clientId; + + public Sender(KafkaClient client, Metadata metadata, RecordAccumulator accumulator, - String clientId, int maxRequestSize, - long reconnectBackoffMs, short acks, + int retries, int requestTimeout, - Time time) { - this.nodeState = new HashMap(); + Metrics metrics, + Time time, + String clientId) { + this.client = client; this.accumulator = accumulator; - this.selector = selector; - this.maxRequestSize = maxRequestSize; - this.reconnectBackoffMs = reconnectBackoffMs; this.metadata = metadata; - this.clientId = clientId; + this.maxRequestSize = maxRequestSize; this.running = true; this.requestTimeout = requestTimeout; this.acks = acks; - this.inFlightRequests = new InFlightRequests(); - this.correlation = 0; - this.metadataFetchInProgress = false; + this.retries = retries; this.time = time; + this.clientId = clientId; + this.sensors = new SenderMetrics(metrics); } /** * The main run loop for the sender thread */ public void run() { + log.debug("Starting Kafka producer I/O thread."); + // main loop, runs until close is called while (running) { try { run(time.milliseconds()); } catch (Exception e) { - e.printStackTrace(); + log.error("Uncaught error in kafka producer I/O thread: ", e); } } - // send anything left in the accumulator - int unsent = 0; - do { + log.debug("Beginning shutdown of Kafka producer I/O thread, sending remaining records."); + + // okay we stopped accepting requests but there may still be + // requests in the accumulator or waiting for acknowledgment, + // wait until these are completed. + while (!forceClose && (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0)) { try { - unsent = run(time.milliseconds()); + run(time.milliseconds()); } catch (Exception e) { - e.printStackTrace(); + log.error("Uncaught error in kafka producer I/O thread: ", e); } - } while (unsent > 0); + } + if (forceClose) { + // We need to fail all the incomplete batches and wake up the threads waiting on + // the futures. + this.accumulator.abortIncompleteBatches(); + } + try { + this.client.close(); + } catch (Exception e) { + log.error("Failed to close network client", e); + } - // close all the connections - this.selector.close(); + log.debug("Shutdown of Kafka producer I/O thread has completed."); } /** * Run a single iteration of sending * - * @param now The current time - * @return The total number of topic/partitions that had data ready (regardless of what we actually sent) + * @param now + * The current POSIX time in milliseconds */ - public int run(long now) { + public void run(long now) { Cluster cluster = metadata.fetch(); // get the list of partitions with data ready to send - List ready = this.accumulator.ready(now); - - // prune the list of ready topics to eliminate any that we aren't ready to send yet - List sendable = processReadyPartitions(cluster, ready, now); - - // should we update our metadata? - List sends = new ArrayList(sendable.size()); - InFlightRequest metadataReq = maybeMetadataRequest(cluster, now); - if (metadataReq != null) { - sends.add(metadataReq.request); - this.inFlightRequests.add(metadataReq); + RecordAccumulator.ReadyCheckResult result = this.accumulator.ready(cluster, now); + + // if there are any partitions whose leaders are not known yet, force metadata update + if (result.unknownLeadersExist) + this.metadata.requestUpdate(); + + // remove any nodes we aren't ready to send to + Iterator iter = result.readyNodes.iterator(); + long notReadyTimeout = Long.MAX_VALUE; + while (iter.hasNext()) { + Node node = iter.next(); + if (!this.client.ready(node, now)) { + iter.remove(); + notReadyTimeout = Math.min(notReadyTimeout, this.client.connectionDelay(node, now)); + } } // create produce requests - List batches = this.accumulator.drain(sendable, this.maxRequestSize); - List requests = collate(cluster, batches); - for (int i = 0; i < requests.size(); i++) { - InFlightRequest request = requests.get(i); - this.inFlightRequests.add(request); - sends.add(request.request); - } - - // do the I/O - try { - this.selector.poll(5L, sends); - } catch (IOException e) { - e.printStackTrace(); - } - - // handle responses, connections, and disconnections - handleSends(this.selector.completedSends()); - handleResponses(this.selector.completedReceives(), now); - handleDisconnects(this.selector.disconnected()); - handleConnects(this.selector.connected()); - - return ready.size(); - } - - private InFlightRequest maybeMetadataRequest(Cluster cluster, long now) { - if (this.metadataFetchInProgress || !metadata.needsUpdate(now)) - return null; - Node node = cluster.nextNode(); - NodeState state = nodeState.get(node.id()); - if (state == null || (state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttempt > this.reconnectBackoffMs)) { - // we don't have a connection to this node right now, make one - initiateConnect(node, now); - return null; - } else if (state.state == ConnectionState.CONNECTED) { - this.metadataFetchInProgress = true; - return metadataRequest(node.id(), metadata.topics()); - } else { - return null; + Map> batches = this.accumulator.drain(cluster, + result.readyNodes, + this.maxRequestSize, + now); + sensors.updateProduceRequestMetrics(batches); + List requests = createProduceRequests(batches, now); + // If we have any nodes that are ready to send + have sendable data, poll with 0 timeout so this can immediately + // loop and try sending more data. Otherwise, the timeout is determined by nodes that have partitions with data + // that isn't yet sendable (e.g. lingering, backing off). Note that this specifically does not include nodes + // with sendable data that aren't ready to send since they would cause busy looping. + long pollTimeout = Math.min(result.nextReadyCheckDelayMs, notReadyTimeout); + if (result.readyNodes.size() > 0) { + log.trace("Nodes with data ready to send: {}", result.readyNodes); + log.trace("Created {} produce requests: {}", requests.size(), requests); + pollTimeout = 0; } + for (ClientRequest request : requests) + client.send(request); + + // if some partitions are already ready to be sent, the select time would be 0; + // otherwise if some partition already has some data accumulated but not ready yet, + // the select time will be the time difference between now and its linger expiry time; + // otherwise the select time will be the time difference between now and the metadata expiry time; + this.client.poll(pollTimeout, now); } /** @@ -188,332 +215,304 @@ private InFlightRequest maybeMetadataRequest(Cluster cluster, long now) { public void initiateClose() { this.running = false; this.accumulator.close(); + this.wakeup(); } /** - * Process the set of topic-partitions with data ready to send. If we have a connection to the appropriate node, add - * it to the returned set. For any partitions we have no connection to either make one, fetch the appropriate - * metdata to be able to do so - */ - private List processReadyPartitions(Cluster cluster, List ready, long now) { - List sendable = new ArrayList(ready.size()); - for (TopicPartition tp : ready) { - Node node = cluster.leaderFor(tp); - if (node == null) { - // we don't know about this topic/partition or it has no leader, re-fetch metadata - metadata.forceUpdate(); - } else { - NodeState state = nodeState.get(node.id()); - // TODO: encapsulate this logic somehow - if (state == null || (state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttempt > this.reconnectBackoffMs)) { - // we don't have a connection to this node right now, make one - initiateConnect(node, now); - } else if (state.state == ConnectionState.CONNECTED && inFlightRequests.canSendMore(node.id())) { - sendable.add(tp); - } - } - } - return sendable; - } - - /** - * Initiate a connection to the given node + * Closes the sender without sending out any pending messages. */ - private void initiateConnect(Node node, long now) { - try { - selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), 64 * 1024 * 1024, 64 * 1024 * 1024); // TODO - // socket - // buffers - nodeState.put(node.id(), new NodeState(ConnectionState.CONNECTING, now)); - } catch (IOException e) { - /* attempt failed, we'll try again after the backoff */ - nodeState.put(node.id(), new NodeState(ConnectionState.DISCONNECTED, now)); - /* maybe the problem is our metadata, update it */ - metadata.forceUpdate(); - } + public void forceClose() { + this.forceClose = true; + initiateClose(); } /** - * Handle any closed connections - */ - private void handleDisconnects(List disconnects) { - for (int node : disconnects) { - for (InFlightRequest request : this.inFlightRequests.clearAll(node)) { - if (request.batches != null) { - for (RecordBatch batch : request.batches.values()) - batch.done(-1L, new NetworkException("The server disconnected unexpectedly without sending a response.")); - this.accumulator.deallocate(request.batches.values()); - } - NodeState state = this.nodeState.get(request.request.destination()); - if (state != null) - state.state = ConnectionState.DISCONNECTED; - } - } - } - - /** - * Record any connections that completed in our node state - */ - private void handleConnects(List connects) { - for (Integer id : connects) - this.nodeState.get(id).state = ConnectionState.CONNECTED; - } - - /** - * Process completed sends + * Handle a produce response */ - public void handleSends(List sends) { - /* if acks = 0 then the request is satisfied once sent */ - for (NetworkSend send : sends) { - Deque requests = this.inFlightRequests.requestQueue(send.destination()); - InFlightRequest request = requests.peekFirst(); - if (!request.expectResponse) { - requests.pollFirst(); - if (request.request.header().apiKey() == ApiKeys.PRODUCE.id) { - for (RecordBatch batch : request.batches.values()) - batch.done(-1L, Errors.NONE.exception()); - this.accumulator.deallocate(request.batches.values()); + private void handleProduceResponse(ClientResponse response, Map batches, long now) { + int correlationId = response.request().request().header().correlationId(); + if (response.wasDisconnected()) { + log.trace("Cancelled request {} due to node {} being disconnected", response, response.request() + .request() + .destination()); + for (RecordBatch batch : batches.values()) + completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, correlationId, now); + } else { + log.trace("Received produce response from node {} with correlation id {}", + response.request().request().destination(), + correlationId); + // if we have a response, parse it + if (response.hasResponse()) { + ProduceResponse produceResponse = new ProduceResponse(response.responseBody()); + for (Map.Entry entry : produceResponse.responses() + .entrySet()) { + TopicPartition tp = entry.getKey(); + ProduceResponse.PartitionResponse partResp = entry.getValue(); + Errors error = Errors.forCode(partResp.errorCode); + RecordBatch batch = batches.get(tp); + completeBatch(batch, error, partResp.baseOffset, correlationId, now); } + this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs()); + } else { + // this is the acks = 0 case, just complete all requests + for (RecordBatch batch : batches.values()) + completeBatch(batch, Errors.NONE, -1L, correlationId, now); } } } /** - * Handle responses from the server - */ - private void handleResponses(List receives, long now) { - for (NetworkReceive receive : receives) { - int source = receive.source(); - InFlightRequest req = inFlightRequests.nextCompleted(source); - ResponseHeader header = ResponseHeader.parse(receive.payload()); - short apiKey = req.request.header().apiKey(); - Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload()); - correlate(req.request.header(), header); - if (req.request.header().apiKey() == ApiKeys.PRODUCE.id) - handleProduceResponse(req, body); - else if (req.request.header().apiKey() == ApiKeys.METADATA.id) - handleMetadataResponse(body, now); - else - throw new IllegalStateException("Unexpected response type: " + req.request.header().apiKey()); - } - } - - private void handleMetadataResponse(Struct body, long now) { - this.metadataFetchInProgress = false; - Cluster cluster = ProtoUtils.parseMetadataResponse(body); - this.metadata.update(cluster, now); - } - - /** - * Handle a produce response + * Complete or retry the given batch of records. + * + * @param batch The record batch + * @param error The error (or null if none) + * @param baseOffset The base offset assigned to the records if successful + * @param correlationId The correlation id for the request + * @param now The current POSIX time stamp in milliseconds */ - private void handleProduceResponse(InFlightRequest request, Struct response) { - for (Object topicResponse : (Object[]) response.get("responses")) { - Struct topicRespStruct = (Struct) topicResponse; - String topic = (String) topicRespStruct.get("topic"); - for (Object partResponse : (Object[]) topicRespStruct.get("partition_responses")) { - Struct partRespStruct = (Struct) partResponse; - int partition = (Integer) partRespStruct.get("partition"); - short errorCode = (Short) partRespStruct.get("error_code"); - long offset = (Long) partRespStruct.get("base_offset"); - RecordBatch batch = request.batches.get(new TopicPartition(topic, partition)); - batch.done(offset, Errors.forCode(errorCode).exception()); - } + private void completeBatch(RecordBatch batch, Errors error, long baseOffset, long correlationId, long now) { + if (error != Errors.NONE && canRetry(batch, error)) { + // retry + log.warn("Got error produce response with correlation id {} on topic-partition {}, retrying ({} attempts left). Error: {}", + correlationId, + batch.topicPartition, + this.retries - batch.attempts - 1, + error); + this.accumulator.reenqueue(batch, now); + this.sensors.recordRetries(batch.topicPartition.topic(), batch.recordCount); + } else { + // tell the user the result of their request + batch.done(baseOffset, error.exception()); + this.accumulator.deallocate(batch); + if (error != Errors.NONE) + this.sensors.recordErrors(batch.topicPartition.topic(), batch.recordCount); } - this.accumulator.deallocate(request.batches.values()); - } - - /** - * Validate that the response corresponds to the request we expect or else explode - */ - private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) { - if (requestHeader.correlationId() != responseHeader.correlationId()) - throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() - + ") does not match request (" - + requestHeader.correlationId() - + ")"); + if (error.exception() instanceof InvalidMetadataException) + metadata.requestUpdate(); } /** - * Create a metadata request for the given topics + * We can retry a send if the error is transient and the number of attempts taken is fewer than the maximum allowed */ - private InFlightRequest metadataRequest(int node, Set topics) { - String[] ts = new String[topics.size()]; - topics.toArray(ts); - Struct body = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id)); - body.set("topics", topics.toArray()); - RequestSend send = new RequestSend(node, new RequestHeader(ApiKeys.METADATA.id, clientId, correlation++), body); - return new InFlightRequest(true, send, null); + private boolean canRetry(RecordBatch batch, Errors error) { + return batch.attempts < this.retries && error.exception() instanceof RetriableException; } /** - * Collate the record batches into a list of produce requests on a per-node basis + * Transfer the record batches into a list of produce requests on a per-node basis */ - private List collate(Cluster cluster, List batches) { - Map> collated = new HashMap>(); - for (RecordBatch batch : batches) { - Node node = cluster.leaderFor(batch.topicPartition); - List found = collated.get(node.id()); - if (found == null) { - found = new ArrayList(); - collated.put(node.id(), found); - } - found.add(batch); - } - List requests = new ArrayList(collated.size()); + private List createProduceRequests(Map> collated, long now) { + List requests = new ArrayList(collated.size()); for (Map.Entry> entry : collated.entrySet()) - requests.add(produceRequest(entry.getKey(), acks, requestTimeout, entry.getValue())); + requests.add(produceRequest(now, entry.getKey(), acks, requestTimeout, entry.getValue())); return requests; } /** * Create a produce request from the given record batches */ - private InFlightRequest produceRequest(int destination, short acks, int timeout, List batches) { - Map batchesByPartition = new HashMap(); - Map> batchesByTopic = new HashMap>(); + private ClientRequest produceRequest(long now, int destination, short acks, int timeout, List batches) { + Map produceRecordsByPartition = new HashMap(batches.size()); + final Map recordsByPartition = new HashMap(batches.size()); for (RecordBatch batch : batches) { - batchesByPartition.put(batch.topicPartition, batch); - List found = batchesByTopic.get(batch.topicPartition.topic()); - if (found == null) { - found = new ArrayList(); - batchesByTopic.put(batch.topicPartition.topic(), found); - } - found.add(batch); + TopicPartition tp = batch.topicPartition; + produceRecordsByPartition.put(tp, (ByteBuffer) batch.records.buffer().flip()); + recordsByPartition.put(tp, batch); } - Struct produce = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id)); - produce.set("acks", acks); - produce.set("timeout", timeout); - List topicDatas = new ArrayList(batchesByTopic.size()); - for (Map.Entry> entry : batchesByTopic.entrySet()) { - Struct topicData = produce.instance("topic_data"); - topicData.set("topic", entry.getKey()); - List parts = entry.getValue(); - Object[] partitionData = new Object[parts.size()]; - for (int i = 0; i < parts.size(); i++) { - ByteBuffer buffer = parts.get(i).records.buffer(); - buffer.flip(); - Struct part = topicData.instance("data") - .set("partition", parts.get(i).topicPartition.partition()) - .set("record_set", buffer); - partitionData[i] = part; + ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition); + RequestSend send = new RequestSend(Integer.toString(destination), + this.client.nextRequestHeader(ApiKeys.PRODUCE), + request.toStruct()); + RequestCompletionHandler callback = new RequestCompletionHandler() { + public void onComplete(ClientResponse response) { + handleProduceResponse(response, recordsByPartition, time.milliseconds()); } - topicData.set("data", partitionData); - topicDatas.add(topicData); - } - produce.set("topic_data", topicDatas.toArray()); - - RequestHeader header = new RequestHeader(ApiKeys.PRODUCE.id, clientId, correlation++); - RequestSend send = new RequestSend(destination, header, produce); - return new InFlightRequest(acks != 0, send, batchesByPartition); + }; + return new ClientRequest(now, acks != 0, send, callback); } /** * Wake up the selector associated with this send thread */ public void wakeup() { - this.selector.wakeup(); + this.client.wakeup(); } /** - * The states of a node connection + * A collection of sensors for the sender */ - private static enum ConnectionState { - DISCONNECTED, CONNECTING, CONNECTED - } - - /** - * The state of a node - */ - private static final class NodeState { - private ConnectionState state; - private long lastConnectAttempt; - - public NodeState(ConnectionState state, long lastConnectAttempt) { - this.state = state; - this.lastConnectAttempt = lastConnectAttempt; - } - - public String toString() { - return "NodeState(" + state + ", " + lastConnectAttempt + ")"; - } - } - - /** - * An request that hasn't been fully processed yet - */ - private static final class InFlightRequest { - public boolean expectResponse; - public Map batches; - public RequestSend request; - - /** - * @param expectResponse Should we expect a response message or is this request complete once it is sent? - * @param request The request - * @param batches The record batches contained in the request if it is a produce request - */ - public InFlightRequest(boolean expectResponse, RequestSend request, Map batches) { - this.batches = batches; - this.request = request; - this.expectResponse = expectResponse; + private class SenderMetrics { + + private final Metrics metrics; + public final Sensor retrySensor; + public final Sensor errorSensor; + public final Sensor queueTimeSensor; + public final Sensor requestTimeSensor; + public final Sensor recordsPerRequestSensor; + public final Sensor batchSizeSensor; + public final Sensor compressionRateSensor; + public final Sensor maxRecordSizeSensor; + + public SenderMetrics(Metrics metrics) { + this.metrics = metrics; + Map metricTags = new LinkedHashMap(); + metricTags.put("client-id", clientId); + String metricGrpName = "producer-metrics"; + + this.batchSizeSensor = metrics.sensor("batch-size"); + MetricName m = new MetricName("batch-size-avg", metricGrpName, "The average number of bytes sent per partition per-request.", metricTags); + this.batchSizeSensor.add(m, new Avg()); + m = new MetricName("batch-size-max", metricGrpName, "The max number of bytes sent per partition per-request.", metricTags); + this.batchSizeSensor.add(m, new Max()); + + this.compressionRateSensor = metrics.sensor("compression-rate"); + m = new MetricName("compression-rate-avg", metricGrpName, "The average compression rate of record batches.", metricTags); + this.compressionRateSensor.add(m, new Avg()); + + this.queueTimeSensor = metrics.sensor("queue-time"); + m = new MetricName("record-queue-time-avg", metricGrpName, "The average time in ms record batches spent in the record accumulator.", metricTags); + this.queueTimeSensor.add(m, new Avg()); + m = new MetricName("record-queue-time-max", metricGrpName, "The maximum time in ms record batches spent in the record accumulator.", metricTags); + this.queueTimeSensor.add(m, new Max()); + + this.requestTimeSensor = metrics.sensor("request-time"); + m = new MetricName("request-latency-avg", metricGrpName, "The average request latency in ms", metricTags); + this.requestTimeSensor.add(m, new Avg()); + m = new MetricName("request-latency-max", metricGrpName, "The maximum request latency in ms", metricTags); + this.requestTimeSensor.add(m, new Max()); + + this.recordsPerRequestSensor = metrics.sensor("records-per-request"); + m = new MetricName("record-send-rate", metricGrpName, "The average number of records sent per second.", metricTags); + this.recordsPerRequestSensor.add(m, new Rate()); + m = new MetricName("records-per-request-avg", metricGrpName, "The average number of records per request.", metricTags); + this.recordsPerRequestSensor.add(m, new Avg()); + + this.retrySensor = metrics.sensor("record-retries"); + m = new MetricName("record-retry-rate", metricGrpName, "The average per-second number of retried record sends", metricTags); + this.retrySensor.add(m, new Rate()); + + this.errorSensor = metrics.sensor("errors"); + m = new MetricName("record-error-rate", metricGrpName, "The average per-second number of record sends that resulted in errors", metricTags); + this.errorSensor.add(m, new Rate()); + + this.maxRecordSizeSensor = metrics.sensor("record-size-max"); + m = new MetricName("record-size-max", metricGrpName, "The maximum record size", metricTags); + this.maxRecordSizeSensor.add(m, new Max()); + m = new MetricName("record-size-avg", metricGrpName, "The average record size", metricTags); + this.maxRecordSizeSensor.add(m, new Avg()); + + m = new MetricName("requests-in-flight", metricGrpName, "The current number of in-flight requests awaiting a response.", metricTags); + this.metrics.addMetric(m, new Measurable() { + public double measure(MetricConfig config, long now) { + return client.inFlightRequestCount(); + } + }); + m = new MetricName("metadata-age", metricGrpName, "The age in seconds of the current producer metadata being used.", metricTags); + metrics.addMetric(m, new Measurable() { + public double measure(MetricConfig config, long now) { + return (now - metadata.lastSuccessfulUpdate()) / 1000.0; + } + }); } - } - /** - * A set of outstanding request queues for each node that have not yet received responses - */ - private static final class InFlightRequests { - private final Map> requests = new HashMap>(); - - /** - * Add the given request to the queue for the node it was directed to - */ - public void add(InFlightRequest request) { - Deque reqs = this.requests.get(request.request.destination()); - if (reqs == null) { - reqs = new ArrayDeque(); - this.requests.put(request.request.destination(), reqs); + public void maybeRegisterTopicMetrics(String topic) { + // if one sensor of the metrics has been registered for the topic, + // then all other sensors should have been registered; and vice versa + String topicRecordsCountName = "topic." + topic + ".records-per-batch"; + Sensor topicRecordCount = this.metrics.getSensor(topicRecordsCountName); + if (topicRecordCount == null) { + Map metricTags = new LinkedHashMap(); + metricTags.put("client-id", clientId); + metricTags.put("topic", topic); + String metricGrpName = "producer-topic-metrics"; + + topicRecordCount = this.metrics.sensor(topicRecordsCountName); + MetricName m = new MetricName("record-send-rate", metricGrpName , metricTags); + topicRecordCount.add(m, new Rate()); + + String topicByteRateName = "topic." + topic + ".bytes"; + Sensor topicByteRate = this.metrics.sensor(topicByteRateName); + m = new MetricName("byte-rate", metricGrpName , metricTags); + topicByteRate.add(m, new Rate()); + + String topicCompressionRateName = "topic." + topic + ".compression-rate"; + Sensor topicCompressionRate = this.metrics.sensor(topicCompressionRateName); + m = new MetricName("compression-rate", metricGrpName , metricTags); + topicCompressionRate.add(m, new Avg()); + + String topicRetryName = "topic." + topic + ".record-retries"; + Sensor topicRetrySensor = this.metrics.sensor(topicRetryName); + m = new MetricName("record-retry-rate", metricGrpName , metricTags); + topicRetrySensor.add(m, new Rate()); + + String topicErrorName = "topic." + topic + ".record-errors"; + Sensor topicErrorSensor = this.metrics.sensor(topicErrorName); + m = new MetricName("record-error-rate", metricGrpName , metricTags); + topicErrorSensor.add(m, new Rate()); } - reqs.addFirst(request); } - public Deque requestQueue(int node) { - Deque reqs = requests.get(node); - if (reqs == null || reqs.isEmpty()) - throw new IllegalStateException("Response from server for which there are no in-flight requests."); - return reqs; + public void updateProduceRequestMetrics(Map> batches) { + long now = time.milliseconds(); + for (List nodeBatch : batches.values()) { + int records = 0; + for (RecordBatch batch : nodeBatch) { + // register all per-topic metrics at once + String topic = batch.topicPartition.topic(); + maybeRegisterTopicMetrics(topic); + + // per-topic record send rate + String topicRecordsCountName = "topic." + topic + ".records-per-batch"; + Sensor topicRecordCount = Utils.notNull(this.metrics.getSensor(topicRecordsCountName)); + topicRecordCount.record(batch.recordCount); + + // per-topic bytes send rate + String topicByteRateName = "topic." + topic + ".bytes"; + Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName)); + topicByteRate.record(batch.records.sizeInBytes()); + + // per-topic compression rate + String topicCompressionRateName = "topic." + topic + ".compression-rate"; + Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName)); + topicCompressionRate.record(batch.records.compressionRate()); + + // global metrics + this.batchSizeSensor.record(batch.records.sizeInBytes(), now); + this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now); + this.compressionRateSensor.record(batch.records.compressionRate()); + this.maxRecordSizeSensor.record(batch.maxRecordSize, now); + records += batch.recordCount; + } + this.recordsPerRequestSensor.record(records, now); + } } - /** - * Get the oldest request (the one that that will be completed next) for the given node - */ - public InFlightRequest nextCompleted(int node) { - return requestQueue(node).pollLast(); + public void recordRetries(String topic, int count) { + long now = time.milliseconds(); + this.retrySensor.record(count, now); + String topicRetryName = "topic." + topic + ".record-retries"; + Sensor topicRetrySensor = this.metrics.getSensor(topicRetryName); + if (topicRetrySensor != null) + topicRetrySensor.record(count, now); } - /** - * Can we send more requests to this node? - * - * @param node Node in question - * @return true iff we have no requests still being sent to the given node - */ - public boolean canSendMore(int node) { - Deque queue = requests.get(node); - return queue == null || queue.isEmpty() || queue.peekFirst().request.complete(); + public void recordErrors(String topic, int count) { + long now = time.milliseconds(); + this.errorSensor.record(count, now); + String topicErrorName = "topic." + topic + ".record-errors"; + Sensor topicErrorSensor = this.metrics.getSensor(topicErrorName); + if (topicErrorSensor != null) + topicErrorSensor.record(count, now); } - /** - * Clear out all the in-flight requests for the given node and return them - * - * @param node The node - * @return All the in-flight requests for that node that have been removed - */ - public Iterable clearAll(int node) { - Deque reqs = requests.get(node); - if (reqs == null) { - return Collections.emptyList(); - } else { - return requests.remove(node); + public void recordLatency(String node, long latency) { + long now = time.milliseconds(); + this.requestTimeSensor.record(latency, now); + if (!node.isEmpty()) { + String nodeTimeName = "node-" + node + ".latency"; + Sensor nodeRequestTime = this.metrics.getSensor(nodeTimeName); + if (nodeRequestTime != null) + nodeRequestTime.record(latency, now); } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java index 108d61e6dba6d..13f4d5958052a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -1,82 +1,219 @@ /** - * 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. + * 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 org.apache.kafka.clients.tools; import java.util.Arrays; import java.util.Properties; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.record.Records; - +import org.apache.kafka.clients.producer.*; public class ProducerPerformance { + private static final long NS_PER_MS = 1000000L; + private static final long NS_PER_SEC = 1000 * NS_PER_MS; + private static final long MIN_SLEEP_NS = 2 * NS_PER_MS; + public static void main(String[] args) throws Exception { - if (args.length != 3) { - System.err.println("USAGE: java " + ProducerPerformance.class.getName() + " url num_records record_size"); + if (args.length < 4) { + System.err.println("USAGE: java " + ProducerPerformance.class.getName() + + " topic_name num_records record_size target_records_sec [prop_name=prop_value]*"); System.exit(1); } - String url = args[0]; - int numRecords = Integer.parseInt(args[1]); + + /* parse args */ + String topicName = args[0]; + long numRecords = Long.parseLong(args[1]); int recordSize = Integer.parseInt(args[2]); + int throughput = Integer.parseInt(args[3]); + Properties props = new Properties(); - props.setProperty(ProducerConfig.REQUIRED_ACKS_CONFIG, "1"); - props.setProperty(ProducerConfig.BROKER_LIST_CONFIG, url); - props.setProperty(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG, Integer.toString(5 * 1000)); - props.setProperty(ProducerConfig.REQUEST_TIMEOUT_CONFIG, Integer.toString(Integer.MAX_VALUE)); - - KafkaProducer producer = new KafkaProducer(props); - Callback callback = new Callback() { - public void onCompletion(RecordMetadata metadata, Exception e) { - if (e != null) - e.printStackTrace(); - } - }; + for (int i = 4; i < args.length; i++) { + String[] pieces = args[i].split("="); + if (pieces.length != 2) + throw new IllegalArgumentException("Invalid property: " + args[i]); + props.put(pieces[0], pieces[1]); + } + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + KafkaProducer producer = new KafkaProducer(props); + + /* setup perf test */ byte[] payload = new byte[recordSize]; Arrays.fill(payload, (byte) 1); - ProducerRecord record = new ProducerRecord("test", payload); + ProducerRecord record = new ProducerRecord(topicName, payload); + long sleepTime = NS_PER_SEC / throughput; + long sleepDeficitNs = 0; + Stats stats = new Stats(numRecords, 5000); long start = System.currentTimeMillis(); - long maxLatency = -1L; - long totalLatency = 0; - int reportingInterval = 1000000; for (int i = 0; i < numRecords; i++) { long sendStart = System.currentTimeMillis(); - producer.send(record, callback); - long sendEllapsed = System.currentTimeMillis() - sendStart; - maxLatency = Math.max(maxLatency, sendEllapsed); - totalLatency += sendEllapsed; - if (i % reportingInterval == 0) { - System.out.printf("%d max latency = %d ms, avg latency = %.5f\n", - i, - maxLatency, - (totalLatency / (double) reportingInterval)); - totalLatency = 0L; - maxLatency = -1L; + Callback cb = stats.nextCompletion(sendStart, payload.length, stats); + producer.send(record, cb); + + /* + * Maybe sleep a little to control throughput. Sleep time can be a bit inaccurate for times < 1 ms so + * instead of sleeping each time instead wait until a minimum sleep time accumulates (the "sleep deficit") + * and then make up the whole deficit in one longer sleep. + */ + if (throughput > 0) { + float elapsed = (sendStart - start) / 1000.f; + if (elapsed > 0 && i / elapsed > throughput) { + sleepDeficitNs += sleepTime; + if (sleepDeficitNs >= MIN_SLEEP_NS) { + long sleepMs = sleepDeficitNs / 1000000; + long sleepNs = sleepDeficitNs - sleepMs * 1000000; + Thread.sleep(sleepMs, (int) sleepNs); + sleepDeficitNs = 0; + } + } } } - long ellapsed = System.currentTimeMillis() - start; - double msgsSec = 1000.0 * numRecords / (double) ellapsed; - double mbSec = msgsSec * (recordSize + Records.LOG_OVERHEAD) / (1024.0 * 1024.0); - System.out.printf("%d records sent in %d ms ms. %.2f records per second (%.2f mb/sec).", numRecords, ellapsed, msgsSec, mbSec); + + /* print final results */ producer.close(); + stats.printTotal(); + } + + private static class Stats { + private long start; + private long windowStart; + private int[] latencies; + private int sampling; + private int iteration; + private int index; + private long count; + private long bytes; + private int maxLatency; + private long totalLatency; + private long windowCount; + private int windowMaxLatency; + private long windowTotalLatency; + private long windowBytes; + private long reportingInterval; + + public Stats(long numRecords, int reportingInterval) { + this.start = System.currentTimeMillis(); + this.windowStart = System.currentTimeMillis(); + this.index = 0; + this.iteration = 0; + this.sampling = (int) (numRecords / Math.min(numRecords, 500000)); + this.latencies = new int[(int) (numRecords / this.sampling) + 1]; + this.index = 0; + this.maxLatency = 0; + this.totalLatency = 0; + this.windowCount = 0; + this.windowMaxLatency = 0; + this.windowTotalLatency = 0; + this.windowBytes = 0; + this.totalLatency = 0; + this.reportingInterval = reportingInterval; + } + + public void record(int iter, int latency, int bytes, long time) { + this.count++; + this.bytes += bytes; + this.totalLatency += latency; + this.maxLatency = Math.max(this.maxLatency, latency); + this.windowCount++; + this.windowBytes += bytes; + this.windowTotalLatency += latency; + this.windowMaxLatency = Math.max(windowMaxLatency, latency); + if (iter % this.sampling == 0) { + this.latencies[index] = latency; + this.index++; + } + /* maybe report the recent perf */ + if (time - windowStart >= reportingInterval) { + printWindow(); + newWindow(); + } + } + + public Callback nextCompletion(long start, int bytes, Stats stats) { + Callback cb = new PerfCallback(this.iteration, start, bytes, stats); + this.iteration++; + return cb; + } + + public void printWindow() { + long ellapsed = System.currentTimeMillis() - windowStart; + double recsPerSec = 1000.0 * windowCount / (double) ellapsed; + double mbPerSec = 1000.0 * this.windowBytes / (double) ellapsed / (1024.0 * 1024.0); + System.out.printf("%d records sent, %.1f records/sec (%.2f MB/sec), %.1f ms avg latency, %.1f max latency.\n", + windowCount, + recsPerSec, + mbPerSec, + windowTotalLatency / (double) windowCount, + (double) windowMaxLatency); + } + + public void newWindow() { + this.windowStart = System.currentTimeMillis(); + this.windowCount = 0; + this.windowMaxLatency = 0; + this.windowTotalLatency = 0; + this.windowBytes = 0; + } + + public void printTotal() { + long ellapsed = System.currentTimeMillis() - start; + double recsPerSec = 1000.0 * count / (double) ellapsed; + double mbPerSec = 1000.0 * this.bytes / (double) ellapsed / (1024.0 * 1024.0); + int[] percs = percentiles(this.latencies, index, 0.5, 0.95, 0.99, 0.999); + System.out.printf("%d records sent, %f records/sec (%.2f MB/sec), %.2f ms avg latency, %.2f ms max latency, %d ms 50th, %d ms 95th, %d ms 99th, %d ms 99.9th.\n", + count, + recsPerSec, + mbPerSec, + totalLatency / (double) count, + (double) maxLatency, + percs[0], + percs[1], + percs[2], + percs[3]); + } + + private static int[] percentiles(int[] latencies, int count, double... percentiles) { + int size = Math.min(count, latencies.length); + Arrays.sort(latencies, 0, size); + int[] values = new int[percentiles.length]; + for (int i = 0; i < percentiles.length; i++) { + int index = (int) (percentiles[i] * size); + values[i] = latencies[index]; + } + return values; + } + } + + private static final class PerfCallback implements Callback { + private final long start; + private final int iteration; + private final int bytes; + private final Stats stats; + + public PerfCallback(int iter, long start, int bytes, Stats stats) { + this.start = start; + this.stats = stats; + this.iteration = iter; + this.bytes = bytes; + } + + public void onCompletion(RecordMetadata metadata, Exception exception) { + long now = System.currentTimeMillis(); + int latency = (int) (now - start); + this.stats.record(iteration, latency, bytes, now); + if (exception != null) + exception.printStackTrace(); + } } } diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index c17a8f8162db9..60594a7dce901 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -1,42 +1,33 @@ /** - * 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. + * 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 org.apache.kafka.common; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; - import org.apache.kafka.common.utils.Utils; +import java.net.InetSocketAddress; +import java.util.*; /** * A representation of a subset of the nodes, topics, and partitions in the Kafka cluster. */ public final class Cluster { - private final AtomicInteger counter = new AtomicInteger(0); private final List nodes; private final Map partitionsByTopicPartition; private final Map> partitionsByTopic; + private final Map> availablePartitionsByTopic; + private final Map> partitionsByNode; + private final Map nodesById; /** * Create a new cluster with the given nodes and partitions @@ -48,24 +39,52 @@ public Cluster(Collection nodes, Collection partitions) { List copy = new ArrayList(nodes); Collections.shuffle(copy); this.nodes = Collections.unmodifiableList(copy); + + this.nodesById = new HashMap(); + for (Node node: nodes) + this.nodesById.put(node.id(), node); // index the partitions by topic/partition for quick lookup this.partitionsByTopicPartition = new HashMap(partitions.size()); for (PartitionInfo p : partitions) this.partitionsByTopicPartition.put(new TopicPartition(p.topic(), p.partition()), p); - // index the partitions by topic and make the lists unmodifiable so we can handle them out in - // user-facing apis without risk of the client modifying the contents - HashMap> parts = new HashMap>(); + // index the partitions by topic and node respectively, and make the lists + // unmodifiable so we can hand them out in user-facing apis without risk + // of the client modifying the contents + HashMap> partsForTopic = new HashMap>(); + HashMap> partsForNode = new HashMap>(); + for (Node n : this.nodes) { + partsForNode.put(n.id(), new ArrayList()); + } for (PartitionInfo p : partitions) { - if (!parts.containsKey(p.topic())) - parts.put(p.topic(), new ArrayList()); - List ps = parts.get(p.topic()); - ps.add(p); + if (!partsForTopic.containsKey(p.topic())) + partsForTopic.put(p.topic(), new ArrayList()); + List psTopic = partsForTopic.get(p.topic()); + psTopic.add(p); + + if (p.leader() != null) { + List psNode = Utils.notNull(partsForNode.get(p.leader().id())); + psNode.add(p); + } + } + this.partitionsByTopic = new HashMap>(partsForTopic.size()); + this.availablePartitionsByTopic = new HashMap>(partsForTopic.size()); + for (Map.Entry> entry : partsForTopic.entrySet()) { + String topic = entry.getKey(); + List partitionList = entry.getValue(); + this.partitionsByTopic.put(topic, Collections.unmodifiableList(partitionList)); + List availablePartitions = new ArrayList(); + for (PartitionInfo part : partitionList) { + if (part.leader() != null) + availablePartitions.add(part); + } + this.availablePartitionsByTopic.put(topic, Collections.unmodifiableList(availablePartitions)); } - this.partitionsByTopic = new HashMap>(parts.size()); - for (Map.Entry> entry : parts.entrySet()) - this.partitionsByTopic.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); + this.partitionsByNode = new HashMap>(partsForNode.size()); + for (Map.Entry> entry : partsForNode.entrySet()) + this.partitionsByNode.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); + } /** @@ -82,9 +101,9 @@ public static Cluster empty() { */ public static Cluster bootstrap(List addresses) { List nodes = new ArrayList(); - int nodeId = Integer.MIN_VALUE; + int nodeId = -1; for (InetSocketAddress address : addresses) - nodes.add(new Node(nodeId++, address.getHostName(), address.getPort())); + nodes.add(new Node(nodeId--, address.getHostName(), address.getPort())); return new Cluster(nodes, new ArrayList(0)); } @@ -94,6 +113,15 @@ public static Cluster bootstrap(List addresses) { public List nodes() { return this.nodes; } + + /** + * Get the node by the node id (or null if no such node exists) + * @param id The id of the node + * @return The node, or null if no such node exists + */ + public Node nodeById(int id) { + return this.nodesById.get(id); + } /** * Get the current leader for the given topic-partition @@ -122,19 +150,39 @@ public PartitionInfo partition(TopicPartition topicPartition) { * @param topic The topic name * @return A list of partitions */ - public List partitionsFor(String topic) { + public List partitionsForTopic(String topic) { return this.partitionsByTopic.get(topic); } /** - * Round-robin over the nodes in this cluster + * Get the list of available partitions for this topic + * @param topic The topic name + * @return A list of partitions + */ + public List availablePartitionsForTopic(String topic) { + return this.availablePartitionsByTopic.get(topic); + } + + /** + * Get the list of partitions whose leader is this node + * @param nodeId The node id + * @return A list of partitions */ - public Node nextNode() { - int size = nodes.size(); - if (size == 0) - throw new IllegalStateException("No known nodes."); - int idx = Utils.abs(counter.getAndIncrement()) % size; - return this.nodes.get(idx); + public List partitionsForNode(int nodeId) { + return this.partitionsByNode.get(nodeId); + } + + /** + * Get all topics. + * @return a set of all topics + */ + public Set topics() { + return this.partitionsByTopic.keySet(); + } + + @Override + public String toString() { + return "Cluster(nodes = " + this.nodes + ", partitions = " + this.partitionsByTopicPartition.values() + ")"; } } diff --git a/clients/src/main/java/org/apache/kafka/common/Metric.java b/clients/src/main/java/org/apache/kafka/common/Metric.java index b023e8e7c486a..d4ef77e36bbea 100644 --- a/clients/src/main/java/org/apache/kafka/common/Metric.java +++ b/clients/src/main/java/org/apache/kafka/common/Metric.java @@ -22,14 +22,9 @@ public interface Metric { /** - * A unique name for this metric + * A name for this metric */ - public String name(); - - /** - * A description of what is measured...this will be "" if no description was given - */ - public String description(); + public MetricName metricName(); /** * The value of the metric diff --git a/clients/src/main/java/org/apache/kafka/common/MetricName.java b/clients/src/main/java/org/apache/kafka/common/MetricName.java new file mode 100644 index 0000000000000..04b4a09badd51 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/MetricName.java @@ -0,0 +1,185 @@ +/** + * 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 org.apache.kafka.common; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.kafka.common.utils.Utils; + +/** + * The MetricName class encapsulates a metric's name, logical group and its related attributes + *

        + * This class captures the following parameters + *

        + *  name The name of the metric
        + *  group logical group name of the metrics to which this metric belongs.
        + *  description A human-readable description to include in the metric. This is optional.
        + *  tags additional key/value attributes of the metric. This is optional.
        + * 
        + * group, tags parameters can be used to create unique metric names while reporting in JMX or any custom reporting. + *

        + * Ex: standard JMX MBean can be constructed like domainName:type=group,key1=val1,key2=val2 + *

        + * Usage looks something like this: + *

        {@code
        + * // set up metrics:
        + * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
        + * Sensor sensor = metrics.sensor("message-sizes");
        + *
        + * Map metricTags = new LinkedHashMap();
        + * metricTags.put("client-id", "producer-1");
        + * metricTags.put("topic", "topic");
        + *
        + * MetricName metricName = new MetricName("message-size-avg", "producer-metrics", "average message size", metricTags);
        + * sensor.add(metricName, new Avg());
        + *
        + * metricName = new MetricName("message-size-max", "producer-metrics", metricTags);
        + * sensor.add(metricName, new Max());
        + *
        + * metricName = new MetricName("message-size-min", "producer-metrics", "message minimum size", "client-id", "my-client", "topic", "my-topic");
        + * sensor.add(metricName, new Min());
        + *
        + * // as messages are sent we record the sizes
        + * sensor.record(messageSize);
        + * }
        + */ +public final class MetricName { + + private final String name; + private final String group; + private final String description; + private Map tags; + private int hash = 0; + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param description A human-readable description to include in the metric + * @param tags additional key/value attributes of the metric + */ + public MetricName(String name, String group, String description, Map tags) { + this.name = Utils.notNull(name); + this.group = Utils.notNull(group); + this.description = Utils.notNull(description); + this.tags = Utils.notNull(tags); + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param description A human-readable description to include in the metric + * @param keyValue additional key/value attributes of the metric (must come in pairs) + */ + public MetricName(String name, String group, String description, String... keyValue) { + this(name, group, description, getTags(keyValue)); + } + + private static Map getTags(String... keyValue) { + if ((keyValue.length % 2) != 0) + throw new IllegalArgumentException("keyValue needs to be specified in paris"); + Map tags = new HashMap(); + + for (int i = 0; i < keyValue.length / 2; i++) + tags.put(keyValue[i], keyValue[i + 1]); + return tags; + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param tags key/value attributes of the metric + */ + public MetricName(String name, String group, Map tags) { + this(name, group, "", tags); + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param description A human-readable description to include in the metric + */ + public MetricName(String name, String group, String description) { + this(name, group, description, new HashMap()); + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + */ + public MetricName(String name, String group) { + this(name, group, "", new HashMap()); + } + + public String name() { + return this.name; + } + + public String group() { + return this.group; + } + + public Map tags() { + return this.tags; + } + + public String description() { + return this.description; + } + + @Override + public int hashCode() { + if (hash != 0) + return hash; + final int prime = 31; + int result = 1; + result = prime * result + ((group == null) ? 0 : group.hashCode()); + result = prime * result + ((name == null) ? 0 : name.hashCode()); + result = prime * result + ((tags == null) ? 0 : tags.hashCode()); + this.hash = result; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + MetricName other = (MetricName) obj; + if (group == null) { + if (other.group != null) + return false; + } else if (!group.equals(other.group)) + return false; + if (name == null) { + if (other.name != null) + return false; + } else if (!name.equals(other.name)) + return false; + if (tags == null) { + if (other.tags != null) + return false; + } else if (!tags.equals(other.tags)) + return false; + return true; + } + + @Override + public String toString() { + return "MetricName [name=" + name + ", group=" + group + ", description=" + + description + ", tags=" + tags + "]"; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/Node.java b/clients/src/main/java/org/apache/kafka/common/Node.java index 4197e5098c655..644cd71c8cb4f 100644 --- a/clients/src/main/java/org/apache/kafka/common/Node.java +++ b/clients/src/main/java/org/apache/kafka/common/Node.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common; @@ -22,16 +18,22 @@ public class Node { private final int id; + private final String idString; private final String host; private final int port; public Node(int id, String host, int port) { super(); this.id = id; + this.idString = Integer.toString(id); this.host = host; this.port = port; } + public static Node noNode() { + return new Node(-1, "", -1); + } + /** * The node id of this node */ @@ -39,6 +41,14 @@ public int id() { return id; } + /** + * String representation of the node id. + * Typically the integer id is used to serialize over the wire, the string representation is used as an identifier with NetworkClient code + */ + public String idString() { + return idString; + } + /** * The host name for this node */ diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java index 08d66f1a71fc5..321da8afc7394 100644 --- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java +++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common; @@ -71,4 +67,29 @@ public Node[] inSyncReplicas() { return inSyncReplicas; } + @Override + public String toString() { + return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s", + topic, + partition, + leader == null ? "none" : leader.id(), + fmtNodeIds(replicas), + fmtNodeIds(inSyncReplicas)); + } + + /* Extract the node ids from each item in the array and format for display */ + private String fmtNodeIds(Node[] nodes) { + StringBuilder b = new StringBuilder("["); + for (int i = 0; i < nodes.length - 1; i++) { + b.append(Integer.toString(nodes[i].id())); + b.append(','); + } + if (nodes.length > 0) { + b.append(Integer.toString(nodes[nodes.length - 1].id())); + b.append(','); + } + b.append("]"); + return b.toString(); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index c3148e5a9061d..bae528d315166 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -1,31 +1,24 @@ /** - * 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. + * 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 org.apache.kafka.common.config; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.Utils; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A convenient base class for configurations to extend. @@ -34,10 +27,17 @@ */ public class AbstractConfig { + private final Logger log = LoggerFactory.getLogger(getClass()); + + /* configs for which values have been requested, used to detect unused configs */ private final Set used; - private final Map values; + + /* the original values passed in by the user */ private final Map originals; + /* the parsed values */ + private final Map values; + @SuppressWarnings("unchecked") public AbstractConfig(ConfigDef definition, Map originals) { /* check that all the keys are really strings */ @@ -47,6 +47,7 @@ public AbstractConfig(ConfigDef definition, Map originals) { this.originals = (Map) originals; this.values = definition.parse(this.originals); this.used = Collections.synchronizedSet(new HashSet()); + logAll(); } protected Object get(String key) { @@ -56,7 +57,11 @@ protected Object get(String key) { return values.get(key); } - public int getInt(String key) { + public Short getShort(String key) { + return (Short) get(key); + } + + public Integer getInt(String key) { return (Integer) get(key); } @@ -64,6 +69,10 @@ public Long getLong(String key) { return (Long) get(key); } + public Double getDouble(String key) { + return (Double) get(key); + } + @SuppressWarnings("unchecked") public List getList(String key) { return (List) get(key); @@ -83,10 +92,39 @@ public Class getClass(String key) { public Set unused() { Set keys = new HashSet(originals.keySet()); - keys.remove(used); + keys.removeAll(used); return keys; } + public Map originals() { + Map copy = new HashMap(); + copy.putAll(originals); + return copy; + } + + private void logAll() { + StringBuilder b = new StringBuilder(); + b.append(getClass().getSimpleName()); + b.append(" values: "); + b.append(Utils.NL); + for (Map.Entry entry : this.values.entrySet()) { + b.append('\t'); + b.append(entry.getKey()); + b.append(" = "); + b.append(entry.getValue()); + b.append(Utils.NL); + } + log.info(b.toString()); + } + + /** + * Log warnings for any unused configurations + */ + public void logUnused() { + for (String key : unused()) + log.warn("The configuration {} = {} was supplied but isn't a known config.", key, this.values.get(key)); + } + /** * Get a configured instance of the give class specified by the given configuration key. If the object implements * Configurable configure it using the configuration. @@ -107,4 +145,26 @@ public T getConfiguredInstance(String key, Class t) { return t.cast(o); } + public List getConfiguredInstances(String key, Class t) { + List klasses = getList(key); + List objects = new ArrayList(); + for (String klass : klasses) { + Class c; + try { + c = Class.forName(klass); + } catch (ClassNotFoundException e) { + throw new ConfigException(key, klass, "Class " + klass + " could not be found."); + } + if (c == null) + return null; + Object o = Utils.newInstance(c); + if (!t.isInstance(o)) + throw new KafkaException(c.getName() + " is not an instance of " + t.getName()); + if (o instanceof Configurable) + ((Configurable) o).configure(this.originals); + objects.add(t.cast(o)); + } + return objects; + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 61257d1ac6ee3..4170bcc7def5b 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -1,126 +1,186 @@ /** - * 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 + * 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 + * 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. + * 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 org.apache.kafka.common.config; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; + +import org.apache.kafka.common.utils.Utils; /** * This class is used for specifying the set of expected configurations, their type, their defaults, their * documentation, and any special validation logic used for checking the correctness of the values the user provides. - *

        + *

        * Usage of this class looks something like this: - * + *

        *

          * ConfigDef defs = new ConfigDef();
          * defs.define("config_name", Type.STRING, "default string value", "This configuration is used for blah blah blah.");
          * defs.define("another_config_name", Type.INT, 42, Range.atLeast(0), "More documentation on this config");
        - * 
        + *
          * Properties props = new Properties();
          * props.setProperty("config_name", "some value");
          * Map<String, Object> configs = defs.parse(props);
        - * 
        + *
          * String someConfig = (String) configs.get("config_name"); // will return "some value"
          * int anotherConfig = (Integer) configs.get("another_config_name"); // will return default value of 42
          * 
        - * + *

        * This class can be used stand-alone or in combination with {@link AbstractConfig} which provides some additional * functionality for accessing configs. */ public class ConfigDef { - private static final Object NO_DEFAULT_VALUE = new Object(); + private static final Object NO_DEFAULT_VALUE = new String(""); private final Map configKeys = new HashMap(); + /** + * Returns unmodifiable set of properties names defined in this {@linkplain ConfigDef} + * + * @return new unmodifiable {@link Set} instance containing the keys + */ + public Set names() { + return Collections.unmodifiableSet(configKeys.keySet()); + } + /** * Define a new configuration - * @param name The name of the config parameter - * @param type The type of the config - * @param defaultValue The default value to use if this config isn't present - * @param validator A validator to use in checking the correctness of the config + * + * @param name The name of the config parameter + * @param type The type of the config + * @param defaultValue The default value to use if this config isn't present + * @param validator A validator to use in checking the correctness of the config + * @param importance The importance of this config: is this something you will likely need to change. * @param documentation The documentation string for the config + * @param required Should the config fail if given property is not set and doesn't have default value specified * @return This ConfigDef so you can chain calls */ - public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, String documentation) { + public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, + boolean required) { if (configKeys.containsKey(name)) throw new ConfigException("Configuration " + name + " is defined twice."); Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type); - configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, documentation)); + configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation, required)); return this; } + /** + * Define a new required configuration + * + * @param name The name of the config parameter + * @param type The type of the config + * @param defaultValue The default value to use if this config isn't present + * @param validator A validator to use in checking the correctness of the config + * @param importance The importance of this config: is this something you will likely need to change. + * @param documentation The documentation string for the config + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation) { + return define(name, type, defaultValue, validator, importance, documentation, true); + } + /** * Define a new configuration with no special validation logic - * @param name The name of the config parameter - * @param type The type of the config - * @param defaultValue The default value to use if this config isn't present + * + * @param name The name of the config parameter + * @param type The type of the config + * @param defaultValue The default value to use if this config isn't present + * @param importance The importance of this config: is this something you will likely need to change. * @param documentation The documentation string for the config * @return This ConfigDef so you can chain calls */ - public ConfigDef define(String name, Type type, Object defaultValue, String documentation) { - return define(name, type, defaultValue, null, documentation); + public ConfigDef define(String name, Type type, Object defaultValue, Importance importance, String documentation) { + return define(name, type, defaultValue, null, importance, documentation, true); } /** * Define a required parameter with no default value - * @param name The name of the config parameter - * @param type The type of the config - * @param validator A validator to use in checking the correctness of the config + * + * @param name The name of the config parameter + * @param type The type of the config + * @param validator A validator to use in checking the correctness of the config + * @param importance The importance of this config: is this something you will likely need to change. * @param documentation The documentation string for the config * @return This ConfigDef so you can chain calls */ - public ConfigDef define(String name, Type type, Validator validator, String documentation) { - return define(name, type, NO_DEFAULT_VALUE, validator, documentation); + public ConfigDef define(String name, Type type, Validator validator, Importance importance, String documentation) { + return define(name, type, NO_DEFAULT_VALUE, validator, importance, documentation, true); } /** * Define a required parameter with no default value and no special validation logic - * @param name The name of the config parameter - * @param type The type of the config + * + * @param name The name of the config parameter + * @param type The type of the config + * @param importance The importance of this config: is this something you will likely need to change. * @param documentation The documentation string for the config * @return This ConfigDef so you can chain calls */ - public ConfigDef define(String name, Type type, String documentation) { - return define(name, type, NO_DEFAULT_VALUE, null, documentation); + public ConfigDef define(String name, Type type, Importance importance, String documentation) { + return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, true); } + /** + * Define a required parameter with no default value and no special validation logic + * + * @param name The name of the config parameter + * @param type The type of the config + * @param importance The importance of this config: is this something you will likely need to change. + * @param documentation The documentation string for the config + * @param required Should the config fail if given property is not set and doesn't have default value specified + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Importance importance, String documentation, boolean required) { + return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, required); + } + + /** * Parse and validate configs against this configuration definition. The input is a map of configs. It is expected * that the keys of the map are strings, but the values can either be strings or they may already be of the * appropriate type (int, string, etc). This will work equally well with either java.util.Properties instances or a * programmatically constructed map. + * * @param props The configs to parse and validate * @return Parsed and validated configs. The key will be the config name and the value will be the value parsed into - * the appropriate type (int, string, etc) + * the appropriate type (int, string, etc) */ public Map parse(Map props) { /* parse all known keys */ Map values = new HashMap(); for (ConfigKey key : configKeys.values()) { Object value; + // props map contains setting - assign ConfigKey value if (props.containsKey(key.name)) value = parseType(key.name, props.get(key.name), key.type); - else if (key.defaultValue == NO_DEFAULT_VALUE) + // props map doesn't contain setting, the key is required and no default value specified - it's an error + else if (key.defaultValue == NO_DEFAULT_VALUE && key.required) throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value."); + // props map doesn't contain setting, no default value specified and the key is not required - assign it to null + else if (!key.hasDefault() && !key.required) + value = null; + // otherwise assign setting it's default value else value = key.defaultValue; + if (key.validator != null) + key.validator.ensureValid(key.name, value); values.put(key.name, value); } return values; @@ -128,9 +188,10 @@ else if (key.defaultValue == NO_DEFAULT_VALUE) /** * Parse a value according to its expected type. - * @param name The config name + * + * @param name The config name * @param value The config value - * @param type The expected type + * @param type The expected type * @return The parsed object */ private Object parseType(String name, Object value, Type type) { @@ -140,9 +201,14 @@ private Object parseType(String name, Object value, Type type) { trimmed = ((String) value).trim(); switch (type) { case BOOLEAN: - if (value instanceof String) - return Boolean.parseBoolean(trimmed); - else if (value instanceof Boolean) + if (value instanceof String) { + if (trimmed.equalsIgnoreCase("true")) + return true; + else if (trimmed.equalsIgnoreCase("false")) + return false; + else + throw new ConfigException(name, value, "Expected value to be either true or false"); + } else if (value instanceof Boolean) return value; else throw new ConfigException(name, value, "Expected value to be either true or false"); @@ -159,6 +225,14 @@ else if (value instanceof Boolean) } else { throw new ConfigException(name, value, "Expected value to be an number."); } + case SHORT: + if (value instanceof Short) { + return (Short) value; + } else if (value instanceof String) { + return Short.parseShort(trimmed); + } else { + throw new ConfigException(name, value, "Expected value to be an number."); + } case LONG: if (value instanceof Integer) return ((Integer) value).longValue(); @@ -179,9 +253,12 @@ else if (value instanceof String) if (value instanceof List) return (List) value; else if (value instanceof String) - return Arrays.asList(trimmed.split("\\s*,\\s*", -1)); + if (trimmed.isEmpty()) + return Collections.emptyList(); + else + return Arrays.asList(trimmed.split("\\s*,\\s*", -1)); else - throw new ConfigException(name, value, "Expected a comma seperated list."); + throw new ConfigException(name, value, "Expected a comma separated list."); case CLASS: if (value instanceof Class) return (Class) value; @@ -203,7 +280,11 @@ else if (value instanceof String) * The config types */ public enum Type { - BOOLEAN, STRING, INT, LONG, DOUBLE, LIST, CLASS; + BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS; + } + + public enum Importance { + HIGH, MEDIUM, LOW } /** @@ -227,10 +308,11 @@ private Range(Number min, Number max) { /** * A numeric range that checks only the lower bound + * * @param min The minimum acceptable value */ public static Range atLeast(Number min) { - return new Range(min, Double.MAX_VALUE); + return new Range(min, null); } /** @@ -242,8 +324,44 @@ public static Range between(Number min, Number max) { public void ensureValid(String name, Object o) { Number n = (Number) o; - if (n.doubleValue() < min.doubleValue() || n.doubleValue() > max.doubleValue()) - throw new ConfigException(name, o, "Value must be in the range [" + min + ", " + max + "]"); + if (min != null && n.doubleValue() < min.doubleValue()) + throw new ConfigException(name, o, "Value must be at least " + min); + if (max != null && n.doubleValue() > max.doubleValue()) + throw new ConfigException(name, o, "Value must be no more than " + max); + } + + public String toString() { + if (min == null) + return "[...," + max + "]"; + else if (max == null) + return "[" + min + ",...]"; + else + return "[" + min + ",...," + max + "]"; + } + } + + public static class ValidString implements Validator { + List validStrings; + + private ValidString(List validStrings) { + this.validStrings = validStrings; + } + + public static ValidString in(String... validStrings) { + return new ValidString(Arrays.asList(validStrings)); + } + + @Override + public void ensureValid(String name, Object o) { + String s = (String) o; + if (!validStrings.contains(s)) { + throw new ConfigException(name, o, "String must be one of: " + Utils.join(validStrings, ", ")); + } + + } + + public String toString() { + return "[" + Utils.join(validStrings, ", ") + "]"; } } @@ -253,17 +371,77 @@ private static class ConfigKey { public final String documentation; public final Object defaultValue; public final Validator validator; + public final Importance importance; + public final boolean required; - public ConfigKey(String name, Type type, Object defaultValue, Validator validator, String documentation) { + public ConfigKey(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, boolean required) { super(); this.name = name; this.type = type; this.defaultValue = defaultValue; this.validator = validator; + this.importance = importance; if (this.validator != null) this.validator.ensureValid(name, defaultValue); this.documentation = documentation; + this.required = required; + } + + public boolean hasDefault() { + return this.defaultValue != NO_DEFAULT_VALUE; } } -} + + public String toHtmlTable() { + // sort first required fields, then by importance, then name + List configs = new ArrayList(this.configKeys.values()); + Collections.sort(configs, new Comparator() { + public int compare(ConfigDef.ConfigKey k1, ConfigDef.ConfigKey k2) { + // first take anything with no default value + if (!k1.hasDefault() && k2.hasDefault()) + return -1; + else if (!k2.hasDefault() && k1.hasDefault()) + return 1; + + // then sort by importance + int cmp = k1.importance.compareTo(k2.importance); + if (cmp == 0) + // then sort in alphabetical order + return k1.name.compareTo(k2.name); + else + return cmp; + } + }); + StringBuilder b = new StringBuilder(); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + for (ConfigKey def : configs) { + b.append("\n"); + b.append(""); + b.append(""); + b.append(""); + b.append(""); + b.append(""); + b.append("\n"); + } + b.append("
        NameTypeDefaultImportanceDescription
        "); + b.append(def.name); + b.append(""); + b.append(def.type.toString().toLowerCase()); + b.append(""); + b.append(def.defaultValue == null ? "" : def.defaultValue); + b.append(""); + b.append(def.importance.toString().toLowerCase()); + b.append(""); + b.append(def.documentation); + b.append("
        "); + return b.toString(); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java b/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java index 7c948b166a8ac..a566b9006a63c 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java @@ -22,7 +22,7 @@ * Any API exception that is part of the public protocol and should be a subclass of this class and be part of this * package. */ -public abstract class ApiException extends KafkaException { +public class ApiException extends KafkaException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java new file mode 100644 index 0000000000000..ba9ce82bf74f3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java @@ -0,0 +1,40 @@ +/** + * 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 org.apache.kafka.common.errors; + +/** + * The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has + * not yet been created. + */ +public class ConsumerCoordinatorNotAvailableException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public ConsumerCoordinatorNotAvailableException() { + super(); + } + + public ConsumerCoordinatorNotAvailableException(String message) { + super(message); + } + + public ConsumerCoordinatorNotAvailableException(String message, Throwable cause) { + super(message, cause); + } + + public ConsumerCoordinatorNotAvailableException(Throwable cause) { + super(cause); + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java b/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java index 673f61d6271c5..eaccf276dbfb3 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java @@ -1,22 +1,22 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * 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 org.apache.kafka.common.errors; -public class CorruptRecordException extends ApiException { +/** + * This exception indicates a record has failed it's internal CRC check, this generally indicates network or disk + * corruption. + */ +public class CorruptRecordException extends RetriableException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java b/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java new file mode 100644 index 0000000000000..18d61a2e55fba --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java @@ -0,0 +1,39 @@ +/** + * 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 org.apache.kafka.common.errors; + + +/** + * Server disconnected before a request could be completed. + */ +public class DisconnectException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public DisconnectException() { + super(); + } + + public DisconnectException(String message, Throwable cause) { + super(message, cause); + } + + public DisconnectException(String message) { + super(message); + } + + public DisconnectException(Throwable cause) { + super(cause); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java new file mode 100644 index 0000000000000..d20b74a46fecf --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java @@ -0,0 +1,33 @@ +/** + * 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 org.apache.kafka.common.errors; + +public class IllegalGenerationException extends RetriableException { + private static final long serialVersionUID = 1L; + + public IllegalGenerationException() { + super(); + } + + public IllegalGenerationException(String message, Throwable cause) { + super(message, cause); + } + + public IllegalGenerationException(String message) { + super(message); + } + + public IllegalGenerationException(Throwable cause) { + super(cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java b/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java new file mode 100644 index 0000000000000..3680f1b658ba5 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java @@ -0,0 +1,39 @@ +/** + * 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 org.apache.kafka.common.errors; + +import org.apache.kafka.common.KafkaException; + +/** + * An unchecked wrapper for InterruptedException + */ +public class InterruptException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public InterruptException(InterruptedException cause) { + super(cause); + Thread.currentThread().interrupt(); + } + + public InterruptException(String message, InterruptedException cause) { + super(message, cause); + Thread.currentThread().interrupt(); + } + + public InterruptException(String message) { + super(message, new InterruptedException()); + Thread.currentThread().interrupt(); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java new file mode 100644 index 0000000000000..8841badb2d783 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java @@ -0,0 +1,39 @@ +/** + * 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 org.apache.kafka.common.errors; + +/** + * An exception that may indicate the client's metadata is out of date + */ +public abstract class InvalidMetadataException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public InvalidMetadataException() { + super(); + } + + public InvalidMetadataException(String message) { + super(message); + } + + public InvalidMetadataException(String message, Throwable cause) { + super(message, cause); + } + + public InvalidMetadataException(Throwable cause) { + super(cause); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java new file mode 100644 index 0000000000000..9d19b2844aec7 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +public class InvalidRequiredAcksException extends ApiException { + private static final long serialVersionUID = 1L; + + public InvalidRequiredAcksException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java new file mode 100644 index 0000000000000..1d90b592d1641 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java @@ -0,0 +1,38 @@ +/** + * 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 org.apache.kafka.common.errors; + +/** + * The client has attempted to perform an operation on an invalid topic. + */ +public class InvalidTopicException extends ApiException { + + private static final long serialVersionUID = 1L; + + public InvalidTopicException() { + super(); + } + + public InvalidTopicException(String message, Throwable cause) { + super(message, cause); + } + + public InvalidTopicException(String message) { + super(message); + } + + public InvalidTopicException(Throwable cause) { + super(cause); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java index 0bde6b5a351fc..9d7ebd47a8439 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java @@ -1,35 +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. + * 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 org.apache.kafka.common.errors; -public class LeaderNotAvailableException extends RetryableException { +/** + * There is no currently available leader for the given partition (either because a leadership election is in progress + * or because all replicas are down). + */ +public class LeaderNotAvailableException extends InvalidMetadataException { private static final long serialVersionUID = 1L; - public LeaderNotAvailableException(String message, Throwable cause) { - super(message, cause); - } - public LeaderNotAvailableException(String message) { super(message); } - public LeaderNotAvailableException(Throwable cause) { - super(cause); - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java b/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java index 3a041593d76cf..f0baa983f081e 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java @@ -1,22 +1,22 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * 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 org.apache.kafka.common.errors; -public class NetworkException extends ApiException { +/** + * A misc. network-related IOException occurred when making a request. This could be because the client's metadata is + * out of date and it is making a request to a node that is now dead. + */ +public class NetworkException extends InvalidMetadataException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java new file mode 100644 index 0000000000000..b6c83b4c243d9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java @@ -0,0 +1,40 @@ +/** + * 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 org.apache.kafka.common.errors; + +/** + * The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is + * not a coordinator for. + */ +public class NotCoordinatorForConsumerException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public NotCoordinatorForConsumerException() { + super(); + } + + public NotCoordinatorForConsumerException(String message) { + super(message); + } + + public NotCoordinatorForConsumerException(String message, Throwable cause) { + super(message, cause); + } + + public NotCoordinatorForConsumerException(Throwable cause) { + super(cause); + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java new file mode 100644 index 0000000000000..fd7f6d8f5c69b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java @@ -0,0 +1,30 @@ +/** + * 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 org.apache.kafka.common.errors; + +/** + * Number of insync replicas for the partition is lower than min.insync.replicas This exception is raised when the low + * ISR size is discovered *after* the message was already appended to the log. Producer retries will cause duplicates. + */ +public class NotEnoughReplicasAfterAppendException extends RetriableException { + private static final long serialVersionUID = 1L; + + public NotEnoughReplicasAfterAppendException(String message) { + super(message); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java similarity index 56% rename from clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java rename to clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java index c7f2f222f712a..1573227acef53 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java @@ -6,7 +6,7 @@ * (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 + * 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, @@ -17,31 +17,24 @@ package org.apache.kafka.common.errors; /** - * A retryable exception is an exception that is safe to retry. To be retryable an exception should be - *

          - *
        1. Transient, there is no point retrying a error due to a non-existant topic or message too large - *
        2. Idempotent, the exception is known to not change any state on the server - *
        - * A client may choose to retry any request they like, but exceptions extending this class are always safe and sane to - * retry. + * Number of insync replicas for the partition is lower than min.insync.replicas */ -public abstract class RetryableException extends ApiException { - +public class NotEnoughReplicasException extends RetriableException { private static final long serialVersionUID = 1L; - public RetryableException(String message, Throwable cause) { + public NotEnoughReplicasException() { + super(); + } + + public NotEnoughReplicasException(String message, Throwable cause) { super(message, cause); } - public RetryableException(String message) { + public NotEnoughReplicasException(String message) { super(message); } - public RetryableException(Throwable cause) { + public NotEnoughReplicasException(Throwable cause) { super(cause); } - - public RetryableException() { - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java index 5adc72ccf2d0c..ad9c77c41c492 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java @@ -1,22 +1,21 @@ /** - * 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. + * 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 org.apache.kafka.common.errors; -public class NotLeaderForPartitionException extends RetryableException { +/** + * This server is not the leader for the given partition + */ +public class NotLeaderForPartitionException extends InvalidMetadataException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java new file mode 100644 index 0000000000000..016506e9d2463 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java @@ -0,0 +1,40 @@ +/** + * 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 org.apache.kafka.common.errors; + +/** + * The broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change + * for that offsets topic partition). + */ +public class OffsetLoadInProgressException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public OffsetLoadInProgressException() { + super(); + } + + public OffsetLoadInProgressException(String message) { + super(message); + } + + public OffsetLoadInProgressException(String message, Throwable cause) { + super(message, cause); + } + + public OffsetLoadInProgressException(Throwable cause) { + super(cause); + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java index a3159bb1034e7..0be2f500685b0 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java @@ -1,21 +1,20 @@ /** - * 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. + * 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 org.apache.kafka.common.errors; +/** + * The client has tried to save its offset with associated metadata larger than the maximum size allowed by the server. + */ public class OffsetMetadataTooLarge extends ApiException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java index d01698a3efca7..fc7c6e3471b05 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java @@ -1,22 +1,22 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * 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 org.apache.kafka.common.errors; -public class OffsetOutOfRangeException extends ApiException { +/** + * This offset is either larger or smaller than the range of offsets the server has for the given partition. + * + */ +public class OffsetOutOfRangeException extends RetriableException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java b/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java new file mode 100644 index 0000000000000..f3f3f27d34dfb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java @@ -0,0 +1,39 @@ +/** + * 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 org.apache.kafka.common.errors; + +/** + * This record batch is larger than the maximum allowable size + */ +public class RecordBatchTooLargeException extends ApiException { + + private static final long serialVersionUID = 1L; + + public RecordBatchTooLargeException() { + super(); + } + + public RecordBatchTooLargeException(String message, Throwable cause) { + super(message, cause); + } + + public RecordBatchTooLargeException(String message) { + super(message); + } + + public RecordBatchTooLargeException(Throwable cause) { + super(cause); + } + +} + diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java b/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java index ce95ca04aa842..737b7f07b16a0 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java @@ -1,21 +1,20 @@ /** - * 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. + * 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 org.apache.kafka.common.errors; +/** + * This record is larger than the maximum allowable size + */ public class RecordTooLargeException extends ApiException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java b/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java new file mode 100644 index 0000000000000..6c639a972d7e4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java @@ -0,0 +1,37 @@ +/** + * 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 org.apache.kafka.common.errors; + +/** + * A retryable exception is a transient exception that if retried may succeed. + */ +public abstract class RetriableException extends ApiException { + + private static final long serialVersionUID = 1L; + + public RetriableException(String message, Throwable cause) { + super(message, cause); + } + + public RetriableException(String message) { + super(message); + } + + public RetriableException(Throwable cause) { + super(cause); + } + + public RetriableException() { + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java b/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java new file mode 100644 index 0000000000000..00388d12794f0 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java @@ -0,0 +1,46 @@ +/** + * 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 org.apache.kafka.common.errors; + +import org.apache.kafka.common.KafkaException; + +/** + * Any exception during serialization in the producer + */ +public class SerializationException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public SerializationException(String message, Throwable cause) { + super(message, cause); + } + + public SerializationException(String message) { + super(message); + } + + public SerializationException(Throwable cause) { + super(cause); + } + + public SerializationException() { + super(); + } + + /* avoid the expensive and useless stack trace for serialization exceptions */ + @Override + public Throwable fillInStackTrace() { + return this; + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java b/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java index dffd64d19c35a..c7f569ca87369 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java @@ -1,22 +1,21 @@ /** - * 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. + * 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 org.apache.kafka.common.errors; -public class TimeoutException extends ApiException { +/** + * Indicates that a request timed out. + */ +public class TimeoutException extends RetriableException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java new file mode 100644 index 0000000000000..9bcbd114e74e8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java @@ -0,0 +1,33 @@ +/** + * 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 org.apache.kafka.common.errors; + +public class UnknownConsumerIdException extends RetriableException { + private static final long serialVersionUID = 1L; + + public UnknownConsumerIdException() { + super(); + } + + public UnknownConsumerIdException(String message, Throwable cause) { + super(message, cause); + } + + public UnknownConsumerIdException(String message) { + super(message); + } + + public UnknownConsumerIdException(Throwable cause) { + super(cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java index a0690fe2870bf..963ef081db51d 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java @@ -1,21 +1,22 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * 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 org.apache.kafka.common.errors; +/** + * An error occurred on the server for which the client doesn't have a corresponding error code. This is generally an + * unexpected error. + * + */ public class UnknownServerException extends ApiException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java index 73d1953cbe045..70fd55c8788be 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java @@ -1,22 +1,21 @@ /** - * 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. + * 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 org.apache.kafka.common.errors; -public class UnknownTopicOrPartitionException extends ApiException { +/** + * This topic/partition doesn't exist + */ +public class UnknownTopicOrPartitionException extends InvalidMetadataException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java index 29185a6a90d00..e0969aad858e4 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.metrics; +import org.apache.kafka.common.MetricName; + import java.util.List; /** @@ -28,25 +30,19 @@ public interface CompoundStat extends Stat { public static class NamedMeasurable { - private final String name; - private final String description; + private final MetricName name; private final Measurable stat; - public NamedMeasurable(String name, String description, Measurable stat) { + public NamedMeasurable(MetricName name, Measurable stat) { super(); this.name = name; - this.description = description; this.stat = stat; } - public String name() { + public MetricName name() { return name; } - public String description() { - return description; - } - public Measurable stat() { return stat; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java index e08c349a6cd76..6b9590c418aed 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics; @@ -36,14 +32,18 @@ import javax.management.ReflectionException; import org.apache.kafka.common.KafkaException; - +import org.apache.kafka.common.MetricName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Register metrics in JMX as dynamic mbeans based on the metric names */ public class JmxReporter implements MetricsReporter { - private final String prefix; + private static final Logger log = LoggerFactory.getLogger(JmxReporter.class); + private static final Object LOCK = new Object(); + private String prefix; private final Map mbeans = new HashMap(); public JmxReporter() { @@ -58,38 +58,65 @@ public JmxReporter(String prefix) { } @Override - public synchronized void init(List metrics) { - for (KafkaMetric metric : metrics) - addAttribute(metric); - for (KafkaMbean mbean : mbeans.values()) - reregister(mbean); + public void configure(Map configs) {} + @Override + public void init(List metrics) { + synchronized (LOCK) { + for (KafkaMetric metric : metrics) + addAttribute(metric); + for (KafkaMbean mbean : mbeans.values()) + reregister(mbean); + } } @Override - public synchronized void metricChange(KafkaMetric metric) { - KafkaMbean mbean = addAttribute(metric); - reregister(mbean); + public void metricChange(KafkaMetric metric) { + synchronized (LOCK) { + KafkaMbean mbean = addAttribute(metric); + reregister(mbean); + } } private KafkaMbean addAttribute(KafkaMetric metric) { try { - String[] names = split(prefix + metric.name()); - String qualifiedName = names[0] + "." + names[1]; - if (!this.mbeans.containsKey(qualifiedName)) - mbeans.put(qualifiedName, new KafkaMbean(names[0], names[1])); - KafkaMbean mbean = this.mbeans.get(qualifiedName); - mbean.setAttribute(names[2], metric); + MetricName metricName = metric.metricName(); + String mBeanName = getMBeanName(metricName); + if (!this.mbeans.containsKey(mBeanName)) + mbeans.put(mBeanName, new KafkaMbean(mBeanName)); + KafkaMbean mbean = this.mbeans.get(mBeanName); + mbean.setAttribute(metricName.name(), metric); return mbean; } catch (JMException e) { - throw new KafkaException("Error creating mbean attribute " + metric.name(), e); + throw new KafkaException("Error creating mbean attribute for metricName :" + metric.metricName(), e); } } - public synchronized void close() { - for (KafkaMbean mbean : this.mbeans.values()) - unregister(mbean); + /** + * @param metricName + * @return standard JMX MBean name in the following format domainName:type=metricType,key1=val1,key2=val2 + */ + private String getMBeanName(MetricName metricName) { + StringBuilder mBeanName = new StringBuilder(); + mBeanName.append(prefix); + mBeanName.append(":type="); + mBeanName.append(metricName.group()); + for (Map.Entry entry : metricName.tags().entrySet()) { + if (entry.getKey().length() <= 0 || entry.getValue().length() <= 0) + continue; + mBeanName.append(","); + mBeanName.append(entry.getKey()); + mBeanName.append("="); + mBeanName.append(entry.getValue()); + } + return mBeanName.toString(); + } + public void close() { + synchronized (LOCK) { + for (KafkaMbean mbean : this.mbeans.values()) + unregister(mbean); + } } private void unregister(KafkaMbean mbean) { @@ -111,29 +138,13 @@ private void reregister(KafkaMbean mbean) { } } - private String[] split(String name) { - int attributeStart = name.lastIndexOf('.'); - if (attributeStart < 0) - throw new IllegalArgumentException("No MBean name in metric name: " + name); - String attributeName = name.substring(attributeStart + 1, name.length()); - String remainder = name.substring(0, attributeStart); - int beanStart = remainder.lastIndexOf('.'); - if (beanStart < 0) - return new String[] { "", remainder, attributeName }; - String packageName = remainder.substring(0, beanStart); - String beanName = remainder.substring(beanStart + 1, remainder.length()); - return new String[] { packageName, beanName, attributeName }; - } - private static class KafkaMbean implements DynamicMBean { - private final String beanName; private final ObjectName objectName; private final Map metrics; - public KafkaMbean(String packageName, String beanName) throws MalformedObjectNameException { - this.beanName = beanName; + public KafkaMbean(String mbeanName) throws MalformedObjectNameException { this.metrics = new HashMap(); - this.objectName = new ObjectName(packageName + ":type=" + beanName); + this.objectName = new ObjectName(mbeanName); } public ObjectName name() { @@ -160,7 +171,7 @@ public AttributeList getAttributes(String[] names) { list.add(new Attribute(name, getAttribute(name))); return list; } catch (Exception e) { - e.printStackTrace(); + log.error("Error getting JMX attribute: ", e); return new AttributeList(); } } @@ -172,10 +183,15 @@ public MBeanInfo getMBeanInfo() { for (Map.Entry entry : this.metrics.entrySet()) { String attribute = entry.getKey(); KafkaMetric metric = entry.getValue(); - attrs[i] = new MBeanAttributeInfo(attribute, double.class.getName(), metric.description(), true, false, false); + attrs[i] = new MBeanAttributeInfo(attribute, + double.class.getName(), + metric.metricName().description(), + true, + false, + false); i += 1; } - return new MBeanInfo(beanName, "", attrs, null, null, null); + return new MBeanInfo(this.getClass().getName(), "", attrs, null, null, null); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java index b2426ac927468..89df1a4ec3e68 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java @@ -17,21 +17,20 @@ package org.apache.kafka.common.metrics; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.utils.Time; public final class KafkaMetric implements Metric { - private final String name; - private final String description; + private MetricName metricName; private final Object lock; private final Time time; private final Measurable measurable; private MetricConfig config; - KafkaMetric(Object lock, String name, String description, Measurable measurable, MetricConfig config, Time time) { + KafkaMetric(Object lock, MetricName metricName, Measurable measurable, MetricConfig config, Time time) { super(); - this.name = name; - this.description = description; + this.metricName = metricName; this.lock = lock; this.measurable = measurable; this.config = config; @@ -43,24 +42,19 @@ MetricConfig config() { } @Override - public String name() { - return this.name; - } - - @Override - public String description() { - return this.description; + public MetricName metricName() { + return this.metricName; } @Override public double value() { synchronized (this.lock) { - return value(time.nanoseconds()); + return value(time.milliseconds()); } } - double value(long time) { - return this.measurable.measure(config, time); + double value(long timeMs) { + return this.measurable.measure(config, timeMs); } public void config(MetricConfig config) { diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java b/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java index 0f405c3f81314..79f61bca4c3fa 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics; @@ -24,7 +20,7 @@ public interface Measurable { /** * Measure this quantity and return the result as a double * @param config The configuration for this metric - * @param now The time the measurement is being taken + * @param now The POSIX time in milliseconds the measurement is being taken * @return The measured value */ public double measure(MetricConfig config, long now); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java index 4d14fbcc68a80..dfa1b0a11042a 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java @@ -26,7 +26,7 @@ public class MetricConfig { private Quota quota; private int samples; private long eventWindow; - private long timeWindowNs; + private long timeWindowMs; private TimeUnit unit; public MetricConfig() { @@ -34,7 +34,7 @@ public MetricConfig() { this.quota = null; this.samples = 2; this.eventWindow = Long.MAX_VALUE; - this.timeWindowNs = TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS); + this.timeWindowMs = TimeUnit.MILLISECONDS.convert(30, TimeUnit.SECONDS); this.unit = TimeUnit.SECONDS; } @@ -56,12 +56,12 @@ public MetricConfig eventWindow(long window) { return this; } - public long timeWindowNs() { - return timeWindowNs; + public long timeWindowMs() { + return timeWindowMs; } public MetricConfig timeWindow(long window, TimeUnit unit) { - this.timeWindowNs = TimeUnit.NANOSECONDS.convert(window, unit); + this.timeWindowMs = TimeUnit.MILLISECONDS.convert(window, unit); return this; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java index 6db2dfbe94c94..5f6caf957e3bd 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java @@ -1,32 +1,29 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics; +import java.io.Closeable; import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.utils.CopyOnWriteMap; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; - /** * A registry of sensors and metrics. *

        @@ -41,17 +38,19 @@ * // set up metrics: * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors * Sensor sensor = metrics.sensor("message-sizes"); - * sensor.add("kafka.producer.message-sizes.avg", new Avg()); - * sensor.add("kafka.producer.message-sizes.max", new Max()); + * MetricName metricName = new MetricName("message-size-avg", "producer-metrics"); + * sensor.add(metricName, new Avg()); + * metricName = new MetricName("message-size-max", "producer-metrics"); + * sensor.add(metricName, new Max()); * * // as messages are sent we record the sizes * sensor.record(messageSize); * */ -public class Metrics { +public class Metrics implements Closeable { private final MetricConfig config; - private final ConcurrentMap metrics; + private final ConcurrentMap metrics; private final ConcurrentMap sensors; private final List reporters; private final Time time; @@ -67,7 +66,7 @@ public Metrics() { * Create a metrics repository with no metric reporters and default configuration. */ public Metrics(Time time) { - this(new MetricConfig(), new ArrayList(), time); + this(new MetricConfig(), new ArrayList(0), time); } /** @@ -87,8 +86,8 @@ public Metrics(MetricConfig defaultConfig) { */ public Metrics(MetricConfig defaultConfig, List reporters, Time time) { this.config = defaultConfig; - this.sensors = new ConcurrentHashMap(); - this.metrics = new ConcurrentHashMap(); + this.sensors = new CopyOnWriteMap(); + this.metrics = new CopyOnWriteMap(); this.reporters = Utils.notNull(reporters); this.time = time; for (MetricsReporter reporter : reporters) @@ -96,8 +95,26 @@ public Metrics(MetricConfig defaultConfig, List reporters, Time } /** - * Create a sensor with the given unique name and zero or more parent sensors. All parent sensors will receive every - * value recorded with this sensor. + * Get the sensor with the given name if it exists + * @param name The name of the sensor + * @return Return the sensor or null if no such sensor exists + */ + public Sensor getSensor(String name) { + return this.sensors.get(Utils.notNull(name)); + } + + /** + * Get or create a sensor with the given unique name and no parent sensors. + * @param name The sensor name + * @return The sensor + */ + public Sensor sensor(String name) { + return sensor(name, null, (Sensor[]) null); + } + + /** + * Get or create a sensor with the given unique name and zero or more parent sensors. All parent sensors will + * receive every value recorded with this sensor. * @param name The name of the sensor * @param parents The parent sensors * @return The sensor that is created @@ -107,15 +124,15 @@ public Sensor sensor(String name, Sensor... parents) { } /** - * Create a sensor with the given unique name and zero or more parent sensors. All parent sensors will receive every - * value recorded with this sensor. + * Get or create a sensor with the given unique name and zero or more parent sensors. All parent sensors will + * receive every value recorded with this sensor. * @param name The name of the sensor * @param config A default configuration to use for this sensor for metrics that don't have their own config * @param parents The parent sensors * @return The sensor that is created */ public synchronized Sensor sensor(String name, MetricConfig config, Sensor... parents) { - Sensor s = this.sensors.get(Utils.notNull(name)); + Sensor s = getSensor(name); if (s == null) { s = new Sensor(this, name, parents, config == null ? this.config : config, time); this.sensors.put(name, s); @@ -126,47 +143,23 @@ public synchronized Sensor sensor(String name, MetricConfig config, Sensor... pa /** * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor. * This is a way to expose existing values as metrics. - * @param name The name of the metric - * @param measurable The measurable that will be measured by this metric - */ - public void addMetric(String name, Measurable measurable) { - addMetric(name, "", measurable); - } - - /** - * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor. - * This is a way to expose existing values as metrics. - * @param name The name of the metric - * @param description A human-readable description to include in the metric - * @param measurable The measurable that will be measured by this metric - */ - public void addMetric(String name, String description, Measurable measurable) { - addMetric(name, description, null, measurable); - } - - /** - * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor. - * This is a way to expose existing values as metrics. - * @param name The name of the metric - * @param config The configuration to use when measuring this measurable + * @param metricName The name of the metric * @param measurable The measurable that will be measured by this metric */ - public void addMetric(String name, MetricConfig config, Measurable measurable) { - addMetric(name, "", config, measurable); + public void addMetric(MetricName metricName, Measurable measurable) { + addMetric(metricName, null, measurable); } /** * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor. * This is a way to expose existing values as metrics. - * @param name The name of the metric - * @param description A human-readable description to include in the metric + * @param metricName The name of the metric * @param config The configuration to use when measuring this measurable * @param measurable The measurable that will be measured by this metric */ - public synchronized void addMetric(String name, String description, MetricConfig config, Measurable measurable) { + public synchronized void addMetric(MetricName metricName, MetricConfig config, Measurable measurable) { KafkaMetric m = new KafkaMetric(new Object(), - Utils.notNull(name), - Utils.notNull(description), + Utils.notNull(metricName), Utils.notNull(measurable), config == null ? this.config : config, time); @@ -182,23 +175,25 @@ public synchronized void addReporter(MetricsReporter reporter) { } synchronized void registerMetric(KafkaMetric metric) { - if (this.metrics.containsKey(metric.name())) - throw new IllegalArgumentException("A metric named '" + metric.name() + "' already exists, can't register another one."); - this.metrics.put(metric.name(), metric); + MetricName metricName = metric.metricName(); + if (this.metrics.containsKey(metricName)) + throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one."); + this.metrics.put(metricName, metric); for (MetricsReporter reporter : reporters) reporter.metricChange(metric); } /** - * Get all the metrics currently maintained indexed by metric name + * Get all the metrics currently maintained indexed by metricName */ - public Map metrics() { + public Map metrics() { return this.metrics; } /** * Close this metrics repository. */ + @Override public void close() { for (MetricsReporter reporter : this.reporters) reporter.close(); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java index 2c395b1f8f343..7acc19e4e9380 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java @@ -1,27 +1,25 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * 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 org.apache.kafka.common.metrics; import java.util.List; +import org.apache.kafka.common.Configurable; + /** - * A plugin interface to allow things to listen as new metrics are created so they can be reported + * A plugin interface to allow things to listen as new metrics are created so they can be reported. */ -public interface MetricsReporter { +public interface MetricsReporter extends Configurable { /** * This is called when the reporter is first registered to initially register all existing metrics diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index 7e4849b7a1480..ca823fd463952 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics; @@ -22,11 +18,11 @@ import java.util.List; import java.util.Set; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.CompoundStat.NamedMeasurable; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; - /** * A sensor applies a continuous sequence of numerical values to a set of associated metrics. For example a sensor on * message size would record a sequence of message sizes using the {@link #record(double)} api and would maintain a set @@ -46,7 +42,7 @@ public final class Sensor { super(); this.registry = registry; this.name = Utils.notNull(name); - this.parents = parents; + this.parents = parents == null ? new Sensor[0] : parents; this.metrics = new ArrayList(); this.stats = new ArrayList(); this.config = config; @@ -83,30 +79,42 @@ public void record() { * bound */ public void record(double value) { - record(value, time.nanoseconds()); + record(value, time.milliseconds()); } - private void record(double value, long time) { + /** + * Record a value at a known time. This method is slightly faster than {@link #record(double)} since it will reuse + * the time stamp. + * @param value The value we are recording + * @param timeMs The current POSIX time in milliseconds + * @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum + * bound + */ + public void record(double value, long timeMs) { synchronized (this) { // increment all the stats for (int i = 0; i < this.stats.size(); i++) - this.stats.get(i).record(config, value, time); - checkQuotas(time); - + this.stats.get(i).record(config, value, timeMs); + checkQuotas(timeMs); } for (int i = 0; i < parents.length; i++) - parents[i].record(value, time); + parents[i].record(value, timeMs); } - private void checkQuotas(long time) { + /** + * Check if we have violated our quota for any metric that has a configured quota + * @param timeMs + */ + private void checkQuotas(long timeMs) { for (int i = 0; i < this.metrics.size(); i++) { KafkaMetric metric = this.metrics.get(i); MetricConfig config = metric.config(); if (config != null) { Quota quota = config.quota(); - if (quota != null) - if (!quota.acceptable(metric.value(time))) - throw new QuotaViolationException("Metric " + metric.name() + " is in violation of its quota of " + quota.bound()); + if (quota != null) { + if (!quota.acceptable(metric.value(timeMs))) + throw new QuotaViolationException(metric.metricName() + " is in violation of its quota of " + quota.bound()); + } } } } @@ -127,52 +135,30 @@ public void add(CompoundStat stat) { public synchronized void add(CompoundStat stat, MetricConfig config) { this.stats.add(Utils.notNull(stat)); for (NamedMeasurable m : stat.stats()) { - KafkaMetric metric = new KafkaMetric(this, m.name(), m.description(), m.stat(), config == null ? this.config : config, time); + KafkaMetric metric = new KafkaMetric(this, m.name(), m.stat(), config == null ? this.config : config, time); this.registry.registerMetric(metric); this.metrics.add(metric); } } /** - * Add a metric with default configuration and no description. Equivalent to - * {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, "", stat, null)} - * - */ - public void add(String name, MeasurableStat stat) { - add(name, stat, null); - } - - /** - * Add a metric with default configuration. Equivalent to - * {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, description, stat, null)} - * - */ - public void add(String name, String description, MeasurableStat stat) { - add(name, description, stat, null); - } - - /** - * Add a metric to this sensor with no description. Equivalent to - * {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, "", stat, config)} - * @param name - * @param stat - * @param config + * Register a metric with this sensor + * @param metricName The name of the metric + * @param stat The statistic to keep */ - public void add(String name, MeasurableStat stat, MetricConfig config) { - add(name, "", stat, config); + public void add(MetricName metricName, MeasurableStat stat) { + add(metricName, stat, null); } /** * Register a metric with this sensor - * @param name The name of the metric - * @param description A description used when reporting the value + * @param metricName The name of the metric * @param stat The statistic to keep * @param config A special configuration for this metric. If null use the sensor default configuration. */ - public synchronized void add(String name, String description, MeasurableStat stat, MetricConfig config) { - KafkaMetric metric = new KafkaMetric(this, - Utils.notNull(name), - Utils.notNull(description), + public synchronized void add(MetricName metricName, MeasurableStat stat, MetricConfig config) { + KafkaMetric metric = new KafkaMetric(new Object(), + Utils.notNull(metricName), Utils.notNull(stat), config == null ? this.config : config, time); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java b/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java index e02389c969f52..0eb7ab2ad1a66 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java @@ -25,8 +25,8 @@ public interface Stat { * Record the given value * @param config The configuration to use for this metric * @param value The value to record - * @param time The time this value occurred + * @param timeMs The POSIX time in milliseconds this value occurred */ - public void record(MetricConfig config, double value, long time); + public void record(MetricConfig config, double value, long timeMs); } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java index 51725b278495f..ed6767f369e02 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics.stats; @@ -20,7 +16,6 @@ import org.apache.kafka.common.metrics.MetricConfig; - /** * A {@link SampledStat} that maintains a simple average over its samples. */ diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java index 3cdd1d054e0b9..90c0bf549a9ec 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics.stats; @@ -20,7 +16,6 @@ import org.apache.kafka.common.metrics.MetricConfig; - /** * A {@link SampledStat} that maintains a simple count of what it has seen. */ diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java index cf91f5f90660f..c571b4b717660 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java @@ -58,7 +58,7 @@ public void clear() { @Override public String toString() { - StringBuilder b = new StringBuilder('{'); + StringBuilder b = new StringBuilder("{"); for (int i = 0; i < this.hist.length - 1; i++) { b.append(String.format("%.10f", binScheme.fromBin(i))); b.append(':'); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java index bba59721faa14..6bbb0a3a706c5 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics.stats; @@ -20,7 +16,6 @@ import org.apache.kafka.common.metrics.MetricConfig; - /** * A {@link SampledStat} that gives the max over its samples. */ diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java index d370049a7f1c0..9f74417193a04 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics.stats; @@ -20,7 +16,6 @@ import org.apache.kafka.common.metrics.MetricConfig; - /** * A {@link SampledStat} that gives the min over its samples. */ diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java index 7365ceb39072a..fb741ae086c71 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java @@ -16,31 +16,23 @@ */ package org.apache.kafka.common.metrics.stats; +import org.apache.kafka.common.MetricName; + public class Percentile { - private final String name; - private final String description; + private final MetricName name; private final double percentile; - public Percentile(String name, double percentile) { - this(name, "", percentile); - } - - public Percentile(String name, String description, double percentile) { + public Percentile(MetricName name, double percentile) { super(); this.name = name; - this.description = description; this.percentile = percentile; } - public String name() { + public MetricName name() { return this.name; } - public String description() { - return this.description; - } - public double percentile() { return this.percentile; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java index 4d549167ea7c1..78c93e88fa0b8 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics.stats; @@ -26,7 +22,6 @@ import org.apache.kafka.common.metrics.stats.Histogram.ConstantBinScheme; import org.apache.kafka.common.metrics.stats.Histogram.LinearBinScheme; - /** * A compound stat that reports one or more percentiles */ @@ -64,7 +59,7 @@ public List stats() { List ms = new ArrayList(this.percentiles.length); for (Percentile percentile : this.percentiles) { final double pct = percentile.percentile(); - ms.add(new NamedMeasurable(percentile.name(), percentile.description(), new Measurable() { + ms.add(new NamedMeasurable(percentile.name(), new Measurable() { public double measure(MetricConfig config, long now) { return value(config, now, pct / 100.0); } @@ -74,7 +69,7 @@ public double measure(MetricConfig config, long now) { } public double value(MetricConfig config, long now, double quantile) { - timeoutObsoleteSamples(config, now); + purgeObsoleteSamples(config, now); float count = 0.0f; for (Sample sample : this.samples) count += sample.eventCount; @@ -99,12 +94,12 @@ public double combine(List samples, MetricConfig config, long now) { } @Override - protected HistogramSample newSample(long now) { - return new HistogramSample(this.binScheme, now); + protected HistogramSample newSample(long timeMs) { + return new HistogramSample(this.binScheme, timeMs); } @Override - protected void update(Sample sample, MetricConfig config, double value, long now) { + protected void update(Sample sample, MetricConfig config, double value, long timeMs) { HistogramSample hist = (HistogramSample) sample; hist.histogram.record(value); } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index 3b0454f26490d..98429da34418f 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics.stats; @@ -22,12 +18,11 @@ import org.apache.kafka.common.metrics.MeasurableStat; import org.apache.kafka.common.metrics.MetricConfig; - /** - * The rate of the given quanitity. By default this is the total observed over a set of samples from a sampled statistic - * divided by the ellapsed time over the sample windows. Alternative {@link SampledStat} implementations can be - * provided, however, to record the rate of occurences (e.g. the count of values measured over the time interval) or - * other such values. + * The rate of the given quantity. By default this is the total observed over a set of samples from a sampled statistic + * divided by the elapsed time over the sample windows. Alternative {@link SampledStat} implementations can be provided, + * however, to record the rate of occurrences (e.g. the count of values measured over the time interval) or other such + * values. */ public class Rate implements MeasurableStat { @@ -42,6 +37,10 @@ public Rate(TimeUnit unit) { this(unit, new SampledTotal()); } + public Rate(SampledStat stat) { + this(TimeUnit.SECONDS, stat); + } + public Rate(TimeUnit unit, SampledStat stat) { this.stat = stat; this.unit = unit; @@ -52,32 +51,33 @@ public String unitName() { } @Override - public void record(MetricConfig config, double value, long time) { - this.stat.record(config, value, time); + public void record(MetricConfig config, double value, long timeMs) { + this.stat.record(config, value, timeMs); } @Override public double measure(MetricConfig config, long now) { - double ellapsed = convert(now - stat.oldest().lastWindow); - return stat.measure(config, now) / ellapsed; + double value = stat.measure(config, now); + double elapsed = convert(now - stat.oldest(now).lastWindowMs); + return value / elapsed; } private double convert(long time) { switch (unit) { case NANOSECONDS: - return time; + return time * 1000.0 * 1000.0; case MICROSECONDS: - return time / 1000.0; + return time * 1000.0; case MILLISECONDS: - return time / (1000.0 * 1000.0); + return time; case SECONDS: - return time / (1000.0 * 1000.0 * 1000.0); + return time / 1000.0; case MINUTES: - return time / (60.0 * 1000.0 * 1000.0 * 1000.0); + return time / (60.0 * 1000.0); case HOURS: - return time / (60.0 * 60.0 * 1000.0 * 1000.0 * 1000.0); + return time / (60.0 * 60.0 * 1000.0); case DAYS: - return time / (24.0 * 60.0 * 60.0 * 1000.0 * 1000.0 * 1000.0); + return time / (24.0 * 60.0 * 60.0 * 1000.0); default: throw new IllegalStateException("Unknown unit: " + unit); } @@ -90,7 +90,7 @@ public SampledTotal() { } @Override - protected void update(Sample sample, MetricConfig config, double value, long now) { + protected void update(Sample sample, MetricConfig config, double value, long timeMs) { sample.value += value; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java index f8b413a8c273c..b341b7daaa102 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics.stats; @@ -22,7 +18,6 @@ import org.apache.kafka.common.metrics.MeasurableStat; import org.apache.kafka.common.metrics.MetricConfig; - /** * A SampledStat records a single scalar value measured over one or more samples. Each sample is recorded over a * configurable window. The window can be defined by number of events or ellapsed time (or both, if both are given the @@ -45,57 +40,65 @@ public SampledStat(double initialValue) { } @Override - public void record(MetricConfig config, double value, long now) { - Sample sample = current(now); - if (sample.isComplete(now, config)) - sample = advance(config, now); - update(sample, config, value, now); + public void record(MetricConfig config, double value, long timeMs) { + Sample sample = current(timeMs); + if (sample.isComplete(timeMs, config)) + sample = advance(config, timeMs); + update(sample, config, value, timeMs); sample.eventCount += 1; } - private Sample advance(MetricConfig config, long now) { + private Sample advance(MetricConfig config, long timeMs) { this.current = (this.current + 1) % config.samples(); if (this.current >= samples.size()) { - Sample sample = newSample(now); + Sample sample = newSample(timeMs); this.samples.add(sample); return sample; } else { - Sample sample = current(now); - sample.reset(now); + Sample sample = current(timeMs); + sample.reset(timeMs); return sample; } } - protected Sample newSample(long now) { - return new Sample(this.initialValue, now); + protected Sample newSample(long timeMs) { + return new Sample(this.initialValue, timeMs); } @Override public double measure(MetricConfig config, long now) { - timeoutObsoleteSamples(config, now); + purgeObsoleteSamples(config, now); return combine(this.samples, config, now); } - public Sample current(long now) { + public Sample current(long timeMs) { if (samples.size() == 0) - this.samples.add(newSample(now)); + this.samples.add(newSample(timeMs)); return this.samples.get(this.current); } - public Sample oldest() { - return this.samples.get((this.current + 1) % this.samples.size()); + public Sample oldest(long now) { + if (samples.size() == 0) + this.samples.add(newSample(now)); + Sample oldest = this.samples.get(0); + for (int i = 1; i < this.samples.size(); i++) { + Sample curr = this.samples.get(i); + if (curr.lastWindowMs < oldest.lastWindowMs) + oldest = curr; + } + return oldest; } - protected abstract void update(Sample sample, MetricConfig config, double value, long now); + protected abstract void update(Sample sample, MetricConfig config, double value, long timeMs); public abstract double combine(List samples, MetricConfig config, long now); - /* Timeout any windows that have expired in the absense of any events */ - protected void timeoutObsoleteSamples(MetricConfig config, long now) { + /* Timeout any windows that have expired in the absence of any events */ + protected void purgeObsoleteSamples(MetricConfig config, long now) { + long expireAge = config.samples() * config.timeWindowMs(); for (int i = 0; i < samples.size(); i++) { - int idx = (this.current + i) % samples.size(); - Sample sample = this.samples.get(idx); - if (now - sample.lastWindow >= (i + 1) * config.timeWindowNs()) + Sample sample = this.samples.get(i); + if (now - sample.lastWindowMs >= expireAge) sample.reset(now); } } @@ -103,24 +106,24 @@ protected void timeoutObsoleteSamples(MetricConfig config, long now) { protected static class Sample { public double initialValue; public long eventCount; - public long lastWindow; + public long lastWindowMs; public double value; public Sample(double initialValue, long now) { this.initialValue = initialValue; this.eventCount = 0; - this.lastWindow = now; + this.lastWindowMs = now; this.value = initialValue; } public void reset(long now) { this.eventCount = 0; - this.lastWindow = now; + this.lastWindowMs = now; this.value = initialValue; } - public boolean isComplete(long now, MetricConfig config) { - return now - lastWindow >= config.timeWindowNs() || eventCount >= config.eventWindow(); + public boolean isComplete(long timeMs, MetricConfig config) { + return timeMs - lastWindowMs >= config.timeWindowMs() || eventCount >= config.eventWindow(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java index a9940ed7ea1db..67999a90ab9fe 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics.stats; @@ -35,7 +31,7 @@ public Total(double value) { } @Override - public void record(MetricConfig config, double value, long time) { + public void record(MetricConfig config, double value, long now) { this.total += value; } diff --git a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java index 129ae827bccbd..159c30190e985 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java @@ -25,11 +25,11 @@ */ public class ByteBufferReceive implements Receive { - private final int source; + private final String source; private final ByteBuffer[] buffers; private int remaining; - public ByteBufferReceive(int source, ByteBuffer... buffers) { + public ByteBufferReceive(String source, ByteBuffer... buffers) { super(); this.source = source; this.buffers = buffers; @@ -38,7 +38,7 @@ public ByteBufferReceive(int source, ByteBuffer... buffers) { } @Override - public int source() { + public String source() { return source; } @@ -54,8 +54,4 @@ public long readFrom(ScatteringByteChannel channel) throws IOException { return read; } - public ByteBuffer[] reify() { - return buffers; - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java index a8e0413681cd1..df0e6d5105ca9 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.network; @@ -26,45 +22,41 @@ */ public class ByteBufferSend implements Send { - private final int destination; + private final String destination; protected final ByteBuffer[] buffers; private int remaining; + private int size; - public ByteBufferSend(int destination, ByteBuffer... buffers) { + public ByteBufferSend(String destination, ByteBuffer... buffers) { super(); this.destination = destination; this.buffers = buffers; for (int i = 0; i < buffers.length; i++) remaining += buffers[i].remaining(); + this.size = remaining; } @Override - public int destination() { + public String destination() { return destination; } @Override - public boolean complete() { - return remaining > 0; + public boolean completed() { + return remaining <= 0; } @Override - public ByteBuffer[] reify() { - return this.buffers; - } - - @Override - public int remaining() { - return this.remaining; + public long size() { + return this.size; } @Override public long writeTo(GatheringByteChannel channel) throws IOException { long written = channel.write(buffers); if (written < 0) - throw new EOFException("This shouldn't happen."); + throw new EOFException("Wrote negative bytes to channel. This shouldn't happen."); remaining -= written; return written; } - } diff --git a/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java b/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java new file mode 100644 index 0000000000000..a5bdd62240c24 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java @@ -0,0 +1,30 @@ +/** + * 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 org.apache.kafka.common.network; + +import org.apache.kafka.common.KafkaException; + +public class InvalidReceiveException extends KafkaException { + + public InvalidReceiveException(String message) { + super(message); + } + + public InvalidReceiveException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/MultiSend.java b/clients/src/main/java/org/apache/kafka/common/network/MultiSend.java new file mode 100644 index 0000000000000..0e14a395cfe37 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/MultiSend.java @@ -0,0 +1,100 @@ +/** + * 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 org.apache.kafka.common.network; + +import org.apache.kafka.common.KafkaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.channels.GatheringByteChannel; +import java.util.Iterator; +import java.util.List; + +/** + * A set of composite sends, sent one after another + */ + +public class MultiSend implements Send { + + private static final Logger log = LoggerFactory.getLogger(MultiSend.class); + private String dest; + private long totalWritten = 0; + private List sends; + private Iterator sendsIterator; + private Send current; + private boolean doneSends = false; + private long size = 0; + + public MultiSend(String dest, List sends) { + this.dest = dest; + this.sends = sends; + this.sendsIterator = sends.iterator(); + nextSendOrDone(); + for (Send send: sends) + this.size += send.size(); + } + + @Override + public long size() { + return size; + } + + @Override + public String destination() { + return dest; + } + + @Override + public boolean completed() { + if (doneSends) { + if (totalWritten != size) + log.error("mismatch in sending bytes over socket; expected: " + size + " actual: " + totalWritten); + return true; + } else { + return false; + } + } + + @Override + public long writeTo(GatheringByteChannel channel) throws IOException { + if (completed()) + throw new KafkaException("This operation cannot be completed on a complete request."); + + int totalWrittenPerCall = 0; + boolean sendComplete = false; + do { + long written = current.writeTo(channel); + totalWritten += written; + totalWrittenPerCall += written; + sendComplete = current.completed(); + if (sendComplete) + nextSendOrDone(); + } while (!completed() && sendComplete); + if (log.isTraceEnabled()) + log.trace("Bytes written as part of multisend call : " + totalWrittenPerCall + "Total bytes written so far : " + totalWritten + "Expected bytes to write : " + size); + return totalWrittenPerCall; + } + + // update current if there's a next Send, mark sends as done if there isn't + private void nextSendOrDone() { + if (sendsIterator.hasNext()) + current = sendsIterator.next(); + else + doneSends = true; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java b/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java index 51d4892dfc185..3ca0098b8ec8c 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java @@ -1,24 +1,21 @@ /** - * 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. + * 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 org.apache.kafka.common.network; import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; import java.nio.channels.ScatteringByteChannel; /** @@ -26,24 +23,42 @@ */ public class NetworkReceive implements Receive { - private final int source; + public final static String UNKNOWN_SOURCE = ""; + public final static int UNLIMITED = -1; + + private final String source; private final ByteBuffer size; + private final int maxSize; private ByteBuffer buffer; - public NetworkReceive(int source, ByteBuffer buffer) { + + public NetworkReceive(String source, ByteBuffer buffer) { this.source = source; this.buffer = buffer; this.size = null; + this.maxSize = UNLIMITED; + } + + public NetworkReceive(String source) { + this.source = source; + this.size = ByteBuffer.allocate(4); + this.buffer = null; + this.maxSize = UNLIMITED; } - public NetworkReceive(int source) { + public NetworkReceive(int maxSize, String source) { this.source = source; this.size = ByteBuffer.allocate(4); this.buffer = null; + this.maxSize = maxSize; + } + + public NetworkReceive() { + this(UNKNOWN_SOURCE); } @Override - public int source() { + public String source() { return source; } @@ -52,13 +67,15 @@ public boolean complete() { return !size.hasRemaining() && !buffer.hasRemaining(); } - @Override - public ByteBuffer[] reify() { - return new ByteBuffer[] { this.buffer }; + public long readFrom(ScatteringByteChannel channel) throws IOException { + return readFromReadableChannel(channel); } - @Override - public long readFrom(ScatteringByteChannel channel) throws IOException { + // Need a method to read from ReadableByteChannel because BlockingChannel requires read with timeout + // See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work + // This can go away after we get rid of BlockingChannel + @Deprecated + public long readFromReadableChannel(ReadableByteChannel channel) throws IOException { int read = 0; if (size.hasRemaining()) { int bytesRead = channel.read(size); @@ -67,10 +84,12 @@ public long readFrom(ScatteringByteChannel channel) throws IOException { read += bytesRead; if (!size.hasRemaining()) { size.rewind(); - int requestSize = size.getInt(); - if (requestSize < 0) - throw new IllegalStateException("Invalid request (size = " + requestSize + ")"); - this.buffer = ByteBuffer.allocate(requestSize); + int receiveSize = size.getInt(); + if (receiveSize < 0) + throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + ")"); + if (maxSize != UNLIMITED && receiveSize > maxSize) + throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + " larger than " + maxSize + ")"); + this.buffer = ByteBuffer.allocate(receiveSize); } } if (buffer != null) { @@ -87,4 +106,14 @@ public ByteBuffer payload() { return this.buffer; } + // Used only by BlockingChannel, so we may be able to get rid of this when/if we get rid of BlockingChannel + @Deprecated + public long readCompletely(ReadableByteChannel channel) throws IOException { + int totalRead = 0; + while (!complete()) { + totalRead += readFromReadableChannel(channel); + } + return totalRead; + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java b/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java index 68327cd3a734f..49964b03a63cf 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java +++ b/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java @@ -23,7 +23,7 @@ */ public class NetworkSend extends ByteBufferSend { - public NetworkSend(int destination, ByteBuffer... buffers) { + public NetworkSend(String destination, ByteBuffer... buffers) { super(destination, sizeDelimit(buffers)); } diff --git a/clients/src/main/java/org/apache/kafka/common/network/Receive.java b/clients/src/main/java/org/apache/kafka/common/network/Receive.java index 4e33078c1eec8..4b14431a6cea1 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Receive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Receive.java @@ -17,7 +17,6 @@ package org.apache.kafka.common.network; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.channels.ScatteringByteChannel; /** @@ -28,18 +27,13 @@ public interface Receive { /** * The numeric id of the source from which we are receiving data. */ - public int source(); + public String source(); /** * Are we done receiving data? */ public boolean complete(); - /** - * Turn this receive into ByteBuffer instances, if possible (otherwise returns null). - */ - public ByteBuffer[] reify(); - /** * Read bytes into this receive from the given channel * @param channel The channel to read from diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java index cef75d8cfd91d..618a0fa53848a 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.network; @@ -33,12 +29,12 @@ public interface Selectable { * @param receiveBufferSize The receive buffer for the socket * @throws IOException If we cannot begin connecting */ - public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException; + public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException; /** * Begin disconnecting the connection identified by the given id */ - public void disconnect(int id); + public void disconnect(String id); /** * Wakeup this selector if it is blocked on I/O @@ -51,34 +47,60 @@ public interface Selectable { public void close(); /** - * Initiate any sends provided, and make progress on any other I/O operations in-flight (connections, - * disconnections, existing sends, and receives) + * Queue the given request for sending in the subsequent {@poll(long)} calls + * @param send The request to send + */ + public void send(Send send); + + /** + * Do I/O. Reads, writes, connection establishment, etc. * @param timeout The amount of time to block if there is nothing to do - * @param sends The new sends to initiate * @throws IOException */ - public void poll(long timeout, List sends) throws IOException; + public void poll(long timeout) throws IOException; /** - * The list of sends that completed on the last {@link #poll(long, List) poll()} call. + * The list of sends that completed on the last {@link #poll(long, List) poll()} call. */ - public List completedSends(); + public List completedSends(); /** - * The list of receives that completed on the last {@link #poll(long, List) poll()} call. + * The list of receives that completed on the last {@link #poll(long, List) poll()} call. */ public List completedReceives(); /** - * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()} + * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()} * call. */ - public List disconnected(); + public List disconnected(); /** - * The list of connections that completed their connection on the last {@link #poll(long, List) poll()} + * The list of connections that completed their connection on the last {@link #poll(long, List) poll()} * call. */ - public List connected(); + public List connected(); + + /** + * Disable reads from the given connection + * @param id The id for the connection + */ + public void mute(String id); + + /** + * Re-enable reads from the given connection + * @param id The id for the connection + */ + public void unmute(String id); + + /** + * Disable reads from all connections + */ + public void muteAll(); + + /** + * Re-enable reads from all connections + */ + public void unmuteAll(); } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 8ed4c73146b2e..aaf60c98c2c0f 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -1,48 +1,51 @@ /** - * 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. + * 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 org.apache.kafka.common.network; +import java.io.EOFException; import java.io.IOException; +import java.net.ConnectException; import java.net.InetSocketAddress; import java.net.Socket; -import java.nio.channels.CancelledKeyException; -import java.nio.channels.SelectionKey; -import java.nio.channels.SocketChannel; -import java.nio.channels.UnresolvedAddressException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.nio.channels.*; +import java.util.*; +import java.util.concurrent.TimeUnit; import org.apache.kafka.common.KafkaException; - +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Count; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * A selector interface for doing non-blocking multi-connection network I/O. + * A nioSelector interface for doing non-blocking multi-connection network I/O. *

        * This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and * responses. *

        - * A connection can be added to the selector associated with an integer id by doing + * A connection can be added to the nioSelector associated with an integer id by doing * *

        - * selector.connect(42, new InetSocketAddress("google.com", server.port), 64000, 64000);
        + * nioSelector.connect(42, new InetSocketAddress("google.com", server.port), 64000, 64000);
          * 
        * * The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating @@ -53,41 +56,71 @@ * *
          * List<NetworkRequest> requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
        - * selector.poll(TIMEOUT_MS, requestsToSend);
        + * nioSelector.poll(TIMEOUT_MS, requestsToSend);
          * 
        * - * The selector maintains several lists that are reset by each call to poll() which are available via + * The nioSelector maintains several lists that are reset by each call to poll() which are available via * various getters. These are reset by each call to poll(). * * This class is not thread safe! */ public class Selector implements Selectable { - private final java.nio.channels.Selector selector; - private final Map keys; - private final List completedSends; + private static final Logger log = LoggerFactory.getLogger(Selector.class); + + private final java.nio.channels.Selector nioSelector; + private final Map keys; + private final List completedSends; private final List completedReceives; - private final List disconnected; - private final List connected; + private final List disconnected; + private final List connected; + private final List failedSends; + private final Time time; + private final SelectorMetrics sensors; + private final String metricGrpPrefix; + private final Map metricTags; + private final Map lruConnections; + private final long connectionsMaxIdleNanos; + private final int maxReceiveSize; + private final boolean metricsPerConnection; + private long currentTimeNanos; + private long nextIdleCloseCheckTime; + /** - * Create a new selector + * Create a new nioSelector */ - public Selector() { + public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, Time time, String metricGrpPrefix, Map metricTags, boolean metricsPerConnection) { try { - this.selector = java.nio.channels.Selector.open(); + this.nioSelector = java.nio.channels.Selector.open(); } catch (IOException e) { throw new KafkaException(e); } - this.keys = new HashMap(); - this.completedSends = new ArrayList(); + this.maxReceiveSize = maxReceiveSize; + this.connectionsMaxIdleNanos = connectionMaxIdleMs * 1000 * 1000; + this.time = time; + this.metricGrpPrefix = metricGrpPrefix; + this.metricTags = metricTags; + this.keys = new HashMap(); + this.completedSends = new ArrayList(); this.completedReceives = new ArrayList(); - this.connected = new ArrayList(); - this.disconnected = new ArrayList(); + this.connected = new ArrayList(); + this.disconnected = new ArrayList(); + this.failedSends = new ArrayList(); + this.sensors = new SelectorMetrics(metrics); + // initial capacity and load factor are default, we set them explicitly because we want to set accessOrder = true + this.lruConnections = new LinkedHashMap(16, .75F, true); + currentTimeNanos = new SystemTime().nanoseconds(); + nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos; + this.metricsPerConnection = metricsPerConnection; + } + + public Selector(long connectionMaxIdleMS, Metrics metrics, Time time, String metricGrpPrefix, Map metricTags) { + this(NetworkReceive.UNLIMITED, connectionMaxIdleMS, metrics, time, metricGrpPrefix, metricTags, true); } /** - * Begin connecting to the given address and add the connection to this selector associated with the given id + * Begin connecting to the given address and add the connection to this nioSelector associated with the given id * number. *

        * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)} @@ -97,10 +130,13 @@ public Selector() { * @param sendBufferSize The send buffer for the new connection * @param receiveBufferSize The receive buffer for the new connection * @throws IllegalStateException if there is already a connection for that id - * @throws UnresolvedAddressException if DNS resolution fails on the hostname + * @throws IOException if DNS resolution fails on the hostname or if the broker is down */ @Override - public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { + public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { + if (this.keys.containsKey(id)) + throw new IllegalStateException("There is already a connection for id " + id); + SocketChannel channel = SocketChannel.open(); channel.configureBlocking(false); Socket socket = channel.socket(); @@ -111,13 +147,25 @@ public void connect(int id, InetSocketAddress address, int sendBufferSize, int r try { channel.connect(address); } catch (UnresolvedAddressException e) { + channel.close(); + throw new IOException("Can't resolve address: " + address, e); + } catch (IOException e) { channel.close(); throw e; } - SelectionKey key = channel.register(this.selector, SelectionKey.OP_CONNECT); + SelectionKey key = channel.register(this.nioSelector, SelectionKey.OP_CONNECT); + key.attach(new Transmissions(id)); + this.keys.put(id, key); + } + + /** + * Register the nioSelector with an existing channel + * Use this on server-side, when a connection is accepted by a different thread but processed by the Selector + * Note that we are not checking if the connection id is valid - since the connection already exists + */ + public void register(String id, SocketChannel channel) throws ClosedChannelException { + SelectionKey key = channel.register(nioSelector, SelectionKey.OP_READ); key.attach(new Transmissions(id)); - if (this.keys.containsKey(key)) - throw new IllegalStateException("There is already a connection for id " + id); this.keys.put(id, key); } @@ -126,18 +174,18 @@ public void connect(int id, InetSocketAddress address, int sendBufferSize, int r * processed until the next {@link #poll(long, List) poll()} call. */ @Override - public void disconnect(int id) { + public void disconnect(String id) { SelectionKey key = this.keys.get(id); if (key != null) key.cancel(); } /** - * Interrupt the selector if it is blocked waiting to do I/O. + * Interrupt the nioSelector if it is blocked waiting to do I/O. */ @Override public void wakeup() { - this.selector.wakeup(); + this.nioSelector.wakeup(); } /** @@ -145,25 +193,38 @@ public void wakeup() { */ @Override public void close() { - for (SelectionKey key : this.selector.keys()) { - try { - close(key); - } catch (IOException e) { - e.printStackTrace(); - } - } + List connections = new LinkedList(keys.keySet()); + for (String id: connections) + close(id); + try { - this.selector.close(); + this.nioSelector.close(); } catch (IOException e) { - e.printStackTrace(); + log.error("Exception closing nioSelector:", e); + } + } + + /** + * Queue the given request for sending in the subsequent {@poll(long)} calls + * @param send The request to send + */ + public void send(Send send) { + SelectionKey key = keyForId(send.destination()); + Transmissions transmissions = transmissions(key); + if (transmissions.hasSend()) + throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress."); + transmissions.send = send; + try { + key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); + } catch (CancelledKeyException e) { + close(transmissions.id); + this.failedSends.add(send.destination()); } } /** * Do whatever I/O can be done on each connection without blocking. This includes completing connections, completing * disconnections, initiating new sends, or making progress on in-progress sends or receives. - *

        - * The provided network sends will be started. * * When this call is completed the user can check for completed sends, receives, connections or disconnects using * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These @@ -171,33 +232,22 @@ public void close() { * completed I/O. * * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely. - * @param sends The list of new sends to begin - * * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is * already an in-progress send */ @Override - public void poll(long timeout, List sends) throws IOException { + public void poll(long timeout) throws IOException { clear(); - /* register for write interest on any new sends */ - for (NetworkSend send : sends) { - SelectionKey key = keyForId(send.destination()); - Transmissions transmissions = transmissions(key); - if (transmissions.hasSend()) - throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress."); - transmissions.send = send; - try { - key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); - } catch (CancelledKeyException e) { - close(key); - } - } - /* check ready keys */ + long startSelect = time.nanoseconds(); int readyKeys = select(timeout); + long endSelect = time.nanoseconds(); + currentTimeNanos = endSelect; + this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds()); + if (readyKeys > 0) { - Set keys = this.selector.selectedKeys(); + Set keys = this.nioSelector.selectedKeys(); Iterator iter = keys.iterator(); while (iter.hasNext()) { SelectionKey key = iter.next(); @@ -205,53 +255,85 @@ public void poll(long timeout, List sends) throws IOException { Transmissions transmissions = transmissions(key); SocketChannel channel = channel(key); + + // register all per-connection metrics at once + sensors.maybeRegisterConnectionMetrics(transmissions.id); + lruConnections.put(transmissions.id, currentTimeNanos); + try { - /* - * complete any connections that have finished their handshake - */ + /* complete any connections that have finished their handshake */ if (key.isConnectable()) { channel.finishConnect(); key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); this.connected.add(transmissions.id); + this.sensors.connectionCreated.record(); + log.debug("Connection {} created", transmissions.id); } /* read from any connections that have readable data */ if (key.isReadable()) { if (!transmissions.hasReceive()) - transmissions.receive = new NetworkReceive(transmissions.id); - transmissions.receive.readFrom(channel); + transmissions.receive = new NetworkReceive(maxReceiveSize, transmissions.id); + try { + transmissions.receive.readFrom(channel); + } catch (InvalidReceiveException e) { + log.error("Invalid data received from " + transmissions.id + " closing connection", e); + close(transmissions.id); + this.disconnected.add(transmissions.id); + throw e; + } if (transmissions.receive.complete()) { transmissions.receive.payload().rewind(); this.completedReceives.add(transmissions.receive); + this.sensors.recordBytesReceived(transmissions.id, transmissions.receive.payload().limit()); transmissions.clearReceive(); } } - /* - * write to any sockets that have space in their buffer and for which we have data - */ + /* write to any sockets that have space in their buffer and for which we have data */ if (key.isWritable()) { transmissions.send.writeTo(channel); - if (transmissions.send.remaining() <= 0) { + if (transmissions.send.completed()) { this.completedSends.add(transmissions.send); + this.sensors.recordBytesSent(transmissions.id, transmissions.send.size()); transmissions.clearSend(); key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE); } } /* cancel any defunct sockets */ - if (!key.isValid()) - close(key); + if (!key.isValid()) { + close(transmissions.id); + this.disconnected.add(transmissions.id); + } } catch (IOException e) { - e.printStackTrace(); - close(key); + String desc = socketDescription(channel); + if (e instanceof EOFException || e instanceof ConnectException) + log.debug("Connection {} disconnected", desc); + else + log.warn("Error in I/O with connection to {}", desc, e); + close(transmissions.id); + this.disconnected.add(transmissions.id); } } } + long endIo = time.nanoseconds(); + this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); + maybeCloseOldestConnection(); + } + + private String socketDescription(SocketChannel channel) { + Socket socket = channel.socket(); + if (socket == null) + return "[unconnected socket]"; + else if (socket.getInetAddress() != null) + return socket.getInetAddress().toString(); + else + return socket.getLocalAddress().toString(); } @Override - public List completedSends() { + public List completedSends() { return this.completedSends; } @@ -261,15 +343,66 @@ public List completedReceives() { } @Override - public List disconnected() { + public List disconnected() { return this.disconnected; } @Override - public List connected() { + public List connected() { return this.connected; } + @Override + public void mute(String id) { + mute(this.keyForId(id)); + } + + private void mute(SelectionKey key) { + key.interestOps(key.interestOps() & ~SelectionKey.OP_READ); + } + + @Override + public void unmute(String id) { + unmute(this.keyForId(id)); + } + + private void unmute(SelectionKey key) { + key.interestOps(key.interestOps() | SelectionKey.OP_READ); + } + + @Override + public void muteAll() { + for (SelectionKey key : this.keys.values()) + mute(key); + } + + @Override + public void unmuteAll() { + for (SelectionKey key : this.keys.values()) + unmute(key); + } + + private void maybeCloseOldestConnection() { + if (currentTimeNanos > nextIdleCloseCheckTime) { + if (lruConnections.isEmpty()) { + nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos; + } else { + Map.Entry oldestConnectionEntry = lruConnections.entrySet().iterator().next(); + Long connectionLastActiveTime = oldestConnectionEntry.getValue(); + nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos; + if (currentTimeNanos > nextIdleCloseCheckTime) { + String connectionId = oldestConnectionEntry.getKey(); + if (log.isTraceEnabled()) + log.trace("About to close the idle connection from " + connectionId + + " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis"); + + disconnected.add(connectionId); + close(connectionId); + } + } + } + } + /** * Clear the results from the prior poll */ @@ -278,6 +411,8 @@ private void clear() { this.completedReceives.clear(); this.connected.clear(); this.disconnected.clear(); + this.disconnected.addAll(this.failedSends); + this.failedSends.clear(); } /** @@ -289,34 +424,44 @@ private void clear() { */ private int select(long ms) throws IOException { if (ms == 0L) - return this.selector.selectNow(); + return this.nioSelector.selectNow(); else if (ms < 0L) - return this.selector.select(); + return this.nioSelector.select(); else - return this.selector.select(ms); + return this.nioSelector.select(ms); } /** * Begin closing this connection */ - private void close(SelectionKey key) throws IOException { + public void close(String id) { + SelectionKey key = keyForId(id); + lruConnections.remove(id); SocketChannel channel = channel(key); Transmissions trans = transmissions(key); - if (trans != null) - this.disconnected.add(trans.id); + if (trans != null) { + this.keys.remove(trans.id); + trans.clearReceive(); + trans.clearSend(); + } key.attach(null); key.cancel(); - channel.socket().close(); - channel.close(); + try { + channel.socket().close(); + channel.close(); + } catch (IOException e) { + log.error("Exception closing connection to node {}:", trans.id, e); + } + this.sensors.connectionClosed.record(); } /** * Get the selection key associated with this numeric id */ - private SelectionKey keyForId(int id) { + private SelectionKey keyForId(String id) { SelectionKey key = this.keys.get(id); if (key == null) - throw new IllegalStateException("Attempt to write to socket for which there is no open connection."); + throw new IllegalStateException("Attempt to write to socket for which there is no open connection. Connection id " + id + " existing connections " + keys.keySet().toString()); return key; } @@ -338,11 +483,11 @@ private SocketChannel channel(SelectionKey key) { * The id and in-progress send and receive associated with a connection */ private static class Transmissions { - public int id; - public NetworkSend send; + public String id; + public Send send; public NetworkReceive receive; - public Transmissions(int id) { + public Transmissions(String id) { this.id = id; } @@ -363,4 +508,137 @@ public void clearReceive() { } } + private class SelectorMetrics { + private final Metrics metrics; + public final Sensor connectionClosed; + public final Sensor connectionCreated; + public final Sensor bytesTransferred; + public final Sensor bytesSent; + public final Sensor bytesReceived; + public final Sensor selectTime; + public final Sensor ioTime; + + public SelectorMetrics(Metrics metrics) { + this.metrics = metrics; + String metricGrpName = metricGrpPrefix + "-metrics"; + StringBuilder tagsSuffix = new StringBuilder(); + + for (Map.Entry tag: metricTags.entrySet()) { + tagsSuffix.append(tag.getKey()); + tagsSuffix.append("-"); + tagsSuffix.append(tag.getValue()); + } + + this.connectionClosed = this.metrics.sensor("connections-closed:" + tagsSuffix.toString()); + MetricName metricName = new MetricName("connection-close-rate", metricGrpName, "Connections closed per second in the window.", metricTags); + this.connectionClosed.add(metricName, new Rate()); + + this.connectionCreated = this.metrics.sensor("connections-created:" + tagsSuffix.toString()); + metricName = new MetricName("connection-creation-rate", metricGrpName, "New connections established per second in the window.", metricTags); + this.connectionCreated.add(metricName, new Rate()); + + this.bytesTransferred = this.metrics.sensor("bytes-sent-received:" + tagsSuffix.toString()); + metricName = new MetricName("network-io-rate", metricGrpName, "The average number of network operations (reads or writes) on all connections per second.", metricTags); + bytesTransferred.add(metricName, new Rate(new Count())); + + this.bytesSent = this.metrics.sensor("bytes-sent:" + tagsSuffix.toString(), bytesTransferred); + metricName = new MetricName("outgoing-byte-rate", metricGrpName, "The average number of outgoing bytes sent per second to all servers.", metricTags); + this.bytesSent.add(metricName, new Rate()); + metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags); + this.bytesSent.add(metricName, new Rate(new Count())); + metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", metricTags); + this.bytesSent.add(metricName, new Avg()); + metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", metricTags); + this.bytesSent.add(metricName, new Max()); + + this.bytesReceived = this.metrics.sensor("bytes-received:" + tagsSuffix.toString(), bytesTransferred); + metricName = new MetricName("incoming-byte-rate", metricGrpName, "Bytes/second read off all sockets", metricTags); + this.bytesReceived.add(metricName, new Rate()); + metricName = new MetricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags); + this.bytesReceived.add(metricName, new Rate(new Count())); + + this.selectTime = this.metrics.sensor("select-time:" + tagsSuffix.toString()); + metricName = new MetricName("select-rate", metricGrpName, "Number of times the I/O layer checked for new I/O to perform per second", metricTags); + this.selectTime.add(metricName, new Rate(new Count())); + metricName = new MetricName("io-wait-time-ns-avg", metricGrpName, "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", metricTags); + this.selectTime.add(metricName, new Avg()); + metricName = new MetricName("io-wait-ratio", metricGrpName, "The fraction of time the I/O thread spent waiting.", metricTags); + this.selectTime.add(metricName, new Rate(TimeUnit.NANOSECONDS)); + + this.ioTime = this.metrics.sensor("io-time:" + tagsSuffix.toString()); + metricName = new MetricName("io-time-ns-avg", metricGrpName, "The average length of time for I/O per select call in nanoseconds.", metricTags); + this.ioTime.add(metricName, new Avg()); + metricName = new MetricName("io-ratio", metricGrpName, "The fraction of time the I/O thread spent doing I/O", metricTags); + this.ioTime.add(metricName, new Rate(TimeUnit.NANOSECONDS)); + + metricName = new MetricName("connection-count", metricGrpName, "The current number of active connections.", metricTags); + this.metrics.addMetric(metricName, new Measurable() { + public double measure(MetricConfig config, long now) { + return keys.size(); + } + }); + } + + public void maybeRegisterConnectionMetrics(String connectionId) { + if (!connectionId.isEmpty() && metricsPerConnection) { + // if one sensor of the metrics has been registered for the connection, + // then all other sensors should have been registered; and vice versa + String nodeRequestName = "node-" + connectionId + ".bytes-sent"; + Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); + if (nodeRequest == null) { + String metricGrpName = metricGrpPrefix + "-node-metrics"; + + Map tags = new LinkedHashMap(metricTags); + tags.put("node-id", "node-" + connectionId); + + nodeRequest = this.metrics.sensor(nodeRequestName); + MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags); + nodeRequest.add(metricName, new Rate()); + metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", tags); + nodeRequest.add(metricName, new Rate(new Count())); + metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", tags); + nodeRequest.add(metricName, new Avg()); + metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", tags); + nodeRequest.add(metricName, new Max()); + + String nodeResponseName = "node-" + connectionId + ".bytes-received"; + Sensor nodeResponse = this.metrics.sensor(nodeResponseName); + metricName = new MetricName("incoming-byte-rate", metricGrpName, tags); + nodeResponse.add(metricName, new Rate()); + metricName = new MetricName("response-rate", metricGrpName, "The average number of responses received per second.", tags); + nodeResponse.add(metricName, new Rate(new Count())); + + String nodeTimeName = "node-" + connectionId + ".latency"; + Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName); + metricName = new MetricName("request-latency-avg", metricGrpName, tags); + nodeRequestTime.add(metricName, new Avg()); + metricName = new MetricName("request-latency-max", metricGrpName, tags); + nodeRequestTime.add(metricName, new Max()); + } + } + } + + public void recordBytesSent(String connectionId, long bytes) { + long now = time.milliseconds(); + this.bytesSent.record(bytes, now); + if (!connectionId.isEmpty()) { + String nodeRequestName = "node-" + connectionId + ".bytes-sent"; + Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); + if (nodeRequest != null) + nodeRequest.record(bytes, now); + } + } + + public void recordBytesReceived(String connection, int bytes) { + long now = time.milliseconds(); + this.bytesReceived.record(bytes, now); + if (!connection.isEmpty()) { + String nodeRequestName = "node-" + connection + ".bytes-received"; + Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); + if (nodeRequest != null) + nodeRequest.record(bytes, now); + } + } + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/network/Send.java b/clients/src/main/java/org/apache/kafka/common/network/Send.java index d62dff9433f50..8f6daadf6b67c 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Send.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Send.java @@ -1,23 +1,18 @@ /** - * 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. + * 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 org.apache.kafka.common.network; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.channels.GatheringByteChannel; /** @@ -28,22 +23,12 @@ public interface Send { /** * The numeric id for the destination of this send */ - public int destination(); - - /** - * The number of bytes remaining to send - */ - public int remaining(); + public String destination(); /** * Is this send complete? */ - public boolean complete(); - - /** - * An optional method to turn this send into an array of ByteBuffers if possible (otherwise returns null) - */ - public ByteBuffer[] reify(); + public boolean completed(); /** * Write some as-yet unwritten bytes from this send to the provided channel. It may take multiple calls for the send @@ -54,4 +39,9 @@ public interface Send { */ public long writeTo(GatheringByteChannel channel) throws IOException; + /** + * Size of the send + */ + public long size(); + } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 21a2592ea7c7f..b39e9bb53dd51 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -20,21 +20,33 @@ * Identifiers for all the Kafka APIs */ public enum ApiKeys { - PRODUCE(0, "produce"), - FETCH(1, "fetch"), - LIST_OFFSETS(2, "list_offsets"), - METADATA(3, "metadata"), - LEADER_AND_ISR(4, "leader_and_isr"), - STOP_REPLICA(5, "stop_replica"), - OFFSET_COMMIT(6, "offset_commit"), - OFFSET_FETCH(7, "offset_fetch"); + PRODUCE(0, "Produce"), + FETCH(1, "Fetch"), + LIST_OFFSETS(2, "Offsets"), + METADATA(3, "Metadata"), + LEADER_AND_ISR(4, "LeaderAndIsr"), + STOP_REPLICA(5, "StopReplica"), + UPDATE_METADATA_KEY(6, "UpdateMetadata"), + CONTROLLED_SHUTDOWN_KEY(7, "ControlledShutdown"), + OFFSET_COMMIT(8, "OffsetCommit"), + OFFSET_FETCH(9, "OffsetFetch"), + CONSUMER_METADATA(10, "ConsumerMetadata"), + JOIN_GROUP(11, "JoinGroup"), + HEARTBEAT(12, "Heartbeat"); - public static int MAX_API_KEY = 0; + private static ApiKeys[] codeToType; + public static final int MAX_API_KEY; static { + int maxKey = -1; for (ApiKeys key : ApiKeys.values()) { - MAX_API_KEY = Math.max(MAX_API_KEY, key.id); + maxKey = Math.max(maxKey, key.id); } + codeToType = new ApiKeys[maxKey + 1]; + for (ApiKeys key : ApiKeys.values()) { + codeToType[key.id] = key; + } + MAX_API_KEY = maxKey; } /** the perminant and immutable id of an API--this can't change ever */ @@ -48,4 +60,7 @@ private ApiKeys(int id, String name) { this.name = name; } + public static ApiKeys forId(int id) { + return codeToType[id]; + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index f88992a0cafd9..d6c41c119c87c 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -19,18 +19,7 @@ import java.util.HashMap; import java.util.Map; -import org.apache.kafka.common.errors.ApiException; -import org.apache.kafka.common.errors.CorruptRecordException; -import org.apache.kafka.common.errors.LeaderNotAvailableException; -import org.apache.kafka.common.errors.NetworkException; -import org.apache.kafka.common.errors.NotLeaderForPartitionException; -import org.apache.kafka.common.errors.OffsetMetadataTooLarge; -import org.apache.kafka.common.errors.OffsetOutOfRangeException; -import org.apache.kafka.common.errors.RecordTooLargeException; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.errors.UnknownServerException; -import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; - +import org.apache.kafka.common.errors.*; /** * This class contains all the client-server errors--those errors that must be sent from the server to the client. These @@ -42,28 +31,67 @@ public enum Errors { UNKNOWN(-1, new UnknownServerException("The server experienced an unexpected error when processing the request")), NONE(0, null), OFFSET_OUT_OF_RANGE(1, - new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")), + new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")), CORRUPT_MESSAGE(2, - new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")), - UNKNOWN_TOPIC_OR_PARTITION(3, new UnknownTopicOrPartitionException("This server does not host this topic-partition.")), + new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")), + UNKNOWN_TOPIC_OR_PARTITION(3, + new UnknownTopicOrPartitionException("This server does not host this topic-partition.")), + // TODO: errorCode 4 for InvalidFetchSize LEADER_NOT_AVAILABLE(5, - new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")), - NOT_LEADER_FOR_PARTITION(6, new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")), - REQUEST_TIMED_OUT(7, new TimeoutException("The request timed out.")), + new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")), + NOT_LEADER_FOR_PARTITION(6, + new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")), + REQUEST_TIMED_OUT(7, + new TimeoutException("The request timed out.")), + // TODO: errorCode 8 for BrokerNotAvailable + REPLICA_NOT_AVAILABLE(9, + new ApiException("The replica is not available for the requested topic-partition")), MESSAGE_TOO_LARGE(10, - new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")), - OFFSET_METADATA_TOO_LARGE(12, new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")), - NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")); + new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")), + OFFSET_METADATA_TOO_LARGE(12, + new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")), + NETWORK_EXCEPTION(13, + new NetworkException("The server disconnected before a response was received.")), + OFFSET_LOAD_IN_PROGRESS(14, + new OffsetLoadInProgressException("The coordinator is loading offsets and can't process requests.")), + CONSUMER_COORDINATOR_NOT_AVAILABLE(15, + new ConsumerCoordinatorNotAvailableException("The coordinator is not available.")), + NOT_COORDINATOR_FOR_CONSUMER(16, + new NotCoordinatorForConsumerException("This is not the correct coordinator for this consumer.")), + INVALID_TOPIC_EXCEPTION(17, + new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), + RECORD_LIST_TOO_LARGE(18, + new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")), + NOT_ENOUGH_REPLICAS(19, + new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")), + NOT_ENOUGH_REPLICAS_AFTER_APPEND(20, + new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")), + INVALID_REQUIRED_ACKS(21, + new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")), + ILLEGAL_GENERATION(22, + new IllegalGenerationException("Specified consumer generation id is not valid.")), + INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23, + new ApiException("The request partition assignment strategy does not match that of the group.")), + UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24, + new ApiException("The request partition assignment strategy is unknown to the broker.")), + UNKNOWN_CONSUMER_ID(25, + new UnknownConsumerIdException("The coordinator is not aware of this consumer.")), + INVALID_SESSION_TIMEOUT(26, + new ApiException("The session timeout is not within an acceptable range.")), + COMMITTING_PARTITIONS_NOT_ASSIGNED(27, + new ApiException("Some of the committing partitions are not assigned the committer")), + INVALID_COMMIT_OFFSET_SIZE(28, + new ApiException("The committing offset data size is not valid")); private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); + static { for (Errors error : Errors.values()) { codeToError.put(error.code(), error); if (error.exception != null) classToError.put(error.exception.getClass(), error); } - } private final short code; @@ -92,8 +120,9 @@ public short code() { * Throw the exception corresponding to this error if there is one */ public void maybeThrow() { - if (exception != null) + if (exception != null) { throw this.exception; + } } /** diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java b/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java index 90df5d5971a3f..c2cbbbd7f1f28 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java @@ -17,18 +17,10 @@ package org.apache.kafka.common.protocol; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; - public class ProtoUtils { private static Schema schemaFor(Schema[][] schemas, int apiKey, int version) { @@ -70,45 +62,4 @@ public static Struct parseResponse(int apiKey, ByteBuffer buffer) { return (Struct) currentResponseSchema(apiKey).read(buffer); } - public static Cluster parseMetadataResponse(Struct response) { - Map brokers = new HashMap(); - Object[] brokerStructs = (Object[]) response.get("brokers"); - for (int i = 0; i < brokerStructs.length; i++) { - Struct broker = (Struct) brokerStructs[i]; - int nodeId = (Integer) broker.get("node_id"); - String host = (String) broker.get("host"); - int port = (Integer) broker.get("port"); - brokers.put(nodeId, new Node(nodeId, host, port)); - } - List partitions = new ArrayList(); - Object[] topicInfos = (Object[]) response.get("topic_metadata"); - for (int i = 0; i < topicInfos.length; i++) { - Struct topicInfo = (Struct) topicInfos[i]; - short topicError = topicInfo.getShort("topic_error_code"); - if (topicError == Errors.NONE.code()) { - String topic = topicInfo.getString("topic"); - Object[] partitionInfos = (Object[]) topicInfo.get("partition_metadata"); - for (int j = 0; j < partitionInfos.length; j++) { - Struct partitionInfo = (Struct) partitionInfos[j]; - short partError = partitionInfo.getShort("partition_error_code"); - if (partError == Errors.NONE.code()) { - int partition = partitionInfo.getInt("partition_id"); - int leader = partitionInfo.getInt("leader"); - Node leaderNode = leader == -1 ? null : brokers.get(leader); - Object[] replicas = (Object[]) partitionInfo.get("replicas"); - Node[] replicaNodes = new Node[replicas.length]; - for (int k = 0; k < replicas.length; k++) - replicaNodes[k] = brokers.get(replicas[k]); - Object[] isr = (Object[]) partitionInfo.get("isr"); - Node[] isrNodes = new Node[isr.length]; - for (int k = 0; k < isr.length; k++) - isrNodes[k] = brokers.get(isr[k]); - partitions.add(new PartitionInfo(topic, partition, leaderNode, replicaNodes, isrNodes)); - } - } - } - } - return new Cluster(brokers.values(), partitions); - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 044b03061802e..3dc8b015afd23 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -28,70 +28,77 @@ public class Protocol { - public static Schema REQUEST_HEADER = new Schema(new Field("api_key", INT16, "The id of the request type."), - new Field("api_version", INT16, "The version of the API."), - new Field("correlation_id", - INT32, - "A user-supplied integer value that will be passed back with the response"), - new Field("client_id", - STRING, - "A user specified identifier for the client making the request.")); - - public static Schema RESPONSE_HEADER = new Schema(new Field("correlation_id", - INT32, - "The user-supplied value passed in with the request")); + public static final Schema REQUEST_HEADER = new Schema(new Field("api_key", INT16, "The id of the request type."), + new Field("api_version", INT16, "The version of the API."), + new Field("correlation_id", + INT32, + "A user-supplied integer value that will be passed back with the response"), + new Field("client_id", + STRING, + "A user specified identifier for the client making the request.")); + + public static final Schema RESPONSE_HEADER = new Schema(new Field("correlation_id", + INT32, + "The user-supplied value passed in with the request")); /* Metadata api */ - public static Schema METADATA_REQUEST_V0 = new Schema(new Field("topics", - new ArrayOf(STRING), - "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics.")); + public static final Schema METADATA_REQUEST_V0 = new Schema(new Field("topics", + new ArrayOf(STRING), + "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics.")); - public static Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."), - new Field("host", STRING, "The hostname of the broker."), - new Field("port", INT32, "The port on which the broker accepts requests.")); + public static final Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."), + new Field("host", STRING, "The hostname of the broker."), + new Field("port", + INT32, + "The port on which the broker accepts requests.")); - public static Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code", - INT16, - "The error code for the partition, if any."), - new Field("partition_id", INT32, "The id of the partition."), - new Field("leader", - INT32, - "The id of the broker acting as leader for this partition."), - new Field("replicas", - new ArrayOf(INT32), - "The set of all nodes that host this partition."), - new Field("isr", - new ArrayOf(INT32), - "The set of nodes that are in sync with the leader for this partition.")); - - public static Schema TOPIC_METADATA_V0 = new Schema(new Field("topic_error_code", INT16, "The error code for the given topic."), - new Field("topic", STRING, "The name of the topic"), - new Field("partition_metadata", - new ArrayOf(PARTITION_METADATA_V0), - "Metadata for each partition of the topic.")); - - public static Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers", - new ArrayOf(BROKER), - "Host and port information for all brokers."), - new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V0))); - - public static Schema[] METADATA_REQUEST = new Schema[] { METADATA_REQUEST_V0 }; - public static Schema[] METADATA_RESPONSE = new Schema[] { METADATA_RESPONSE_V0 }; + public static final Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code", + INT16, + "The error code for the partition, if any."), + new Field("partition_id", + INT32, + "The id of the partition."), + new Field("leader", + INT32, + "The id of the broker acting as leader for this partition."), + new Field("replicas", + new ArrayOf(INT32), + "The set of all nodes that host this partition."), + new Field("isr", + new ArrayOf(INT32), + "The set of nodes that are in sync with the leader for this partition.")); + + public static final Schema TOPIC_METADATA_V0 = new Schema(new Field("topic_error_code", + INT16, + "The error code for the given topic."), + new Field("topic", STRING, "The name of the topic"), + new Field("partition_metadata", + new ArrayOf(PARTITION_METADATA_V0), + "Metadata for each partition of the topic.")); + + public static final Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers", + new ArrayOf(BROKER), + "Host and port information for all brokers."), + new Field("topic_metadata", + new ArrayOf(TOPIC_METADATA_V0))); + + public static final Schema[] METADATA_REQUEST = new Schema[] {METADATA_REQUEST_V0}; + public static final Schema[] METADATA_RESPONSE = new Schema[] {METADATA_RESPONSE_V0}; /* Produce api */ - public static Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING), - new Field("data", new ArrayOf(new Schema(new Field("partition", INT32), + public static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING), + new Field("data", new ArrayOf(new Schema(new Field("partition", INT32), new Field("record_set", BYTES))))); - public static Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks", + public static final Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks", INT16, "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."), - new Field("timeout", INT32, "The time to await a response in ms."), - new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0))); + new Field("timeout", INT32, "The time to await a response in ms."), + new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0))); - public static Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses", + public static final Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses", new ArrayOf(new Schema(new Field("topic", STRING), new Field("partition_responses", new ArrayOf(new Schema(new Field("partition", @@ -101,34 +108,353 @@ public class Protocol { new Field("base_offset", INT64)))))))); - public static Schema[] PRODUCE_REQUEST = new Schema[] { PRODUCE_REQUEST_V0 }; - public static Schema[] PRODUCE_RESPONSE = new Schema[] { PRODUCE_RESPONSE_V0 }; + public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0}; + public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0}; + + /* Offset commit api */ + public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Message offset to be committed."), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep.")); + + public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Message offset to be committed."), + new Field("timestamp", + INT64, + "Timestamp of the commit"), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep.")); + + public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V2 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Message offset to be committed."), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep.")); + + public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + STRING, + "Topic to commit."), + new Field("partitions", + new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), + "Partitions to commit offsets.")); + + public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(new Field("topic", + STRING, + "Topic to commit."), + new Field("partitions", + new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1), + "Partitions to commit offsets.")); + + public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V2 = new Schema(new Field("topic", + STRING, + "Topic to commit."), + new Field("partitions", + new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V2), + "Partitions to commit offsets.")); + + public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), + "Topics to commit offsets.")); + + public static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1), + "Topics to commit offsets.")); + + public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator."), + new Field("retention_time", + INT64, + "Time period in ms to retain the offset."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2), + "Topics to commit offsets.")); + + public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("error_code", + INT16)); + + public static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0))); + + public static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); + + public static final Schema[] OFFSET_COMMIT_REQUEST = new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2}; + + /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */ + public static final Schema OFFSET_COMMIT_RESPONSE_V1 = OFFSET_COMMIT_RESPONSE_V0; + public static final Schema OFFSET_COMMIT_RESPONSE_V2 = OFFSET_COMMIT_RESPONSE_V0; + + public static final Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V1, OFFSET_COMMIT_RESPONSE_V2}; + + /* Offset fetch api */ + + /* + * Wire formats of version 0 and 1 are the same, but with different functionality. + * Version 0 will read the offsets from ZK; + * Version 1 will read the offsets from Kafka. + */ + public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id.")); + + public static final Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + STRING, + "Topic to fetch offset."), + new Field("partitions", + new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0), + "Partitions to fetch offsets.")); + + public static final Schema OFFSET_FETCH_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("topics", + new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0), + "Topics to fetch offsets.")); + + public static final Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Last committed message offset."), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep."), + new Field("error_code", INT16)); + + public static final Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0))); + + public static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0))); + + public static final Schema OFFSET_FETCH_REQUEST_V1 = OFFSET_FETCH_REQUEST_V0; + public static final Schema OFFSET_FETCH_RESPONSE_V1 = OFFSET_FETCH_RESPONSE_V0; + + public static final Schema[] OFFSET_FETCH_REQUEST = new Schema[] {OFFSET_FETCH_REQUEST_V0, OFFSET_FETCH_REQUEST_V1}; + public static final Schema[] OFFSET_FETCH_RESPONSE = new Schema[] {OFFSET_FETCH_RESPONSE_V0, OFFSET_FETCH_RESPONSE_V1}; + + /* List offset api */ + public static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("timestamp", INT64, "Timestamp."), + new Field("max_num_offsets", + INT32, + "Maximum offsets to return.")); + + public static final Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + STRING, + "Topic to list offset."), + new Field("partitions", + new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0), + "Partitions to list offset.")); + + public static final Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id", + INT32, + "Broker id of the follower. For normal consumers, use -1."), + new Field("topics", + new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0), + "Topics to list offsets.")); + + public static final Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("error_code", INT16), + new Field("offsets", + new ArrayOf(INT64), + "A list of offsets.")); + + public static final Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0))); + + public static final Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0))); + + public static final Schema[] LIST_OFFSET_REQUEST = new Schema[] {LIST_OFFSET_REQUEST_V0}; + public static final Schema[] LIST_OFFSET_RESPONSE = new Schema[] {LIST_OFFSET_RESPONSE_V0}; + + /* Fetch api */ + public static final Schema FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("fetch_offset", + INT64, + "Message offset."), + new Field("max_bytes", + INT32, + "Maximum bytes to fetch.")); + + public static final Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", STRING, "Topic to fetch."), + new Field("partitions", + new ArrayOf(FETCH_REQUEST_PARTITION_V0), + "Partitions to fetch.")); + + public static final Schema FETCH_REQUEST_V0 = new Schema(new Field("replica_id", + INT32, + "Broker id of the follower. For normal consumers, use -1."), + new Field("max_wait_time", + INT32, + "Maximum time in ms to wait for the response."), + new Field("min_bytes", + INT32, + "Minimum bytes to accumulate in the response."), + new Field("topics", + new ArrayOf(FETCH_REQUEST_TOPIC_V0), + "Topics to fetch.")); + + public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("error_code", INT16), + new Field("high_watermark", + INT64, + "Last committed offset."), + new Field("record_set", BYTES)); + + public static final Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(FETCH_RESPONSE_PARTITION_V0))); + + public static final Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); + + public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0}; + public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0}; + + /* Consumer metadata api */ + public static final Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id.")); + + public static final Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", INT16), + new Field("coordinator", + BROKER, + "Host and port information for the coordinator for a consumer group.")); + + public static final Schema[] CONSUMER_METADATA_REQUEST = new Schema[] {CONSUMER_METADATA_REQUEST_V0}; + public static final Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] {CONSUMER_METADATA_RESPONSE_V0}; + + /* Join group api */ + public static final Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("session_timeout", + INT32, + "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."), + new Field("topics", + new ArrayOf(STRING), + "An array of topics to subscribe to."), + new Field("consumer_id", + STRING, + "The assigned consumer id or an empty string for a new consumer."), + new Field("partition_assignment_strategy", + STRING, + "The strategy for the coordinator to assign partitions.")); + + public static final Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partitions", new ArrayOf(INT32))); + public static final Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", INT16), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator."), + new Field("assigned_partitions", + new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0))); + + public static final Schema[] JOIN_GROUP_REQUEST = new Schema[] {JOIN_GROUP_REQUEST_V0}; + public static final Schema[] JOIN_GROUP_RESPONSE = new Schema[] {JOIN_GROUP_RESPONSE_V0}; + + /* Heartbeat api */ + public static final Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id", STRING, "The consumer group id."), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator.")); + + public static final Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", INT16)); + + public static final Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0}; + public static final Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0}; /* an array of all requests and responses with all schema versions */ - public static Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][]; - public static Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][]; + public static final Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][]; + public static final Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][]; /* the latest version of each api */ - public static short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1]; + public static final short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1]; static { REQUESTS[ApiKeys.PRODUCE.id] = PRODUCE_REQUEST; - REQUESTS[ApiKeys.FETCH.id] = new Schema[] {}; - REQUESTS[ApiKeys.LIST_OFFSETS.id] = new Schema[] {}; + REQUESTS[ApiKeys.FETCH.id] = FETCH_REQUEST; + REQUESTS[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_REQUEST; REQUESTS[ApiKeys.METADATA.id] = METADATA_REQUEST; REQUESTS[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {}; REQUESTS[ApiKeys.STOP_REPLICA.id] = new Schema[] {}; - REQUESTS[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {}; - REQUESTS[ApiKeys.OFFSET_FETCH.id] = new Schema[] {}; + REQUESTS[ApiKeys.UPDATE_METADATA_KEY.id] = new Schema[] {}; + REQUESTS[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = new Schema[] {}; + REQUESTS[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_REQUEST; + REQUESTS[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_REQUEST; + REQUESTS[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_REQUEST; + REQUESTS[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_REQUEST; + REQUESTS[ApiKeys.HEARTBEAT.id] = HEARTBEAT_REQUEST; RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE; - RESPONSES[ApiKeys.FETCH.id] = new Schema[] {}; - RESPONSES[ApiKeys.LIST_OFFSETS.id] = new Schema[] {}; + RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE; + RESPONSES[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_RESPONSE; RESPONSES[ApiKeys.METADATA.id] = METADATA_RESPONSE; RESPONSES[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {}; RESPONSES[ApiKeys.STOP_REPLICA.id] = new Schema[] {}; - RESPONSES[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {}; - RESPONSES[ApiKeys.OFFSET_FETCH.id] = new Schema[] {}; + RESPONSES[ApiKeys.UPDATE_METADATA_KEY.id] = new Schema[] {}; + RESPONSES[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = new Schema[] {}; + RESPONSES[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_RESPONSE; + RESPONSES[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_RESPONSE; + RESPONSES[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_RESPONSE; + RESPONSES[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_RESPONSE; + RESPONSES[ApiKeys.HEARTBEAT.id] = HEARTBEAT_RESPONSE; /* set the maximum version of each api */ for (ApiKeys api : ApiKeys.values()) @@ -137,11 +463,8 @@ public class Protocol { /* sanity check that we have the same number of request and response versions for each api */ for (ApiKeys api : ApiKeys.values()) if (REQUESTS[api.id].length != RESPONSES[api.id].length) - throw new IllegalStateException(REQUESTS[api.id].length + " request versions for api " - + api.name - + " but " - + RESPONSES[api.id].length - + " response versions."); + throw new IllegalStateException(REQUESTS[api.id].length + " request versions for api " + api.name + + " but " + RESPONSES[api.id].length + " response versions."); } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java new file mode 100644 index 0000000000000..dab1a94dd2956 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java @@ -0,0 +1,63 @@ +/** + * 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 org.apache.kafka.common.protocol; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public enum SecurityProtocol { + /** Un-authenticated, non-encrypted channel */ + PLAINTEXT(0, "PLAINTEXT"), + /** Currently identical to PLAINTEXT and used for testing only. We may implement extra instrumentation when testing channel code. */ + TRACE(Short.MAX_VALUE, "TRACE"); + + private static final Map CODE_TO_SECURITY_PROTOCOL = new HashMap(); + private static final List NAMES = new ArrayList(); + + static { + for (SecurityProtocol proto: SecurityProtocol.values()) { + CODE_TO_SECURITY_PROTOCOL.put(proto.id, proto); + NAMES.add(proto.name); + } + } + + /** The permanent and immutable id of a security protocol -- this can't change, and must match kafka.cluster.SecurityProtocol */ + public final short id; + + /** Name of the security protocol. This may be used by client configuration. */ + public final String name; + + private SecurityProtocol(int id, String name) { + this.id = (short) id; + this.name = name; + } + + public static String getName(int id) { + return CODE_TO_SECURITY_PROTOCOL.get((short) id).name; + } + + public static List getNames() { + return NAMES; + } + + public static SecurityProtocol forId(Short id) { + return CODE_TO_SECURITY_PROTOCOL.get(id); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java index 899195819159a..1eb119589c3d9 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java @@ -30,6 +30,11 @@ public class Field { public final String doc; final Schema schema; + /** + * Create the field. + * + * @throws SchemaException If the default value is not primitive and the validation fails + */ public Field(int index, String name, Type type, String doc, Object defaultValue, Schema schema) { this.index = index; this.name = name; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java index c10d9ef28db01..3a14ac0fb350a 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.protocol.types; @@ -28,6 +24,11 @@ public class Schema extends Type { private final Field[] fields; private final Map fieldsByName; + /** + * Construct the schema with a given list of its field values + * + * @throws SchemaException If the given list have duplicate fields + */ public Schema(Field... fs) { this.fields = new Field[fs.length]; this.fieldsByName = new HashMap(); @@ -51,8 +52,9 @@ public void write(ByteBuffer buffer, Object o) { Object value = f.type().validate(r.get(f)); f.type.write(buffer, value); } catch (Exception e) { - throw new SchemaException("Error writing field '" + f.name + "': " + e.getMessage() == null ? e.getMessage() : e.getClass() - .getName()); + throw new SchemaException("Error writing field '" + f.name + + "': " + + (e.getMessage() == null ? e.getClass().getName() : e.getMessage())); } } } @@ -62,8 +64,15 @@ public void write(ByteBuffer buffer, Object o) { */ public Object read(ByteBuffer buffer) { Object[] objects = new Object[fields.length]; - for (int i = 0; i < fields.length; i++) - objects[i] = fields[i].type.read(buffer); + for (int i = 0; i < fields.length; i++) { + try { + objects[i] = fields[i].type.read(buffer); + } catch (Exception e) { + throw new SchemaException("Error reading field '" + fields[i].name + + "': " + + (e.getMessage() == null ? e.getClass().getName() : e.getMessage())); + } + } return new Struct(this, objects); } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java index ea4e46f4d768e..58b685bd61bc7 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java @@ -18,6 +18,9 @@ import org.apache.kafka.common.KafkaException; +/** + * Thrown if the protocol schema validation fails while parsing request or response. + */ public class SchemaException extends KafkaException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index 73ad6cdb465ed..92de6a9624be0 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.protocol.types; @@ -56,7 +52,7 @@ private Object getFieldOrDefault(Field field) { else if (field.defaultValue != Field.NO_DEFAULT) return field.defaultValue; else - throw new SchemaException("Missing value for field '" + field.name + " which has no default value."); + throw new SchemaException("Missing value for field '" + field.name + "' which has no default value."); } /** @@ -64,6 +60,7 @@ else if (field.defaultValue != Field.NO_DEFAULT) * * @param field The field to look up * @return The value for that field. + * @throws SchemaException if the field has no value and has no default. */ public Object get(Field field) { validateField(field); @@ -75,6 +72,7 @@ public Object get(Field field) { * * @param name The name of the field * @return The value in the record + * @throws SchemaException If no such field exists */ public Object get(String name) { Field field = schema.get(name); @@ -83,6 +81,15 @@ public Object get(String name) { return getFieldOrDefault(field); } + /** + * Check if the struct contains a field. + * @param name + * @return Whether a field exists. + */ + public boolean hasField(String name) { + return schema.get(name) != null; + } + public Struct getStruct(Field field) { return (Struct) get(field); } @@ -107,6 +114,14 @@ public Integer getInt(String name) { return (Integer) get(name); } + public Long getLong(Field field) { + return (Long) get(field); + } + + public Long getLong(String name) { + return (Long) get(name); + } + public Object[] getArray(Field field) { return (Object[]) get(field); } @@ -123,11 +138,20 @@ public String getString(String name) { return (String) get(name); } + public ByteBuffer getBytes(Field field) { + return (ByteBuffer) get(field); + } + + public ByteBuffer getBytes(String name) { + return (ByteBuffer) get(name); + } + /** * Set the given field to the specified value * * @param field The field * @param value The value + * @throws SchemaException If the validation of the field failed */ public Struct set(Field field, Object value) { validateField(field); @@ -140,6 +164,7 @@ public Struct set(Field field, Object value) { * * @param name The name of the field * @param value The value to set + * @throws SchemaException If the field is not known */ public Struct set(String name, Object value) { Field field = this.schema.get(name); @@ -150,10 +175,13 @@ public Struct set(String name, Object value) { } /** - * Create a struct for the schema of a container type (struct or array) + * Create a struct for the schema of a container type (struct or array). Note that for array type, this method + * assumes that the type is an array of schema and creates a struct of that schema. Arrays of other types can't be + * instantiated with this method. * * @param field The field to create an instance of * @return The struct + * @throws SchemaException If the given field is not a container type */ public Struct instance(Field field) { validateField(field); @@ -163,7 +191,7 @@ public Struct instance(Field field) { ArrayOf array = (ArrayOf) field.type(); return new Struct((Schema) array.type()); } else { - throw new SchemaException("Field " + field.name + " is not a container type, it is of type " + field.type()); + throw new SchemaException("Field '" + field.name + "' is not a container type, it is of type " + field.type()); } } @@ -172,6 +200,7 @@ public Struct instance(Field field) { * * @param field The name of the field to create (field must be a schema type) * @return The struct + * @throws SchemaException If the given field is not a container type */ public Struct instance(String field) { return instance(schema.get(field)); @@ -200,16 +229,20 @@ public void writeTo(ByteBuffer buffer) { /** * Ensure the user doesn't try to access fields from the wrong schema + * + * @throws SchemaException If validation fails */ private void validateField(Field field) { if (this.schema != field.schema) - throw new SchemaException("Attempt to access field '" + field.name + " from a different schema instance."); + throw new SchemaException("Attempt to access field '" + field.name + "' from a different schema instance."); if (field.index > values.length) throw new SchemaException("Invalid field index: " + field.index); } /** * Validate the contents of this struct against its schema + * + * @throws SchemaException If validation fails */ public void validate() { this.schema.validate(this); @@ -222,7 +255,7 @@ public void validate() { public ByteBuffer[] toBytes() { ByteBuffer buffer = ByteBuffer.allocate(sizeOf()); writeTo(buffer); - return new ByteBuffer[] { buffer }; + return new ByteBuffer[] {buffer}; } @Override @@ -230,9 +263,20 @@ public String toString() { StringBuilder b = new StringBuilder(); b.append('{'); for (int i = 0; i < this.values.length; i++) { - b.append(this.schema.get(i).name); + Field f = this.schema.get(i); + b.append(f.name); b.append('='); - b.append(this.values[i]); + if (f.type() instanceof ArrayOf) { + Object[] arrayValue = (Object[]) this.values[i]; + b.append('['); + for (int j = 0; j < arrayValue.length; j++) { + b.append(arrayValue[j]); + if (j < arrayValue.length - 1) + b.append(','); + } + b.append(']'); + } else + b.append(this.values[i]); if (i < this.values.length - 1) b.append(','); } @@ -240,4 +284,46 @@ public String toString() { return b.toString(); } + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + for (int i = 0; i < this.values.length; i++) { + Field f = this.schema.get(i); + if (f.type() instanceof ArrayOf) { + Object[] arrayObject = (Object []) this.get(f); + for (Object arrayItem: arrayObject) + result = prime * result + arrayItem.hashCode(); + } else { + result = prime * result + this.get(f).hashCode(); + } + } + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Struct other = (Struct) obj; + if (schema != other.schema) + return false; + for (int i = 0; i < this.values.length; i++) { + Field f = this.schema.get(i); + Boolean result; + if (f.type() instanceof ArrayOf) { + result = Arrays.equals((Object []) this.get(f), (Object []) other.get(f)); + } else { + result = this.get(f).equals(other.get(f)); + } + if (!result) + return false; + } + return true; + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java index 1bd8ce4dfb7c5..9ea28b27c347d 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java @@ -20,20 +20,37 @@ import org.apache.kafka.common.utils.Utils; - /** * A serializable type */ public abstract class Type { + /** + * Write the typed object to the buffer + * + * @throws SchemaException If the object is not valid for its type + */ public abstract void write(ByteBuffer buffer, Object o); + /** + * Read the typed object from the buffer + * + * @throws SchemaException If the object is not valid for its type + */ public abstract Object read(ByteBuffer buffer); - public abstract int sizeOf(Object o); - + /** + * Validate the object. If succeeded return its typed object. + * + * @throws SchemaException If validation failed + */ public abstract Object validate(Object o); + /** + * Return the size of the object in bytes + */ + public abstract int sizeOf(Object o); + public static final Type INT8 = new Type() { @Override public void write(ByteBuffer buffer, Object o) { diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java new file mode 100644 index 0000000000000..12651d46e7887 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java @@ -0,0 +1,49 @@ +/** + * 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 org.apache.kafka.common.record; + +import java.io.InputStream; +import java.nio.ByteBuffer; + +/** + * A byte buffer backed input outputStream + */ +public class ByteBufferInputStream extends InputStream { + + private ByteBuffer buffer; + + public ByteBufferInputStream(ByteBuffer buffer) { + this.buffer = buffer; + } + + public int read() { + if (!buffer.hasRemaining()) { + return -1; + } + return buffer.get() & 0xFF; + } + + public int read(byte[] bytes, int off, int len) { + if (!buffer.hasRemaining()) { + return -1; + } + + len = Math.min(len, buffer.remaining()); + buffer.get(bytes, off, len); + return len; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java new file mode 100644 index 0000000000000..1c9fbaa958423 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java @@ -0,0 +1,57 @@ +/** + * 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 org.apache.kafka.common.record; + +import java.io.OutputStream; +import java.nio.ByteBuffer; + +/** + * A byte buffer backed output outputStream + */ +public class ByteBufferOutputStream extends OutputStream { + + private static final float REALLOCATION_FACTOR = 1.1f; + + private ByteBuffer buffer; + + public ByteBufferOutputStream(ByteBuffer buffer) { + this.buffer = buffer; + } + + public void write(int b) { + if (buffer.remaining() < 1) + expandBuffer(buffer.capacity() + 1); + buffer.put((byte) b); + } + + public void write(byte[] bytes, int off, int len) { + if (buffer.remaining() < len) + expandBuffer(buffer.capacity() + len); + buffer.put(bytes, off, len); + } + + public ByteBuffer buffer() { + return buffer; + } + + private void expandBuffer(int size) { + int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size); + ByteBuffer temp = ByteBuffer.allocate(expandSize); + temp.put(buffer.array(), buffer.arrayOffset(), buffer.position()); + buffer = temp; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java index 906da02d02c03..65a7e4323793d 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java @@ -20,14 +20,16 @@ * The compression type to use */ public enum CompressionType { - NONE(0, "none"), GZIP(1, "gzip"), SNAPPY(2, "snappy"); + NONE(0, "none", 1.0f), GZIP(1, "gzip", 0.5f), SNAPPY(2, "snappy", 0.5f), LZ4(3, "lz4", 0.5f); public final int id; public final String name; + public final float rate; - private CompressionType(int id, String name) { + private CompressionType(int id, String name, float rate) { this.id = id; this.name = name; + this.rate = rate; } public static CompressionType forId(int id) { @@ -38,6 +40,8 @@ public static CompressionType forId(int id) { return GZIP; case 2: return SNAPPY; + case 3: + return LZ4; default: throw new IllegalArgumentException("Unknown compression type id: " + id); } @@ -50,7 +54,10 @@ else if (GZIP.name.equals(name)) return GZIP; else if (SNAPPY.name.equals(name)) return SNAPPY; + else if (LZ4.name.equals(name)) + return LZ4; else throw new IllegalArgumentException("Unknown compression name: " + name); } + } diff --git a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java new file mode 100644 index 0000000000000..e570b29d5ffba --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java @@ -0,0 +1,270 @@ +/** + * 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 org.apache.kafka.common.record; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.Utils; + +import java.io.InputStream; +import java.io.OutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; + +public class Compressor { + + static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f; + static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f; + static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024; + + private static final float[] TYPE_TO_RATE; + + static { + int maxTypeId = -1; + for (CompressionType type : CompressionType.values()) + maxTypeId = Math.max(maxTypeId, type.id); + TYPE_TO_RATE = new float[maxTypeId + 1]; + for (CompressionType type : CompressionType.values()) { + TYPE_TO_RATE[type.id] = type.rate; + } + } + + private final CompressionType type; + private final DataOutputStream appendStream; + private final ByteBufferOutputStream bufferStream; + private final int initPos; + + public long writtenUncompressed; + public long numRecords; + + public Compressor(ByteBuffer buffer, CompressionType type, int blockSize) { + this.type = type; + this.initPos = buffer.position(); + + this.numRecords = 0; + this.writtenUncompressed = 0; + + if (type != CompressionType.NONE) { + // for compressed records, leave space for the header and the shallow message metadata + // and move the starting position to the value payload offset + buffer.position(initPos + Records.LOG_OVERHEAD + Record.RECORD_OVERHEAD); + } + + // create the stream + bufferStream = new ByteBufferOutputStream(buffer); + appendStream = wrapForOutput(bufferStream, type, blockSize); + } + + public Compressor(ByteBuffer buffer, CompressionType type) { + this(buffer, type, COMPRESSION_DEFAULT_BUFFER_SIZE); + } + + public ByteBuffer buffer() { + return bufferStream.buffer(); + } + + public double compressionRate() { + ByteBuffer buffer = bufferStream.buffer(); + if (this.writtenUncompressed == 0) + return 1.0; + else + return (double) buffer.position() / this.writtenUncompressed; + } + + public void close() { + try { + appendStream.close(); + } catch (IOException e) { + throw new KafkaException(e); + } + + if (type != CompressionType.NONE) { + ByteBuffer buffer = bufferStream.buffer(); + int pos = buffer.position(); + // write the header, for the end offset write as number of records - 1 + buffer.position(initPos); + buffer.putLong(numRecords - 1); + buffer.putInt(pos - initPos - Records.LOG_OVERHEAD); + // write the shallow message (the crc and value size are not correct yet) + Record.write(buffer, null, null, type, 0, -1); + // compute the fill the value size + int valueSize = pos - initPos - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD; + buffer.putInt(initPos + Records.LOG_OVERHEAD + Record.KEY_OFFSET, valueSize); + // compute and fill the crc at the beginning of the message + long crc = Record.computeChecksum(buffer, + initPos + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET, + pos - initPos - Records.LOG_OVERHEAD - Record.MAGIC_OFFSET); + Utils.writeUnsignedInt(buffer, initPos + Records.LOG_OVERHEAD + Record.CRC_OFFSET, crc); + // reset the position + buffer.position(pos); + + // update the compression ratio + float compressionRate = (float) buffer.position() / this.writtenUncompressed; + TYPE_TO_RATE[type.id] = TYPE_TO_RATE[type.id] * COMPRESSION_RATE_DAMPING_FACTOR + + compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR); + } + } + + // Note that for all the write operations below, IO exceptions should + // never be thrown since the underlying ByteBufferOutputStream does not throw IOException; + // therefore upon encountering this issue we just close the append stream. + + public void putLong(final long value) { + try { + appendStream.writeLong(value); + } catch (IOException e) { + throw new KafkaException("I/O exception when writing to the append stream, closing", e); + } + } + + public void putInt(final int value) { + try { + appendStream.writeInt(value); + } catch (IOException e) { + throw new KafkaException("I/O exception when writing to the append stream, closing", e); + } + } + + public void put(final ByteBuffer buffer) { + try { + appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit()); + } catch (IOException e) { + throw new KafkaException("I/O exception when writing to the append stream, closing", e); + } + } + + public void putByte(final byte value) { + try { + appendStream.write(value); + } catch (IOException e) { + throw new KafkaException("I/O exception when writing to the append stream, closing", e); + } + } + + public void put(final byte[] bytes, final int offset, final int len) { + try { + appendStream.write(bytes, offset, len); + } catch (IOException e) { + throw new KafkaException("I/O exception when writing to the append stream, closing", e); + } + } + + public void putRecord(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) { + // put a record as un-compressed into the underlying stream + long crc = Record.computeChecksum(key, value, type, valueOffset, valueSize); + byte attributes = Record.computeAttributes(type); + putRecord(crc, attributes, key, value, valueOffset, valueSize); + } + + public void putRecord(byte[] key, byte[] value) { + putRecord(key, value, CompressionType.NONE, 0, -1); + } + + private void putRecord(final long crc, final byte attributes, final byte[] key, final byte[] value, final int valueOffset, final int valueSize) { + Record.write(this, crc, attributes, key, value, valueOffset, valueSize); + } + + public void recordWritten(int size) { + numRecords += 1; + writtenUncompressed += size; + } + + public long estimatedBytesWritten() { + if (type == CompressionType.NONE) { + return bufferStream.buffer().position(); + } else { + // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes + return (long) (writtenUncompressed * TYPE_TO_RATE[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR); + } + } + + // the following two functions also need to be public since they are used in MemoryRecords.iteration + + static public DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, int bufferSize) { + try { + switch (type) { + case NONE: + return new DataOutputStream(buffer); + case GZIP: + return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize)); + case SNAPPY: + // dynamically load the snappy class to avoid runtime dependency + // on snappy if we are not using it + try { + Class outputStreamClass = Class.forName("org.xerial.snappy.SnappyOutputStream"); + OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class, Integer.TYPE) + .newInstance(buffer, bufferSize); + return new DataOutputStream(stream); + } catch (Exception e) { + throw new KafkaException(e); + } + case LZ4: + try { + Class outputStreamClass = Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream"); + OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class) + .newInstance(buffer); + return new DataOutputStream(stream); + } catch (Exception e) { + throw new KafkaException(e); + } + default: + throw new IllegalArgumentException("Unknown compression type: " + type); + } + } catch (IOException e) { + throw new KafkaException(e); + } + } + + static public DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type) { + try { + switch (type) { + case NONE: + return new DataInputStream(buffer); + case GZIP: + return new DataInputStream(new GZIPInputStream(buffer)); + case SNAPPY: + // dynamically load the snappy class to avoid runtime dependency + // on snappy if we are not using it + try { + Class inputStreamClass = Class.forName("org.xerial.snappy.SnappyInputStream"); + InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class) + .newInstance(buffer); + return new DataInputStream(stream); + } catch (Exception e) { + throw new KafkaException(e); + } + case LZ4: + // dynamically load LZ4 class to avoid runtime dependency + try { + Class inputStreamClass = Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream"); + InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class) + .newInstance(buffer); + return new DataInputStream(stream); + } catch (Exception e) { + throw new KafkaException(e); + } + default: + throw new IllegalArgumentException("Unknown compression type: " + type); + } + } catch (IOException e) { + throw new KafkaException(e); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java new file mode 100644 index 0000000000000..f480da2ae0992 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java @@ -0,0 +1,234 @@ +/** + * 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 org.apache.kafka.common.record; + +import static org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK; +import static org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.LZ4_MAX_HEADER_LENGTH; +import static org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.MAGIC; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; + +import org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.BD; +import org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.FLG; +import org.apache.kafka.common.utils.Utils; + +import net.jpountz.lz4.LZ4Exception; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4SafeDecompressor; +import net.jpountz.xxhash.XXHash32; +import net.jpountz.xxhash.XXHashFactory; + +/** + * A partial implementation of the v1.4.1 LZ4 Frame format. + * + * @see LZ4 Framing + * Format Spec + */ +public final class KafkaLZ4BlockInputStream extends FilterInputStream { + + public static final String PREMATURE_EOS = "Stream ended prematurely"; + public static final String NOT_SUPPORTED = "Stream unsupported"; + public static final String BLOCK_HASH_MISMATCH = "Block checksum mismatch"; + public static final String DESCRIPTOR_HASH_MISMATCH = "Stream frame descriptor corrupted"; + + private final LZ4SafeDecompressor decompressor; + private final XXHash32 checksum; + private final byte[] buffer; + private final byte[] compressedBuffer; + private final int maxBlockSize; + private FLG flg; + private BD bd; + private int bufferOffset; + private int bufferSize; + private boolean finished; + + /** + * Create a new {@link InputStream} that will decompress data using the LZ4 algorithm. + * + * @param in The stream to decompress + * @throws IOException + */ + public KafkaLZ4BlockInputStream(InputStream in) throws IOException { + super(in); + decompressor = LZ4Factory.fastestInstance().safeDecompressor(); + checksum = XXHashFactory.fastestInstance().hash32(); + readHeader(); + maxBlockSize = bd.getBlockMaximumSize(); + buffer = new byte[maxBlockSize]; + compressedBuffer = new byte[maxBlockSize]; + bufferOffset = 0; + bufferSize = 0; + finished = false; + } + + /** + * Reads the magic number and frame descriptor from the underlying {@link InputStream}. + * + * @throws IOException + */ + private void readHeader() throws IOException { + byte[] header = new byte[LZ4_MAX_HEADER_LENGTH]; + + // read first 6 bytes into buffer to check magic and FLG/BD descriptor flags + bufferOffset = 6; + if (in.read(header, 0, bufferOffset) != bufferOffset) { + throw new IOException(PREMATURE_EOS); + } + + if (MAGIC != Utils.readUnsignedIntLE(header, bufferOffset - 6)) { + throw new IOException(NOT_SUPPORTED); + } + flg = FLG.fromByte(header[bufferOffset - 2]); + bd = BD.fromByte(header[bufferOffset - 1]); + // TODO read uncompressed content size, update flg.validate() + // TODO read dictionary id, update flg.validate() + + // check stream descriptor hash + byte hash = (byte) ((checksum.hash(header, 0, bufferOffset, 0) >> 8) & 0xFF); + header[bufferOffset++] = (byte) in.read(); + if (hash != header[bufferOffset - 1]) { + throw new IOException(DESCRIPTOR_HASH_MISMATCH); + } + } + + /** + * Decompresses (if necessary) buffered data, optionally computes and validates a XXHash32 checksum, and writes the + * result to a buffer. + * + * @throws IOException + */ + private void readBlock() throws IOException { + int blockSize = Utils.readUnsignedIntLE(in); + + // Check for EndMark + if (blockSize == 0) { + finished = true; + // TODO implement content checksum, update flg.validate() + return; + } else if (blockSize > maxBlockSize) { + throw new IOException(String.format("Block size %s exceeded max: %s", blockSize, maxBlockSize)); + } + + boolean compressed = (blockSize & LZ4_FRAME_INCOMPRESSIBLE_MASK) == 0; + byte[] bufferToRead; + if (compressed) { + bufferToRead = compressedBuffer; + } else { + blockSize &= ~LZ4_FRAME_INCOMPRESSIBLE_MASK; + bufferToRead = buffer; + bufferSize = blockSize; + } + + if (in.read(bufferToRead, 0, blockSize) != blockSize) { + throw new IOException(PREMATURE_EOS); + } + + // verify checksum + if (flg.isBlockChecksumSet() && Utils.readUnsignedIntLE(in) != checksum.hash(bufferToRead, 0, blockSize, 0)) { + throw new IOException(BLOCK_HASH_MISMATCH); + } + + if (compressed) { + try { + bufferSize = decompressor.decompress(compressedBuffer, 0, blockSize, buffer, 0, maxBlockSize); + } catch (LZ4Exception e) { + throw new IOException(e); + } + } + + bufferOffset = 0; + } + + @Override + public int read() throws IOException { + if (finished) { + return -1; + } + if (available() == 0) { + readBlock(); + } + if (finished) { + return -1; + } + int value = buffer[bufferOffset++] & 0xFF; + + return value; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + net.jpountz.util.Utils.checkRange(b, off, len); + if (finished) { + return -1; + } + if (available() == 0) { + readBlock(); + } + if (finished) { + return -1; + } + len = Math.min(len, available()); + System.arraycopy(buffer, bufferOffset, b, off, len); + bufferOffset += len; + return len; + } + + @Override + public long skip(long n) throws IOException { + if (finished) { + return 0; + } + if (available() == 0) { + readBlock(); + } + if (finished) { + return 0; + } + n = Math.min(n, available()); + bufferOffset += n; + return n; + } + + @Override + public int available() throws IOException { + return bufferSize - bufferOffset; + } + + @Override + public void close() throws IOException { + in.close(); + } + + @Override + public synchronized void mark(int readlimit) { + throw new RuntimeException("mark not supported"); + } + + @Override + public synchronized void reset() throws IOException { + throw new RuntimeException("reset not supported"); + } + + @Override + public boolean markSupported() { + return false; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java new file mode 100644 index 0000000000000..6a2231f477577 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java @@ -0,0 +1,392 @@ +/** + * 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 org.apache.kafka.common.record; + +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.kafka.common.utils.Utils; + +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.xxhash.XXHash32; +import net.jpountz.xxhash.XXHashFactory; + +/** + * A partial implementation of the v1.4.1 LZ4 Frame format. + * + * @see LZ4 Framing + * Format Spec + */ +public final class KafkaLZ4BlockOutputStream extends FilterOutputStream { + + public static final int MAGIC = 0x184D2204; + public static final int LZ4_MAX_HEADER_LENGTH = 19; + public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000; + + public static final String CLOSED_STREAM = "The stream is already closed"; + + public static final int BLOCKSIZE_64KB = 4; + public static final int BLOCKSIZE_256KB = 5; + public static final int BLOCKSIZE_1MB = 6; + public static final int BLOCKSIZE_4MB = 7; + + private final LZ4Compressor compressor; + private final XXHash32 checksum; + private final FLG flg; + private final BD bd; + private final byte[] buffer; + private final byte[] compressedBuffer; + private final int maxBlockSize; + private int bufferOffset; + private boolean finished; + + /** + * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. + * + * @param out The output stream to compress + * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other + * values will generate an exception + * @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for + * every block of data + * @throws IOException + */ + public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum) throws IOException { + super(out); + compressor = LZ4Factory.fastestInstance().fastCompressor(); + checksum = XXHashFactory.fastestInstance().hash32(); + bd = new BD(blockSize); + flg = new FLG(blockChecksum); + bufferOffset = 0; + maxBlockSize = bd.getBlockMaximumSize(); + buffer = new byte[maxBlockSize]; + compressedBuffer = new byte[compressor.maxCompressedLength(maxBlockSize)]; + finished = false; + writeHeader(); + } + + /** + * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. + * + * @param out The stream to compress + * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other + * values will generate an exception + * @throws IOException + */ + public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize) throws IOException { + this(out, blockSize, false); + } + + /** + * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. + * + * @param out The output stream to compress + * @throws IOException + */ + public KafkaLZ4BlockOutputStream(OutputStream out) throws IOException { + this(out, BLOCKSIZE_64KB); + } + + /** + * Writes the magic number and frame descriptor to the underlying {@link OutputStream}. + * + * @throws IOException + */ + private void writeHeader() throws IOException { + Utils.writeUnsignedIntLE(buffer, 0, MAGIC); + bufferOffset = 4; + buffer[bufferOffset++] = flg.toByte(); + buffer[bufferOffset++] = bd.toByte(); + // TODO write uncompressed content size, update flg.validate() + // TODO write dictionary id, update flg.validate() + // compute checksum on all descriptor fields + int hash = (checksum.hash(buffer, 0, bufferOffset, 0) >> 8) & 0xFF; + buffer[bufferOffset++] = (byte) hash; + // write out frame descriptor + out.write(buffer, 0, bufferOffset); + bufferOffset = 0; + } + + /** + * Compresses buffered data, optionally computes an XXHash32 checksum, and writes the result to the underlying + * {@link OutputStream}. + * + * @throws IOException + */ + private void writeBlock() throws IOException { + if (bufferOffset == 0) { + return; + } + + int compressedLength = compressor.compress(buffer, 0, bufferOffset, compressedBuffer, 0); + byte[] bufferToWrite = compressedBuffer; + int compressMethod = 0; + + // Store block uncompressed if compressed length is greater (incompressible) + if (compressedLength >= bufferOffset) { + bufferToWrite = buffer; + compressedLength = bufferOffset; + compressMethod = LZ4_FRAME_INCOMPRESSIBLE_MASK; + } + + // Write content + Utils.writeUnsignedIntLE(out, compressedLength | compressMethod); + out.write(bufferToWrite, 0, compressedLength); + + // Calculate and write block checksum + if (flg.isBlockChecksumSet()) { + int hash = checksum.hash(bufferToWrite, 0, compressedLength, 0); + Utils.writeUnsignedIntLE(out, hash); + } + bufferOffset = 0; + } + + /** + * Similar to the {@link #writeBlock()} method. Writes a 0-length block (without block checksum) to signal the end + * of the block stream. + * + * @throws IOException + */ + private void writeEndMark() throws IOException { + Utils.writeUnsignedIntLE(out, 0); + // TODO implement content checksum, update flg.validate() + finished = true; + } + + @Override + public void write(int b) throws IOException { + ensureNotFinished(); + if (bufferOffset == maxBlockSize) { + writeBlock(); + } + buffer[bufferOffset++] = (byte) b; + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + net.jpountz.util.Utils.checkRange(b, off, len); + ensureNotFinished(); + + int bufferRemainingLength = maxBlockSize - bufferOffset; + // while b will fill the buffer + while (len > bufferRemainingLength) { + // fill remaining space in buffer + System.arraycopy(b, off, buffer, bufferOffset, bufferRemainingLength); + bufferOffset = maxBlockSize; + writeBlock(); + // compute new offset and length + off += bufferRemainingLength; + len -= bufferRemainingLength; + bufferRemainingLength = maxBlockSize; + } + + System.arraycopy(b, off, buffer, bufferOffset, len); + bufferOffset += len; + } + + @Override + public void flush() throws IOException { + if (!finished) { + writeBlock(); + } + if (out != null) { + out.flush(); + } + } + + /** + * A simple state check to ensure the stream is still open. + */ + private void ensureNotFinished() { + if (finished) { + throw new IllegalStateException(CLOSED_STREAM); + } + } + + @Override + public void close() throws IOException { + if (!finished) { + writeEndMark(); + flush(); + finished = true; + } + if (out != null) { + out.close(); + out = null; + } + } + + public static class FLG { + + private static final int VERSION = 1; + + private final int presetDictionary; + private final int reserved1; + private final int contentChecksum; + private final int contentSize; + private final int blockChecksum; + private final int blockIndependence; + private final int version; + + public FLG() { + this(false); + } + + public FLG(boolean blockChecksum) { + this(0, 0, 0, 0, blockChecksum ? 1 : 0, 1, VERSION); + } + + private FLG(int presetDictionary, + int reserved1, + int contentChecksum, + int contentSize, + int blockChecksum, + int blockIndependence, + int version) { + this.presetDictionary = presetDictionary; + this.reserved1 = reserved1; + this.contentChecksum = contentChecksum; + this.contentSize = contentSize; + this.blockChecksum = blockChecksum; + this.blockIndependence = blockIndependence; + this.version = version; + validate(); + } + + public static FLG fromByte(byte flg) { + int presetDictionary = (flg >>> 0) & 1; + int reserved1 = (flg >>> 1) & 1; + int contentChecksum = (flg >>> 2) & 1; + int contentSize = (flg >>> 3) & 1; + int blockChecksum = (flg >>> 4) & 1; + int blockIndependence = (flg >>> 5) & 1; + int version = (flg >>> 6) & 3; + + return new FLG(presetDictionary, + reserved1, + contentChecksum, + contentSize, + blockChecksum, + blockIndependence, + version); + } + + public byte toByte() { + return (byte) (((presetDictionary & 1) << 0) | ((reserved1 & 1) << 1) | ((contentChecksum & 1) << 2) + | ((contentSize & 1) << 3) | ((blockChecksum & 1) << 4) | ((blockIndependence & 1) << 5) | ((version & 3) << 6)); + } + + private void validate() { + if (presetDictionary != 0) { + throw new RuntimeException("Preset dictionary is unsupported"); + } + if (reserved1 != 0) { + throw new RuntimeException("Reserved1 field must be 0"); + } + if (contentChecksum != 0) { + throw new RuntimeException("Content checksum is unsupported"); + } + if (contentSize != 0) { + throw new RuntimeException("Content size is unsupported"); + } + if (blockIndependence != 1) { + throw new RuntimeException("Dependent block stream is unsupported"); + } + if (version != VERSION) { + throw new RuntimeException(String.format("Version %d is unsupported", version)); + } + } + + public boolean isPresetDictionarySet() { + return presetDictionary == 1; + } + + public boolean isContentChecksumSet() { + return contentChecksum == 1; + } + + public boolean isContentSizeSet() { + return contentSize == 1; + } + + public boolean isBlockChecksumSet() { + return blockChecksum == 1; + } + + public boolean isBlockIndependenceSet() { + return blockIndependence == 1; + } + + public int getVersion() { + return version; + } + } + + public static class BD { + + private final int reserved2; + private final int blockSizeValue; + private final int reserved3; + + public BD() { + this(0, BLOCKSIZE_64KB, 0); + } + + public BD(int blockSizeValue) { + this(0, blockSizeValue, 0); + } + + private BD(int reserved2, int blockSizeValue, int reserved3) { + this.reserved2 = reserved2; + this.blockSizeValue = blockSizeValue; + this.reserved3 = reserved3; + validate(); + } + + public static BD fromByte(byte bd) { + int reserved2 = (bd >>> 0) & 15; + int blockMaximumSize = (bd >>> 4) & 7; + int reserved3 = (bd >>> 7) & 1; + + return new BD(reserved2, blockMaximumSize, reserved3); + } + + private void validate() { + if (reserved2 != 0) { + throw new RuntimeException("Reserved2 field must be 0"); + } + if (blockSizeValue < 4 || blockSizeValue > 7) { + throw new RuntimeException("Block size value must be between 4 and 7"); + } + if (reserved3 != 0) { + throw new RuntimeException("Reserved3 field must be 0"); + } + } + + // 2^(2n+8) + public int getBlockMaximumSize() { + return 1 << ((2 * blockSizeValue) + 8); + } + + public byte toByte() { + return (byte) (((reserved2 & 15) << 0) | ((blockSizeValue & 7) << 4) | ((reserved3 & 1) << 7)); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java index e4d688cbe0c61..2e54b560ed98b 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java +++ b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java @@ -41,4 +41,8 @@ public Record record() { public String toString() { return "LogEntry(" + offset + ", " + record + ")"; } + + public int size() { + return record.size() + Records.LOG_OVERHEAD; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 9d8935fa3beeb..5f1b45c2970e4 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -1,68 +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. + * 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 org.apache.kafka.common.record; +import java.io.DataInputStream; +import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.GatheringByteChannel; import java.util.Iterator; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.AbstractIterator; - /** * A {@link Records} implementation backed by a ByteBuffer. */ public class MemoryRecords implements Records { - private final ByteBuffer buffer; + private final Compressor compressor; + private final int capacity; + private final int sizeLimit; + private ByteBuffer buffer; + private boolean writable; - public MemoryRecords(int size) { - this(ByteBuffer.allocate(size)); + // Construct a writable memory records + private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int sizeLimit) { + this.writable = writable; + this.capacity = buffer.capacity(); + this.sizeLimit = sizeLimit; + if (this.writable) { + this.buffer = null; + this.compressor = new Compressor(buffer, type); + } else { + this.buffer = buffer; + this.compressor = null; + } } - public MemoryRecords(ByteBuffer buffer) { - this.buffer = buffer; + public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int capacity) { + return new MemoryRecords(buffer, type, true, capacity); + } + + public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) { + return emptyRecords(buffer, type, buffer.capacity()); + } + + public static MemoryRecords readableRecords(ByteBuffer buffer) { + return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity()); } /** * Append the given record and offset to the buffer */ public void append(long offset, Record record) { - buffer.putLong(offset); - buffer.putInt(record.size()); - buffer.put(record.buffer()); + if (!writable) + throw new IllegalStateException("Memory records is not writable"); + + int size = record.size(); + compressor.putLong(offset); + compressor.putInt(size); + compressor.put(record.buffer()); + compressor.recordWritten(size + Records.LOG_OVERHEAD); record.buffer().rewind(); } /** * Append a new record and offset to the buffer */ - public void append(long offset, byte[] key, byte[] value, CompressionType type) { - buffer.putLong(offset); - buffer.putInt(Record.recordSize(key, value)); - Record.write(this.buffer, key, value, type); + public void append(long offset, byte[] key, byte[] value) { + if (!writable) + throw new IllegalStateException("Memory records is not writable"); + + int size = Record.recordSize(key, value); + compressor.putLong(offset); + compressor.putInt(size); + compressor.putRecord(key, value); + compressor.recordWritten(size + Records.LOG_OVERHEAD); } /** * Check if we have room for a new record containing the given key/value pair + * + * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be + * accurate if compression is really used. When this happens, the following append may cause dynamic buffer + * re-allocation in the underlying byte buffer stream. + * + * Also note that besides the records' capacity, there is also a size limit for the batch. This size limit may be + * smaller than the capacity (e.g. when appending a single message whose size is larger than the batch size, the + * capacity will be the message size, but the size limit will still be the batch size), and when the records' size + * has exceed this limit we also mark this record as full. */ public boolean hasRoomFor(byte[] key, byte[] value) { - return this.buffer.remaining() >= Records.LOG_OVERHEAD + Record.recordSize(key, value); + return this.writable && this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + + Record.recordSize(key, value) && + this.sizeLimit >= this.compressor.estimatedBytesWritten(); + } + + public boolean isFull() { + return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten() || + this.sizeLimit <= this.compressor.estimatedBytesWritten(); + } + + /** + * Close this batch for no more appends + */ + public void close() { + if (writable) { + compressor.close(); + writable = false; + buffer = compressor.buffer(); + } } /** Write the records in this set to the given channel */ @@ -74,7 +131,24 @@ public int writeTo(GatheringByteChannel channel) throws IOException { * The size of this record set */ public int sizeInBytes() { - return this.buffer.position(); + return compressor.buffer().position(); + } + + /** + * The compression rate of this record set + */ + public double compressionRate() { + if (compressor == null) + return 1.0; + else + return compressor.compressionRate(); + } + + /** + * Return the capacity of the buffer + */ + public int capacity() { + return this.capacity; } /** @@ -84,36 +158,114 @@ public ByteBuffer buffer() { return buffer.duplicate(); } + /** + * Return a flipped duplicate of the closed buffer to reading records + */ + public ByteBuffer flip() { + if (writable) + throw new IllegalStateException("The memory records need to be closed for write before rewinding for read"); + + return (ByteBuffer) buffer.flip(); + } + @Override public Iterator iterator() { - return new RecordsIterator(this.buffer); + ByteBuffer copy = this.buffer.duplicate(); + return new RecordsIterator(copy, CompressionType.NONE, false); + } + + @Override + public String toString() { + Iterator iter = iterator(); + StringBuilder builder = new StringBuilder(); + builder.append('['); + while (iter.hasNext()) { + LogEntry entry = iter.next(); + builder.append('('); + builder.append("offset="); + builder.append(entry.offset()); + builder.append(","); + builder.append("record="); + builder.append(entry.record()); + builder.append(")"); + } + builder.append(']'); + return builder.toString(); } - /* TODO: allow reuse of the buffer used for iteration */ public static class RecordsIterator extends AbstractIterator { private final ByteBuffer buffer; + private final DataInputStream stream; + private final CompressionType type; + private final boolean shallow; + private RecordsIterator innerIter; - public RecordsIterator(ByteBuffer buffer) { - ByteBuffer copy = buffer.duplicate(); - copy.flip(); - this.buffer = copy; + public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) { + this.type = type; + this.buffer = buffer; + this.shallow = shallow; + this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); } + /* + * Read the next record from the buffer. + * + * Note that in the compressed message set, each message value size is set as the size of the un-compressed + * version of the message value, so when we do de-compression allocating an array of the specified size for + * reading compressed value data is sufficient. + */ @Override protected LogEntry makeNext() { - if (buffer.remaining() < Records.LOG_OVERHEAD) - return allDone(); - long offset = buffer.getLong(); - int size = buffer.getInt(); - if (size < 0) - throw new IllegalStateException("Record with size " + size); - if (buffer.remaining() < size) - return allDone(); - ByteBuffer rec = buffer.slice(); - rec.limit(size); - this.buffer.position(this.buffer.position() + size); - return new LogEntry(offset, new Record(rec)); + if (innerDone()) { + try { + // read the offset + long offset = stream.readLong(); + // read record size + int size = stream.readInt(); + if (size < 0) + throw new IllegalStateException("Record with size " + size); + // read the record, if compression is used we cannot depend on size + // and hence has to do extra copy + ByteBuffer rec; + if (type == CompressionType.NONE) { + rec = buffer.slice(); + int newPos = buffer.position() + size; + if (newPos > buffer.limit()) + return allDone(); + buffer.position(newPos); + rec.limit(size); + } else { + byte[] recordBuffer = new byte[size]; + stream.readFully(recordBuffer, 0, size); + rec = ByteBuffer.wrap(recordBuffer); + } + LogEntry entry = new LogEntry(offset, new Record(rec)); + + // decide whether to go shallow or deep iteration if it is compressed + CompressionType compression = entry.record().compressionType(); + if (compression == CompressionType.NONE || shallow) { + return entry; + } else { + // init the inner iterator with the value payload of the message, + // which will de-compress the payload to a set of messages; + // since we assume nested compression is not allowed, the deep iterator + // would not try to further decompress underlying messages + ByteBuffer value = entry.record().value(); + innerIter = new RecordsIterator(value, compression, true); + return innerIter.next(); + } + } catch (EOFException e) { + return allDone(); + } catch (IOException e) { + throw new KafkaException(e); + } + } else { + return innerIter.next(); + } } - } + private boolean innerDone() { + return innerIter == null || !innerIter.hasNext(); + } + } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/Record.java b/clients/src/main/java/org/apache/kafka/common/record/Record.java index f1dc9778502cb..50fac24597ca9 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Record.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Record.java @@ -18,6 +18,7 @@ import java.nio.ByteBuffer; +import org.apache.kafka.common.utils.Crc32; import org.apache.kafka.common.utils.Utils; @@ -40,13 +41,15 @@ public final class Record { public static final int KEY_OFFSET = KEY_SIZE_OFFSET + KEY_SIZE_LENGTH; public static final int VALUE_SIZE_LENGTH = 4; - /** The amount of overhead bytes in a record */ - public static final int RECORD_OVERHEAD = KEY_OFFSET + VALUE_SIZE_LENGTH; + /** + * The size for the record header + */ + public static final int HEADER_SIZE = CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH; /** - * The minimum valid size for the record header + * The amount of overhead bytes in a record */ - public static final int MIN_HEADER_SIZE = CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH; + public static final int RECORD_OVERHEAD = HEADER_SIZE + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH; /** * The current "magic" value @@ -54,10 +57,10 @@ public final class Record { public static final byte CURRENT_MAGIC_VALUE = 0; /** - * Specifies the mask for the compression code. 2 bits to hold the compression codec. 0 is reserved to indicate no + * Specifies the mask for the compression code. 3 bits to hold the compression codec. 0 is reserved to indicate no * compression */ - public static final int COMPRESSION_CODEC_MASK = 0x03; + public static final int COMPRESSION_CODEC_MASK = 0x07; /** * Compression code for uncompressed records @@ -71,27 +74,29 @@ public Record(ByteBuffer buffer) { } /** - * A constructor to create a LogRecord + * A constructor to create a LogRecord. If the record's compression type is not none, then + * its value payload should be already compressed with the specified type; the constructor + * would always write the value payload as is and will not do the compression itself. * * @param key The key of the record (null, if none) * @param value The record value - * @param codec The compression codec used on the contents of the record (if any) + * @param type The compression type used on the contents of the record (if any) * @param valueOffset The offset into the payload array used to extract payload * @param valueSize The size of the payload to use */ - public Record(byte[] key, byte[] value, CompressionType codec, int valueOffset, int valueSize) { - this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length, value == null ? 0 : valueSize >= 0 ? valueSize - : value.length - valueOffset))); - write(this.buffer, key, value, codec, valueOffset, valueSize); + public Record(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) { + this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length, + value == null ? 0 : valueSize >= 0 ? valueSize : value.length - valueOffset))); + write(this.buffer, key, value, type, valueOffset, valueSize); this.buffer.rewind(); } - public Record(byte[] key, byte[] value, CompressionType codec) { - this(key, value, codec, 0, -1); + public Record(byte[] key, byte[] value, CompressionType type) { + this(key, value, type, 0, -1); } - public Record(byte[] value, CompressionType codec) { - this(null, value, codec); + public Record(byte[] value, CompressionType type) { + this(null, value, type); } public Record(byte[] key, byte[] value) { @@ -102,40 +107,37 @@ public Record(byte[] value) { this(null, value, CompressionType.NONE); } - public static void write(ByteBuffer buffer, byte[] key, byte[] value, CompressionType codec, int valueOffset, int valueSize) { - // skip crc, we will fill that in at the end - int pos = buffer.position(); - buffer.position(pos + MAGIC_OFFSET); - buffer.put(CURRENT_MAGIC_VALUE); - byte attributes = 0; - if (codec.id > 0) - attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & codec.id)); - buffer.put(attributes); + // Write a record to the buffer, if the record's compression type is none, then + // its value payload should be already compressed with the specified type + public static void write(ByteBuffer buffer, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) { + // construct the compressor with compression type none since this function will not do any + //compression according to the input type, it will just write the record's payload as is + Compressor compressor = new Compressor(buffer, CompressionType.NONE, buffer.capacity()); + compressor.putRecord(key, value, type, valueOffset, valueSize); + } + + public static void write(Compressor compressor, long crc, byte attributes, byte[] key, byte[] value, int valueOffset, int valueSize) { + // write crc + compressor.putInt((int) (crc & 0xffffffffL)); + // write magic value + compressor.putByte(CURRENT_MAGIC_VALUE); + // write attributes + compressor.putByte(attributes); // write the key if (key == null) { - buffer.putInt(-1); + compressor.putInt(-1); } else { - buffer.putInt(key.length); - buffer.put(key, 0, key.length); + compressor.putInt(key.length); + compressor.put(key, 0, key.length); } // write the value if (value == null) { - buffer.putInt(-1); + compressor.putInt(-1); } else { int size = valueSize >= 0 ? valueSize : (value.length - valueOffset); - buffer.putInt(size); - buffer.put(value, valueOffset, size); + compressor.putInt(size); + compressor.put(value, valueOffset, size); } - - // now compute the checksum and fill it in - long crc = computeChecksum(buffer, - buffer.arrayOffset() + pos + MAGIC_OFFSET, - buffer.position() - pos - MAGIC_OFFSET - buffer.arrayOffset()); - Utils.writeUnsignedInt(buffer, pos + CRC_OFFSET, crc); - } - - public static void write(ByteBuffer buffer, byte[] key, byte[] value, CompressionType codec) { - write(buffer, key, value, codec, 0, -1); } public static int recordSize(byte[] key, byte[] value) { @@ -150,13 +152,51 @@ public ByteBuffer buffer() { return this.buffer; } + public static byte computeAttributes(CompressionType type) { + byte attributes = 0; + if (type.id > 0) + attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id)); + return attributes; + } + /** * Compute the checksum of the record from the record contents */ public static long computeChecksum(ByteBuffer buffer, int position, int size) { - return Utils.crc32(buffer.array(), buffer.arrayOffset() + position, size - buffer.arrayOffset()); + Crc32 crc = new Crc32(); + crc.update(buffer.array(), buffer.arrayOffset() + position, size); + return crc.getValue(); + } + + /** + * Compute the checksum of the record from the attributes, key and value payloads + */ + public static long computeChecksum(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) { + Crc32 crc = new Crc32(); + crc.update(CURRENT_MAGIC_VALUE); + byte attributes = 0; + if (type.id > 0) + attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id)); + crc.update(attributes); + // update for the key + if (key == null) { + crc.updateInt(-1); + } else { + crc.updateInt(key.length); + crc.update(key, 0, key.length); + } + // update for the value + if (value == null) { + crc.updateInt(-1); + } else { + int size = valueSize >= 0 ? valueSize : (value.length - valueOffset); + crc.updateInt(size); + crc.update(value, valueOffset, size); + } + return crc.getValue(); } + /** * Compute the checksum of the record from the record contents */ @@ -239,7 +279,7 @@ public byte attributes() { } /** - * The compression codec used with this record + * The compression type used with this record */ public CompressionType compressionType() { return CompressionType.forId(buffer.get(ATTRIBUTES_OFFSET) & COMPRESSION_CODEC_MASK); @@ -277,12 +317,13 @@ private ByteBuffer sliceDelimited(int start) { } public String toString() { - return String.format("Record(magic = %d, attributes = %d, crc = %d, key = %d bytes, value = %d bytes)", + return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, key = %d bytes, value = %d bytes)", magic(), attributes(), + compressionType(), checksum(), - key().limit(), - value().limit()); + key() == null ? 0 : key().limit(), + value() == null ? 0 : value().limit()); } public boolean equals(Object other) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java new file mode 100644 index 0000000000000..5d3d52859587c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public abstract class AbstractRequest extends AbstractRequestResponse { + + public AbstractRequest(Struct struct) { + super(struct); + } + + /** + * Get an error response for a request for a given api version + */ + public abstract AbstractRequestResponse getErrorResponse(int versionId, Throwable e); + + /** + * Factory method for getting a request object based on ApiKey ID and a buffer + */ + public static AbstractRequest getRequest(int requestId, int versionId, ByteBuffer buffer) { + switch (ApiKeys.forId(requestId)) { + case PRODUCE: + return ProduceRequest.parse(buffer, versionId); + case FETCH: + return FetchRequest.parse(buffer, versionId); + case LIST_OFFSETS: + return ListOffsetRequest.parse(buffer, versionId); + case METADATA: + return MetadataRequest.parse(buffer, versionId); + case OFFSET_COMMIT: + return OffsetCommitRequest.parse(buffer, versionId); + case OFFSET_FETCH: + return OffsetFetchRequest.parse(buffer, versionId); + case CONSUMER_METADATA: + return ConsumerMetadataRequest.parse(buffer, versionId); + case JOIN_GROUP: + return JoinGroupRequest.parse(buffer, versionId); + case HEARTBEAT: + return HeartbeatRequest.parse(buffer, versionId); + default: + return null; + } + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java new file mode 100644 index 0000000000000..37aff6c0fd2ec --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java @@ -0,0 +1,66 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public abstract class AbstractRequestResponse { + protected final Struct struct; + + + public AbstractRequestResponse(Struct struct) { + this.struct = struct; + } + + public Struct toStruct() { + return struct; + } + + /** + * Get the serialized size of this object + */ + public int sizeOf() { + return struct.sizeOf(); + } + + /** + * Write this object to a buffer + */ + public void writeTo(ByteBuffer buffer) { + struct.writeTo(buffer); + } + + @Override + public String toString() { + return struct.toString(); + } + + @Override + public int hashCode() { + return struct.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + AbstractRequestResponse other = (AbstractRequestResponse) obj; + return struct.equals(other.struct); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java new file mode 100644 index 0000000000000..5b3e04a54436b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java @@ -0,0 +1,65 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class ConsumerMetadataRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + + private final String groupId; + + public ConsumerMetadataRequest(String groupId) { + super(new Struct(CURRENT_SCHEMA)); + + struct.set(GROUP_ID_KEY_NAME, groupId); + this.groupId = groupId; + } + + public ConsumerMetadataRequest(Struct struct) { + super(struct); + groupId = struct.getString(GROUP_ID_KEY_NAME); + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch (versionId) { + case 0: + return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CONSUMER_METADATA.id))); + } + } + + public String groupId() { + return groupId; + } + + public static ConsumerMetadataRequest parse(ByteBuffer buffer, int versionId) { + return new ConsumerMetadataRequest(ProtoUtils.parseRequest(ApiKeys.CONSUMER_METADATA.id, versionId, buffer)); + } + + public static ConsumerMetadataRequest parse(ByteBuffer buffer) { + return new ConsumerMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java new file mode 100644 index 0000000000000..0c250c389a445 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class ConsumerMetadataResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id); + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String COORDINATOR_KEY_NAME = "coordinator"; + + // coordinator level field names + private static final String NODE_ID_KEY_NAME = "node_id"; + private static final String HOST_KEY_NAME = "host"; + private static final String PORT_KEY_NAME = "port"; + + private final short errorCode; + private final Node node; + + public ConsumerMetadataResponse(short errorCode, Node node) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(ERROR_CODE_KEY_NAME, errorCode); + Struct coordinator = struct.instance(COORDINATOR_KEY_NAME); + coordinator.set(NODE_ID_KEY_NAME, node.id()); + coordinator.set(HOST_KEY_NAME, node.host()); + coordinator.set(PORT_KEY_NAME, node.port()); + struct.set(COORDINATOR_KEY_NAME, coordinator); + this.errorCode = errorCode; + this.node = node; + } + + public ConsumerMetadataResponse(Struct struct) { + super(struct); + errorCode = struct.getShort(ERROR_CODE_KEY_NAME); + Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME); + int nodeId = broker.getInt(NODE_ID_KEY_NAME); + String host = broker.getString(HOST_KEY_NAME); + int port = broker.getInt(PORT_KEY_NAME); + node = new Node(nodeId, host, port); + } + + public short errorCode() { + return errorCode; + } + + public Node node() { + return node; + } + + public static ConsumerMetadataResponse parse(ByteBuffer buffer) { + return new ConsumerMetadataResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java new file mode 100644 index 0000000000000..df073a0e76cc5 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -0,0 +1,165 @@ +/** + * 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 org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +public class FetchRequest extends AbstractRequest { + + public static final int CONSUMER_REPLICA_ID = -1; + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id); + private static final String REPLICA_ID_KEY_NAME = "replica_id"; + private static final String MAX_WAIT_KEY_NAME = "max_wait_time"; + private static final String MIN_BYTES_KEY_NAME = "min_bytes"; + private static final String TOPICS_KEY_NAME = "topics"; + + // topic level field names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + // partition level field names + private static final String PARTITION_KEY_NAME = "partition"; + private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset"; + private static final String MAX_BYTES_KEY_NAME = "max_bytes"; + + private final int replicaId; + private final int maxWait; + private final int minBytes; + private final Map fetchData; + + public static final class PartitionData { + public final long offset; + public final int maxBytes; + + public PartitionData(long offset, int maxBytes) { + this.offset = offset; + this.maxBytes = maxBytes; + } + } + + /** + * Create a non-replica fetch request + */ + public FetchRequest(int maxWait, int minBytes, Map fetchData) { + this(CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData); + } + + /** + * Create a replica fetch request + */ + public FetchRequest(int replicaId, int maxWait, int minBytes, Map fetchData) { + super(new Struct(CURRENT_SCHEMA)); + Map> topicsData = CollectionUtils.groupDataByTopic(fetchData); + + struct.set(REPLICA_ID_KEY_NAME, replicaId); + struct.set(MAX_WAIT_KEY_NAME, maxWait); + struct.set(MIN_BYTES_KEY_NAME, minBytes); + List topicArray = new ArrayList(); + for (Map.Entry> topicEntry : topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset); + partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + this.replicaId = replicaId; + this.maxWait = maxWait; + this.minBytes = minBytes; + this.fetchData = fetchData; + } + + public FetchRequest(Struct struct) { + super(struct); + replicaId = struct.getInt(REPLICA_ID_KEY_NAME); + maxWait = struct.getInt(MAX_WAIT_KEY_NAME); + minBytes = struct.getInt(MIN_BYTES_KEY_NAME); + fetchData = new HashMap(); + for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { + Struct topicResponse = (Struct) topicResponseObj; + String topic = topicResponse.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + long offset = partitionResponse.getLong(FETCH_OFFSET_KEY_NAME); + int maxBytes = partitionResponse.getInt(MAX_BYTES_KEY_NAME); + PartitionData partitionData = new PartitionData(offset, maxBytes); + fetchData.put(new TopicPartition(topic, partition), partitionData); + } + } + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + Map responseData = new HashMap(); + + for (Map.Entry entry: fetchData.entrySet()) { + FetchResponse.PartitionData partitionResponse = new FetchResponse.PartitionData(Errors.forException(e).code(), + FetchResponse.INVALID_HIGHWATERMARK, + FetchResponse.EMPTY_RECORD_SET); + responseData.put(entry.getKey(), partitionResponse); + } + + switch (versionId) { + case 0: + return new FetchResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.FETCH.id))); + } + } + + public int replicaId() { + return replicaId; + } + + public int maxWait() { + return maxWait; + } + + public int minBytes() { + return minBytes; + } + + public Map fetchData() { + return fetchData; + } + + public static FetchRequest parse(ByteBuffer buffer, int versionId) { + return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer)); + } + + public static FetchRequest parse(ByteBuffer buffer) { + return new FetchRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java new file mode 100644 index 0000000000000..eb8951fba48c3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class FetchResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id); + private static final String RESPONSES_KEY_NAME = "responses"; + + // topic level field names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; + + // partition level field names + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * OFFSET_OUT_OF_RANGE (1) + * UNKNOWN_TOPIC_OR_PARTITION (3) + * NOT_LEADER_FOR_PARTITION (6) + * REPLICA_NOT_AVAILABLE (9) + * UNKNOWN (-1) + */ + + private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark"; + private static final String RECORD_SET_KEY_NAME = "record_set"; + + public static final long INVALID_HIGHWATERMARK = -1L; + public static final ByteBuffer EMPTY_RECORD_SET = ByteBuffer.allocate(0); + + private final Map responseData; + + public static final class PartitionData { + public final short errorCode; + public final long highWatermark; + public final ByteBuffer recordSet; + + public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) { + this.errorCode = errorCode; + this.highWatermark = highWatermark; + this.recordSet = recordSet; + } + } + + public FetchResponse(Map responseData) { + super(new Struct(CURRENT_SCHEMA)); + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + + List topicArray = new ArrayList(); + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode); + partitionData.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark); + partitionData.set(RECORD_SET_KEY_NAME, fetchPartitionData.recordSet); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + this.responseData = responseData; + } + + public FetchResponse(Struct struct) { + super(struct); + responseData = new HashMap(); + for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { + Struct topicResponse = (Struct) topicResponseObj; + String topic = topicResponse.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); + long highWatermark = partitionResponse.getLong(HIGH_WATERMARK_KEY_NAME); + ByteBuffer recordSet = partitionResponse.getBytes(RECORD_SET_KEY_NAME); + PartitionData partitionData = new PartitionData(errorCode, highWatermark, recordSet); + responseData.put(new TopicPartition(topic, partition), partitionData); + } + } + } + + public Map responseData() { + return responseData; + } + + public static FetchResponse parse(ByteBuffer buffer) { + return new FetchResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java new file mode 100644 index 0000000000000..89719f13e2870 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class HeartbeatRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + + private final String groupId; + private final int groupGenerationId; + private final String consumerId; + + public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId); + struct.set(CONSUMER_ID_KEY_NAME, consumerId); + this.groupId = groupId; + this.groupGenerationId = groupGenerationId; + this.consumerId = consumerId; + } + + public HeartbeatRequest(Struct struct) { + super(struct); + groupId = struct.getString(GROUP_ID_KEY_NAME); + groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME); + consumerId = struct.getString(CONSUMER_ID_KEY_NAME); + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch (versionId) { + case 0: + return new HeartbeatResponse(Errors.forException(e).code()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.HEARTBEAT.id))); + } + } + + public String groupId() { + return groupId; + } + + public int groupGenerationId() { + return groupGenerationId; + } + + public String consumerId() { + return consumerId; + } + + public static HeartbeatRequest parse(ByteBuffer buffer, int versionId) { + return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer)); + } + + public static HeartbeatRequest parse(ByteBuffer buffer) { + return new HeartbeatRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java new file mode 100644 index 0000000000000..96e6ab07dfa77 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -0,0 +1,55 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class HeartbeatResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); + private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * CONSUMER_COORDINATOR_NOT_AVAILABLE (15) + * NOT_COORDINATOR_FOR_CONSUMER (16) + * ILLEGAL_GENERATION (22) + * UNKNOWN_CONSUMER_ID (25) + */ + + private final short errorCode; + public HeartbeatResponse(short errorCode) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(ERROR_CODE_KEY_NAME, errorCode); + this.errorCode = errorCode; + } + + public HeartbeatResponse(Struct struct) { + super(struct); + errorCode = struct.getShort(ERROR_CODE_KEY_NAME); + } + + public short errorCode() { + return errorCode; + } + + public static HeartbeatResponse parse(ByteBuffer buffer) { + return new HeartbeatResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java new file mode 100644 index 0000000000000..1ffe0760b40c4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -0,0 +1,112 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class JoinGroupRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String SESSION_TIMEOUT_KEY_NAME = "session_timeout"; + private static final String TOPICS_KEY_NAME = "topics"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + private static final String STRATEGY_KEY_NAME = "partition_assignment_strategy"; + + public static final String UNKNOWN_CONSUMER_ID = ""; + + private final String groupId; + private final int sessionTimeout; + private final List topics; + private final String consumerId; + private final String strategy; + + public JoinGroupRequest(String groupId, int sessionTimeout, List topics, String consumerId, String strategy) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); + struct.set(TOPICS_KEY_NAME, topics.toArray()); + struct.set(CONSUMER_ID_KEY_NAME, consumerId); + struct.set(STRATEGY_KEY_NAME, strategy); + this.groupId = groupId; + this.sessionTimeout = sessionTimeout; + this.topics = topics; + this.consumerId = consumerId; + this.strategy = strategy; + } + + public JoinGroupRequest(Struct struct) { + super(struct); + groupId = struct.getString(GROUP_ID_KEY_NAME); + sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME); + Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME); + topics = new ArrayList(); + for (Object topic: topicsArray) + topics.add((String) topic); + consumerId = struct.getString(CONSUMER_ID_KEY_NAME); + strategy = struct.getString(STRATEGY_KEY_NAME); + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch (versionId) { + case 0: + return new JoinGroupResponse( + Errors.forException(e).code(), + JoinGroupResponse.UNKNOWN_GENERATION_ID, + JoinGroupResponse.UNKNOWN_CONSUMER_ID, + Collections.emptyList()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id))); + } + } + + public String groupId() { + return groupId; + } + + public int sessionTimeout() { + return sessionTimeout; + } + + public List topics() { + return topics; + } + + public String consumerId() { + return consumerId; + } + + public String strategy() { + return strategy; + } + + public static JoinGroupRequest parse(ByteBuffer buffer, int versionId) { + return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, versionId, buffer)); + } + + public static JoinGroupRequest parse(ByteBuffer buffer) { + return new JoinGroupRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java new file mode 100644 index 0000000000000..7bf544ef17068 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -0,0 +1,111 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +import java.nio.ByteBuffer; +import java.util.*; + +public class JoinGroupResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); + private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * CONSUMER_COORDINATOR_NOT_AVAILABLE (15) + * NOT_COORDINATOR_FOR_CONSUMER (16) + * INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY (23) + * UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY (24) + * UNKNOWN_CONSUMER_ID (25) + * INVALID_SESSION_TIMEOUT (26) + */ + + private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + private static final String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + public static final int UNKNOWN_GENERATION_ID = -1; + public static final String UNKNOWN_CONSUMER_ID = ""; + + private final short errorCode; + private final int generationId; + private final String consumerId; + private final List assignedPartitions; + + public JoinGroupResponse(short errorCode, int generationId, String consumerId, List assignedPartitions) { + super(new Struct(CURRENT_SCHEMA)); + + Map> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions); + + struct.set(ERROR_CODE_KEY_NAME, errorCode); + struct.set(GENERATION_ID_KEY_NAME, generationId); + struct.set(CONSUMER_ID_KEY_NAME, consumerId); + List topicArray = new ArrayList(); + for (Map.Entry> entries: partitionsByTopic.entrySet()) { + Struct topicData = struct.instance(ASSIGNED_PARTITIONS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entries.getKey()); + topicData.set(PARTITIONS_KEY_NAME, entries.getValue().toArray()); + topicArray.add(topicData); + } + struct.set(ASSIGNED_PARTITIONS_KEY_NAME, topicArray.toArray()); + + this.errorCode = errorCode; + this.generationId = generationId; + this.consumerId = consumerId; + this.assignedPartitions = assignedPartitions; + } + + public JoinGroupResponse(Struct struct) { + super(struct); + assignedPartitions = new ArrayList(); + for (Object topicDataObj : struct.getArray(ASSIGNED_PARTITIONS_KEY_NAME)) { + Struct topicData = (Struct) topicDataObj; + String topic = topicData.getString(TOPIC_KEY_NAME); + for (Object partitionObj : topicData.getArray(PARTITIONS_KEY_NAME)) + assignedPartitions.add(new TopicPartition(topic, (Integer) partitionObj)); + } + errorCode = struct.getShort(ERROR_CODE_KEY_NAME); + generationId = struct.getInt(GENERATION_ID_KEY_NAME); + consumerId = struct.getString(CONSUMER_ID_KEY_NAME); + } + + public short errorCode() { + return errorCode; + } + + public int generationId() { + return generationId; + } + + public String consumerId() { + return consumerId; + } + + public List assignedPartitions() { + return assignedPartitions; + } + + public static JoinGroupResponse parse(ByteBuffer buffer) { + return new JoinGroupResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java new file mode 100644 index 0000000000000..6da4a0e14b099 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -0,0 +1,142 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ListOffsetRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id); + private static final String REPLICA_ID_KEY_NAME = "replica_id"; + private static final String TOPICS_KEY_NAME = "topics"; + + // topic level field names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + // partition level field names + private static final String PARTITION_KEY_NAME = "partition"; + private static final String TIMESTAMP_KEY_NAME = "timestamp"; + private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets"; + + private final int replicaId; + private final Map offsetData; + + public static final class PartitionData { + public final long timestamp; + public final int maxNumOffsets; + + public PartitionData(long timestamp, int maxNumOffsets) { + this.timestamp = timestamp; + this.maxNumOffsets = maxNumOffsets; + } + } + + public ListOffsetRequest(Map offsetData) { + this(-1, offsetData); + } + + public ListOffsetRequest(int replicaId, Map offsetData) { + super(new Struct(CURRENT_SCHEMA)); + Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); + + struct.set(REPLICA_ID_KEY_NAME, replicaId); + List topicArray = new ArrayList(); + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData offsetPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); + partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + this.replicaId = replicaId; + this.offsetData = offsetData; + } + + public ListOffsetRequest(Struct struct) { + super(struct); + replicaId = struct.getInt(REPLICA_ID_KEY_NAME); + offsetData = new HashMap(); + for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { + Struct topicResponse = (Struct) topicResponseObj; + String topic = topicResponse.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); + int maxNumOffsets = partitionResponse.getInt(MAX_NUM_OFFSETS_KEY_NAME); + PartitionData partitionData = new PartitionData(timestamp, maxNumOffsets); + offsetData.put(new TopicPartition(topic, partition), partitionData); + } + } + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + Map responseData = new HashMap(); + + for (Map.Entry entry: offsetData.entrySet()) { + ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(Errors.forException(e).code(), new ArrayList()); + responseData.put(entry.getKey(), partitionResponse); + } + + switch (versionId) { + case 0: + return new ListOffsetResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id))); + } + } + + public int replicaId() { + return replicaId; + } + + public Map offsetData() { + return offsetData; + } + + public static ListOffsetRequest parse(ByteBuffer buffer, int versionId) { + return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer)); + } + + public static ListOffsetRequest parse(ByteBuffer buffer) { + return new ListOffsetRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java new file mode 100644 index 0000000000000..f70608600fc4e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -0,0 +1,118 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ListOffsetResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id); + private static final String RESPONSES_KEY_NAME = "responses"; + + // topic level field names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; + + // partition level field names + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * UNKNOWN_TOPIC_OR_PARTITION (3) + * NOT_LEADER_FOR_PARTITION (6) + * UNKNOWN (-1) + */ + + private static final String OFFSETS_KEY_NAME = "offsets"; + + private final Map responseData; + + public static final class PartitionData { + public final short errorCode; + public final List offsets; + + public PartitionData(short errorCode, List offsets) { + this.errorCode = errorCode; + this.offsets = offsets; + } + } + + public ListOffsetResponse(Map responseData) { + super(new Struct(CURRENT_SCHEMA)); + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + + List topicArray = new ArrayList(); + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData offsetPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.errorCode); + partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray()); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + this.responseData = responseData; + } + + public ListOffsetResponse(Struct struct) { + super(struct); + responseData = new HashMap(); + for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { + Struct topicResponse = (Struct) topicResponseObj; + String topic = topicResponse.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); + Object[] offsets = partitionResponse.getArray(OFFSETS_KEY_NAME); + List offsetsList = new ArrayList(); + for (Object offset: offsets) + offsetsList.add((Long) offset); + PartitionData partitionData = new PartitionData(errorCode, offsetsList); + responseData.put(new TopicPartition(topic, partition), partitionData); + } + } + } + + public Map responseData() { + return responseData; + } + + public static ListOffsetResponse parse(ByteBuffer buffer) { + return new ListOffsetResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java new file mode 100644 index 0000000000000..f70e8dac70c1a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -0,0 +1,80 @@ +/** + * 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 org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +public class MetadataRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); + private static final String TOPICS_KEY_NAME = "topics"; + + private final List topics; + + public MetadataRequest(List topics) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(TOPICS_KEY_NAME, topics.toArray()); + this.topics = topics; + } + + public MetadataRequest(Struct struct) { + super(struct); + Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); + topics = new ArrayList(); + for (Object topicObj: topicArray) { + topics.add((String) topicObj); + } + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + Map topicErrors = new HashMap(); + for (String topic : topics) { + topicErrors.put(topic, Errors.forException(e)); + } + + Cluster cluster = new Cluster(new ArrayList(), new ArrayList()); + switch (versionId) { + case 0: + return new MetadataResponse(cluster, topicErrors); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id))); + } + } + + public List topics() { + return topics; + } + + public static MetadataRequest parse(ByteBuffer buffer, int versionId) { + return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer)); + } + + public static MetadataRequest parse(ByteBuffer buffer) { + return new MetadataRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java new file mode 100644 index 0000000000000..c8f2d087dfc85 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -0,0 +1,177 @@ +/** + * 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 org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +public class MetadataResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); + private static final String BROKERS_KEY_NAME = "brokers"; + private static final String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; + + // broker level field names + private static final String NODE_ID_KEY_NAME = "node_id"; + private static final String HOST_KEY_NAME = "host"; + private static final String PORT_KEY_NAME = "port"; + + // topic level field names + private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code"; + + /** + * Possible error code: + * + * TODO + */ + + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata"; + + // partition level field names + private static final String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code"; + + /** + * Possible error code: + * + * TODO + */ + + private static final String PARTITION_KEY_NAME = "partition_id"; + private static final String LEADER_KEY_NAME = "leader"; + private static final String REPLICAS_KEY_NAME = "replicas"; + private static final String ISR_KEY_NAME = "isr"; + + private final Cluster cluster; + private final Map errors; + + /** + * Constructor for MetadataResponse where there are errors for some of the topics, + * error data take precedence over cluster information for particular topic + */ + public MetadataResponse(Cluster cluster, Map errors) { + super(new Struct(CURRENT_SCHEMA)); + + List brokerArray = new ArrayList(); + for (Node node : cluster.nodes()) { + Struct broker = struct.instance(BROKERS_KEY_NAME); + broker.set(NODE_ID_KEY_NAME, node.id()); + broker.set(HOST_KEY_NAME, node.host()); + broker.set(PORT_KEY_NAME, node.port()); + brokerArray.add(broker); + } + struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); + + List topicArray = new ArrayList(); + for (String topic : cluster.topics()) { + Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); + + topicData.set(TOPIC_KEY_NAME, topic); + if (errors.containsKey(topic)) { + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, errors.get(topic).code()); + } else { + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, Errors.NONE.code()); + List partitionArray = new ArrayList(); + for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) { + Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); + partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, Errors.NONE.code()); + partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition()); + partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id()); + ArrayList replicas = new ArrayList(); + for (Node node : fetchPartitionData.replicas()) + replicas.add(node.id()); + partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); + ArrayList isr = new ArrayList(); + for (Node node : fetchPartitionData.inSyncReplicas()) + isr.add(node.id()); + partitionData.set(ISR_KEY_NAME, isr.toArray()); + partitionArray.add(partitionData); + } + topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray()); + } + + topicArray.add(topicData); + } + struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray()); + + this.cluster = cluster; + this.errors = new HashMap(); + } + + public MetadataResponse(Struct struct) { + super(struct); + Map errors = new HashMap(); + Map brokers = new HashMap(); + Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME); + for (int i = 0; i < brokerStructs.length; i++) { + Struct broker = (Struct) brokerStructs[i]; + int nodeId = broker.getInt(NODE_ID_KEY_NAME); + String host = broker.getString(HOST_KEY_NAME); + int port = broker.getInt(PORT_KEY_NAME); + brokers.put(nodeId, new Node(nodeId, host, port)); + } + List partitions = new ArrayList(); + Object[] topicInfos = (Object[]) struct.get(TOPIC_METATDATA_KEY_NAME); + for (int i = 0; i < topicInfos.length; i++) { + Struct topicInfo = (Struct) topicInfos[i]; + short topicError = topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME); + String topic = topicInfo.getString(TOPIC_KEY_NAME); + if (topicError == Errors.NONE.code()) { + Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME); + for (int j = 0; j < partitionInfos.length; j++) { + Struct partitionInfo = (Struct) partitionInfos[j]; + int partition = partitionInfo.getInt(PARTITION_KEY_NAME); + int leader = partitionInfo.getInt(LEADER_KEY_NAME); + Node leaderNode = leader == -1 ? null : brokers.get(leader); + Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME); + Node[] replicaNodes = new Node[replicas.length]; + for (int k = 0; k < replicas.length; k++) + replicaNodes[k] = brokers.get(replicas[k]); + Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME); + Node[] isrNodes = new Node[isr.length]; + for (int k = 0; k < isr.length; k++) + isrNodes[k] = brokers.get(isr[k]); + partitions.add(new PartitionInfo(topic, partition, leaderNode, replicaNodes, isrNodes)); + } + } else { + errors.put(topic, Errors.forCode(topicError)); + } + } + this.errors = errors; + this.cluster = new Cluster(brokers.values(), partitions); + } + + public Map errors() { + return this.errors; + } + + public Cluster cluster() { + return this.cluster; + } + + public static MetadataResponse parse(ByteBuffer buffer) { + return new MetadataResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java new file mode 100644 index 0000000000000..d6e63861d02be --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -0,0 +1,266 @@ +/** + * 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 org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +/** + * This wrapper supports both v0 and v1 of OffsetCommitRequest. + */ +public class OffsetCommitRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + private static final String TOPICS_KEY_NAME = "topics"; + private static final String RETENTION_TIME_KEY_NAME = "retention_time"; + + // topic level field names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + // partition level field names + private static final String PARTITION_KEY_NAME = "partition"; + private static final String COMMIT_OFFSET_KEY_NAME = "offset"; + private static final String METADATA_KEY_NAME = "metadata"; + + @Deprecated + private static final String TIMESTAMP_KEY_NAME = "timestamp"; // for v0, v1 + + // default values for the current version + public static final int DEFAULT_GENERATION_ID = -1; + public static final String DEFAULT_CONSUMER_ID = ""; + public static final long DEFAULT_RETENTION_TIME = -1L; + + // default values for old versions, + // will be removed after these versions are deprecated + @Deprecated + public static final long DEFAULT_TIMESTAMP = -1L; // for V0, V1 + + private final String groupId; + private final String consumerId; + private final int generationId; + private final long retentionTime; + private final Map offsetData; + + public static final class PartitionData { + @Deprecated + public final long timestamp; // for V1 + + public final long offset; + public final String metadata; + + @Deprecated + public PartitionData(long offset, long timestamp, String metadata) { + this.offset = offset; + this.timestamp = timestamp; + this.metadata = metadata; + } + + public PartitionData(long offset, String metadata) { + this(offset, DEFAULT_TIMESTAMP, metadata); + } + } + + /** + * Constructor for version 0. + * @param groupId + * @param offsetData + */ + @Deprecated + public OffsetCommitRequest(String groupId, Map offsetData) { + super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0))); + + initCommonFields(groupId, offsetData); + this.groupId = groupId; + this.generationId = DEFAULT_GENERATION_ID; + this.consumerId = DEFAULT_CONSUMER_ID; + this.retentionTime = DEFAULT_RETENTION_TIME; + this.offsetData = offsetData; + } + + /** + * Constructor for version 1. + * @param groupId + * @param generationId + * @param consumerId + * @param offsetData + */ + @Deprecated + public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { + super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1))); + + initCommonFields(groupId, offsetData); + struct.set(GENERATION_ID_KEY_NAME, generationId); + struct.set(CONSUMER_ID_KEY_NAME, consumerId); + this.groupId = groupId; + this.generationId = generationId; + this.consumerId = consumerId; + this.retentionTime = DEFAULT_RETENTION_TIME; + this.offsetData = offsetData; + } + + /** + * Constructor for version 2. + * @param groupId + * @param generationId + * @param consumerId + * @param retentionTime + * @param offsetData + */ + public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map offsetData) { + super(new Struct(CURRENT_SCHEMA)); + + initCommonFields(groupId, offsetData); + struct.set(GENERATION_ID_KEY_NAME, generationId); + struct.set(CONSUMER_ID_KEY_NAME, consumerId); + struct.set(RETENTION_TIME_KEY_NAME, retentionTime); + this.groupId = groupId; + this.generationId = generationId; + this.consumerId = consumerId; + this.retentionTime = retentionTime; + this.offsetData = offsetData; + } + + private void initCommonFields(String groupId, Map offsetData) { + Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); + + struct.set(GROUP_ID_KEY_NAME, groupId); + List topicArray = new ArrayList(); + + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); + // Only for v1 + if (partitionData.hasField(TIMESTAMP_KEY_NAME)) + partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); + partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + } + + public OffsetCommitRequest(Struct struct) { + super(struct); + + groupId = struct.getString(GROUP_ID_KEY_NAME); + // This field only exists in v1. + if (struct.hasField(GENERATION_ID_KEY_NAME)) + generationId = struct.getInt(GENERATION_ID_KEY_NAME); + else + generationId = DEFAULT_GENERATION_ID; + + // This field only exists in v1. + if (struct.hasField(CONSUMER_ID_KEY_NAME)) + consumerId = struct.getString(CONSUMER_ID_KEY_NAME); + else + consumerId = DEFAULT_CONSUMER_ID; + + // This field only exists in v2 + if (struct.hasField(RETENTION_TIME_KEY_NAME)) + retentionTime = struct.getLong(RETENTION_TIME_KEY_NAME); + else + retentionTime = DEFAULT_RETENTION_TIME; + + offsetData = new HashMap(); + for (Object topicDataObj : struct.getArray(TOPICS_KEY_NAME)) { + Struct topicData = (Struct) topicDataObj; + String topic = topicData.getString(TOPIC_KEY_NAME); + for (Object partitionDataObj : topicData.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionDataStruct = (Struct) partitionDataObj; + int partition = partitionDataStruct.getInt(PARTITION_KEY_NAME); + long offset = partitionDataStruct.getLong(COMMIT_OFFSET_KEY_NAME); + String metadata = partitionDataStruct.getString(METADATA_KEY_NAME); + PartitionData partitionOffset; + // This field only exists in v1 + if (partitionDataStruct.hasField(TIMESTAMP_KEY_NAME)) { + long timestamp = partitionDataStruct.getLong(TIMESTAMP_KEY_NAME); + partitionOffset = new PartitionData(offset, timestamp, metadata); + } else { + partitionOffset = new PartitionData(offset, metadata); + } + offsetData.put(new TopicPartition(topic, partition), partitionOffset); + } + } + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + Map responseData = new HashMap(); + for (Map.Entry entry: offsetData.entrySet()) { + responseData.put(entry.getKey(), Errors.forException(e).code()); + } + + switch (versionId) { + // OffsetCommitResponseV0 == OffsetCommitResponseV1 == OffsetCommitResponseV2 + case 0: + case 1: + case 2: + return new OffsetCommitResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_COMMIT.id))); + } + } + + public String groupId() { + return groupId; + } + + public int generationId() { + return generationId; + } + + public String consumerId() { + return consumerId; + } + + public long retentionTime() { + return retentionTime; + } + + public Map offsetData() { + return offsetData; + } + + public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) { + Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId); + return new OffsetCommitRequest((Struct) schema.read(buffer)); + } + + public static OffsetCommitRequest parse(ByteBuffer buffer) { + return new OffsetCommitRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java new file mode 100644 index 0000000000000..a1633330dc0c9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -0,0 +1,100 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class OffsetCommitResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id); + private static final String RESPONSES_KEY_NAME = "responses"; + + // topic level fields + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; + + // partition level fields + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * OFFSET_METADATA_TOO_LARGE (12) + * CONSUMER_COORDINATOR_NOT_AVAILABLE (15) + * NOT_COORDINATOR_FOR_CONSUMER (16) + * ILLEGAL_GENERATION (22) + * UNKNOWN_CONSUMER_ID (25) + * COMMITTING_PARTITIONS_NOT_ASSIGNED (27) + * INVALID_COMMIT_OFFSET_SIZE (28) + */ + + private final Map responseData; + + public OffsetCommitResponse(Map responseData) { + super(new Struct(CURRENT_SCHEMA)); + + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + + List topicArray = new ArrayList(); + for (Map.Entry> entries: topicsData.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entries.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : entries.getValue().entrySet()) { + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(ERROR_CODE_KEY_NAME, partitionEntry.getValue()); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + this.responseData = responseData; + } + + public OffsetCommitResponse(Struct struct) { + super(struct); + responseData = new HashMap(); + for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { + Struct topicResponse = (Struct) topicResponseObj; + String topic = topicResponse.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); + responseData.put(new TopicPartition(topic, partition), errorCode); + } + } + } + + public Map responseData() { + return responseData; + } + + public static OffsetCommitResponse parse(ByteBuffer buffer) { + return new OffsetCommitResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java new file mode 100644 index 0000000000000..6ee75973d6484 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -0,0 +1,123 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This wrapper supports both v0 and v1 of OffsetFetchRequest. + */ +public class OffsetFetchRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String TOPICS_KEY_NAME = "topics"; + + // topic level field names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + // partition level field names + private static final String PARTITION_KEY_NAME = "partition"; + + private final String groupId; + private final List partitions; + + public OffsetFetchRequest(String groupId, List partitions) { + super(new Struct(CURRENT_SCHEMA)); + + Map> topicsData = CollectionUtils.groupDataByTopic(partitions); + + struct.set(GROUP_ID_KEY_NAME, groupId); + List topicArray = new ArrayList(); + for (Map.Entry> entries: topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entries.getKey()); + List partitionArray = new ArrayList(); + for (Integer partiitonId : entries.getValue()) { + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partiitonId); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + this.groupId = groupId; + this.partitions = partitions; + } + + public OffsetFetchRequest(Struct struct) { + super(struct); + partitions = new ArrayList(); + for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { + Struct topicResponse = (Struct) topicResponseObj; + String topic = topicResponse.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + partitions.add(new TopicPartition(topic, partition)); + } + } + groupId = struct.getString(GROUP_ID_KEY_NAME); + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + Map responseData = new HashMap(); + + for (TopicPartition partition: partitions) { + responseData.put(partition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, + OffsetFetchResponse.NO_METADATA, + Errors.forException(e).code())); + } + + switch (versionId) { + // OffsetFetchResponseV0 == OffsetFetchResponseV1 + case 0: + case 1: + return new OffsetFetchResponse(responseData); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id))); + } + } + + public String groupId() { + return groupId; + } + + public List partitions() { + return partitions; + } + + public static OffsetFetchRequest parse(ByteBuffer buffer, int versionId) { + return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer)); + } + + public static OffsetFetchRequest parse(ByteBuffer buffer) { + return new OffsetFetchRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java new file mode 100644 index 0000000000000..3dc8521296e37 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -0,0 +1,126 @@ +/** + * 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 org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +public class OffsetFetchResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id); + private static final String RESPONSES_KEY_NAME = "responses"; + + // topic level fields + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; + + // partition level fields + private static final String PARTITION_KEY_NAME = "partition"; + private static final String COMMIT_OFFSET_KEY_NAME = "offset"; + private static final String METADATA_KEY_NAME = "metadata"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + + public static final long INVALID_OFFSET = -1L; + public static final String NO_METADATA = ""; + + /** + * Possible error code: + * + * UNKNOWN_TOPIC_OR_PARTITION (3) <- only for request v0 + * OFFSET_LOAD_IN_PROGRESS (14) + * NOT_COORDINATOR_FOR_CONSUMER (16) + * ILLEGAL_GENERATION (22) + * UNKNOWN_CONSUMER_ID (25) + */ + + private final Map responseData; + + public static final class PartitionData { + public final long offset; + public final String metadata; + public final short errorCode; + + public PartitionData(long offset, String metadata, short errorCode) { + this.offset = offset; + this.metadata = metadata; + this.errorCode = errorCode; + } + + public boolean hasError() { + return this.errorCode != Errors.NONE.code(); + } + } + + public OffsetFetchResponse(Map responseData) { + super(new Struct(CURRENT_SCHEMA)); + + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + + List topicArray = new ArrayList(); + for (Map.Entry> entries : topicsData.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entries.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : entries.getValue().entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); + partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); + partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + this.responseData = responseData; + } + + public OffsetFetchResponse(Struct struct) { + super(struct); + responseData = new HashMap(); + for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { + Struct topicResponse = (Struct) topicResponseObj; + String topic = topicResponse.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME); + String metadata = partitionResponse.getString(METADATA_KEY_NAME); + short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); + PartitionData partitionData = new PartitionData(offset, metadata, errorCode); + responseData.put(new TopicPartition(topic, partition), partitionData); + } + } + } + + public Map responseData() { + return responseData; + } + + public static OffsetFetchResponse parse(ByteBuffer buffer) { + return new OffsetFetchResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java new file mode 100644 index 0000000000000..715504b329506 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -0,0 +1,132 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ProduceRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id); + private static final String ACKS_KEY_NAME = "acks"; + private static final String TIMEOUT_KEY_NAME = "timeout"; + private static final String TOPIC_DATA_KEY_NAME = "topic_data"; + + // topic level field names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_DATA_KEY_NAME = "data"; + + // partition level field names + private static final String PARTITION_KEY_NAME = "partition"; + private static final String RECORD_SET_KEY_NAME = "record_set"; + + private final short acks; + private final int timeout; + private final Map partitionRecords; + + public ProduceRequest(short acks, int timeout, Map partitionRecords) { + super(new Struct(CURRENT_SCHEMA)); + Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); + struct.set(ACKS_KEY_NAME, acks); + struct.set(TIMEOUT_KEY_NAME, timeout); + List topicDatas = new ArrayList(recordsByTopic.size()); + for (Map.Entry> entry : recordsByTopic.entrySet()) { + Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entry.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : entry.getValue().entrySet()) { + ByteBuffer buffer = partitionEntry.getValue().duplicate(); + Struct part = topicData.instance(PARTITION_DATA_KEY_NAME) + .set(PARTITION_KEY_NAME, partitionEntry.getKey()) + .set(RECORD_SET_KEY_NAME, buffer); + partitionArray.add(part); + } + topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray()); + topicDatas.add(topicData); + } + struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray()); + this.acks = acks; + this.timeout = timeout; + this.partitionRecords = partitionRecords; + } + + public ProduceRequest(Struct struct) { + super(struct); + partitionRecords = new HashMap(); + for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) { + Struct topicData = (Struct) topicDataObj; + String topic = topicData.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicData.getArray(PARTITION_DATA_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + ByteBuffer records = partitionResponse.getBytes(RECORD_SET_KEY_NAME); + partitionRecords.put(new TopicPartition(topic, partition), records); + } + } + acks = struct.getShort(ACKS_KEY_NAME); + timeout = struct.getInt(TIMEOUT_KEY_NAME); + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + /* In case the producer doesn't actually want any response */ + if (acks == 0) + return null; + + Map responseMap = new HashMap(); + + for (Map.Entry entry : partitionRecords.entrySet()) { + responseMap.put(entry.getKey(), new ProduceResponse.PartitionResponse(Errors.forException(e).code(), ProduceResponse.INVALID_OFFSET)); + } + + switch (versionId) { + case 0: + return new ProduceResponse(responseMap); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id))); + } + } + + public short acks() { + return acks; + } + + public int timeout() { + return timeout; + } + + public Map partitionRecords() { + return partitionRecords; + } + + public static ProduceRequest parse(ByteBuffer buffer, int versionId) { + return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer)); + } + + public static ProduceRequest parse(ByteBuffer buffer) { + return new ProduceRequest((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java new file mode 100644 index 0000000000000..37ec0b79beafc --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -0,0 +1,122 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.utils.CollectionUtils; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ProduceResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id); + private static final String RESPONSES_KEY_NAME = "responses"; + + // topic level field names + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; + + // partition level field names + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + + public static final long INVALID_OFFSET = -1L; + + /** + * Possible error code: + * + * TODO + */ + + private static final String BASE_OFFSET_KEY_NAME = "base_offset"; + + private final Map responses; + + public ProduceResponse(Map responses) { + super(new Struct(CURRENT_SCHEMA)); + Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); + List topicDatas = new ArrayList(responseByTopic.size()); + for (Map.Entry> entry : responseByTopic.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entry.getKey()); + List partitionArray = new ArrayList(); + for (Map.Entry partitionEntry : entry.getValue().entrySet()) { + PartitionResponse part = partitionEntry.getValue(); + Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME) + .set(PARTITION_KEY_NAME, partitionEntry.getKey()) + .set(ERROR_CODE_KEY_NAME, part.errorCode) + .set(BASE_OFFSET_KEY_NAME, part.baseOffset); + partitionArray.add(partStruct); + } + topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray()); + topicDatas.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicDatas.toArray()); + this.responses = responses; + } + + public ProduceResponse(Struct struct) { + super(struct); + responses = new HashMap(); + for (Object topicResponse : struct.getArray("responses")) { + Struct topicRespStruct = (Struct) topicResponse; + String topic = topicRespStruct.getString("topic"); + for (Object partResponse : topicRespStruct.getArray("partition_responses")) { + Struct partRespStruct = (Struct) partResponse; + int partition = partRespStruct.getInt("partition"); + short errorCode = partRespStruct.getShort("error_code"); + long offset = partRespStruct.getLong("base_offset"); + TopicPartition tp = new TopicPartition(topic, partition); + responses.put(tp, new PartitionResponse(errorCode, offset)); + } + } + } + + public Map responses() { + return this.responses; + } + + public static final class PartitionResponse { + public short errorCode; + public long baseOffset; + + public PartitionResponse(short errorCode, long baseOffset) { + this.errorCode = errorCode; + this.baseOffset = baseOffset; + } + + @Override + public String toString() { + StringBuilder b = new StringBuilder(); + b.append('{'); + b.append("error: "); + b.append(errorCode); + b.append(",offset: "); + b.append(baseOffset); + b.append('}'); + return b.toString(); + } + } + + public static ProduceResponse parse(ByteBuffer buffer) { + return new ProduceResponse((Struct) CURRENT_SCHEMA.read(buffer)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java index 457abb1ad44ed..14bcde7d7b336 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.requests; @@ -25,22 +21,27 @@ import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Struct; - /** * The header for a request in the Kafka protocol */ -public class RequestHeader { +public class RequestHeader extends AbstractRequestResponse { - private static Field API_KEY_FIELD = REQUEST_HEADER.get("api_key"); - private static Field API_VERSION_FIELD = REQUEST_HEADER.get("api_version"); - private static Field CLIENT_ID_FIELD = REQUEST_HEADER.get("client_id"); - private static Field CORRELATION_ID_FIELD = REQUEST_HEADER.get("correlation_id"); + private static final Field API_KEY_FIELD = REQUEST_HEADER.get("api_key"); + private static final Field API_VERSION_FIELD = REQUEST_HEADER.get("api_version"); + private static final Field CLIENT_ID_FIELD = REQUEST_HEADER.get("client_id"); + private static final Field CORRELATION_ID_FIELD = REQUEST_HEADER.get("correlation_id"); - private final Struct header; + private final short apiKey; + private final short apiVersion; + private final String clientId; + private final int correlationId; public RequestHeader(Struct header) { - super(); - this.header = header; + super(header); + apiKey = struct.getShort(API_KEY_FIELD); + apiVersion = struct.getShort(API_VERSION_FIELD); + clientId = struct.getString(CLIENT_ID_FIELD); + correlationId = struct.getInt(CORRELATION_ID_FIELD); } public RequestHeader(short apiKey, String client, int correlation) { @@ -48,38 +49,34 @@ public RequestHeader(short apiKey, String client, int correlation) { } public RequestHeader(short apiKey, short version, String client, int correlation) { - this(new Struct(Protocol.REQUEST_HEADER)); - this.header.set(API_KEY_FIELD, apiKey); - this.header.set(API_VERSION_FIELD, version); - this.header.set(CLIENT_ID_FIELD, client); - this.header.set(CORRELATION_ID_FIELD, correlation); + super(new Struct(Protocol.REQUEST_HEADER)); + struct.set(API_KEY_FIELD, apiKey); + struct.set(API_VERSION_FIELD, version); + struct.set(CLIENT_ID_FIELD, client); + struct.set(CORRELATION_ID_FIELD, correlation); + this.apiKey = apiKey; + this.apiVersion = version; + this.clientId = client; + this.correlationId = correlation; } public short apiKey() { - return (Short) this.header.get(API_KEY_FIELD); + return apiKey; } public short apiVersion() { - return (Short) this.header.get(API_VERSION_FIELD); + return apiVersion; } public String clientId() { - return (String) this.header.get(CLIENT_ID_FIELD); + return clientId; } public int correlationId() { - return (Integer) this.header.get(CORRELATION_ID_FIELD); + return correlationId; } public static RequestHeader parse(ByteBuffer buffer) { return new RequestHeader((Struct) Protocol.REQUEST_HEADER.read(buffer)); } - - public void writeTo(ByteBuffer buffer) { - header.writeTo(buffer); - } - - public int sizeOf() { - return header.sizeOf(); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java index c5e9020b2e769..3fec60beda761 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.requests; @@ -21,7 +17,6 @@ import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.protocol.types.Struct; - /** * A send object for a kafka request */ @@ -30,7 +25,7 @@ public class RequestSend extends NetworkSend { private final RequestHeader header; private final Struct body; - public RequestSend(int destination, RequestHeader header, Struct body) { + public RequestSend(String destination, RequestHeader header, Struct body) { super(destination, serialize(header, body)); this.header = header; this.body = body; @@ -52,4 +47,9 @@ public Struct body() { return body; } + @Override + public String toString() { + return "RequestSend(header=" + header.toString() + ", body=" + body.toString() + ")"; + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java index 257b8287757e4..e8a7ef9a0b464 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java @@ -28,35 +28,29 @@ /** * A response header in the kafka protocol. */ -public class ResponseHeader { +public class ResponseHeader extends AbstractRequestResponse { - private static Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id"); + private static final Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id"); - private final Struct header; + private final int correlationId; public ResponseHeader(Struct header) { - this.header = header; + super(header); + correlationId = struct.getInt(CORRELATION_KEY_FIELD); } public ResponseHeader(int correlationId) { - this(new Struct(Protocol.RESPONSE_HEADER)); - this.header.set(CORRELATION_KEY_FIELD, correlationId); + super(new Struct(Protocol.RESPONSE_HEADER)); + struct.set(CORRELATION_KEY_FIELD, correlationId); + this.correlationId = correlationId; } public int correlationId() { - return (Integer) header.get(CORRELATION_KEY_FIELD); - } - - public void writeTo(ByteBuffer buffer) { - header.writeTo(buffer); - } - - public int sizeOf() { - return header.sizeOf(); + return correlationId; } public static ResponseHeader parse(ByteBuffer buffer) { - return new ResponseHeader(((Struct) Protocol.RESPONSE_HEADER.read(buffer))); + return new ResponseHeader((Struct) Protocol.RESPONSE_HEADER.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java new file mode 100644 index 0000000000000..12b06d16e0380 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java @@ -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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.network.NetworkSend; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + +public class ResponseSend extends NetworkSend { + + public ResponseSend(String destination, ResponseHeader header, Struct body) { + super(destination, serialize(header, body)); + } + + public ResponseSend(String destination, ResponseHeader header, AbstractRequestResponse response) { + this(destination, header, response.toStruct()); + } + + private static ByteBuffer serialize(ResponseHeader header, Struct body) { + ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf()); + header.writeTo(buffer); + body.writeTo(buffer); + buffer.rewind(); + return buffer; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java new file mode 100644 index 0000000000000..d89b3ff0509eb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java @@ -0,0 +1,34 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import java.util.Map; + +public class ByteArrayDeserializer implements Deserializer { + + @Override + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + @Override + public byte[] deserialize(String topic, byte[] data) { + return data; + } + + @Override + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java new file mode 100644 index 0000000000000..beaef948e38f2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java @@ -0,0 +1,34 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import java.util.Map; + +public class ByteArraySerializer implements Serializer { + + @Override + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + @Override + public byte[] serialize(String topic, byte[] data) { + return data; + } + + @Override + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java new file mode 100644 index 0000000000000..254b5562da15d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.common.serialization; + +import java.io.Closeable; +import java.util.Map; + +/** + * + * @param Type to be deserialized into. + * + * A class that implements this interface is expected to have a constructor with no parameter. + */ +public interface Deserializer extends Closeable { + + /** + * Configure this class. + * @param configs configs in key/value pairs + * @param isKey whether is for key or value + */ + public void configure(Map configs, boolean isKey); + + /** + * + * @param topic topic associated with the data + * @param data serialized bytes + * @return deserialized typed data + */ + public T deserialize(String topic, byte[] data); + + @Override + public void close(); +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/IntegerDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/IntegerDeserializer.java new file mode 100644 index 0000000000000..0b5a58d858401 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/IntegerDeserializer.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.serialization; + +import org.apache.kafka.common.errors.SerializationException; + +import java.util.Map; + +public class IntegerDeserializer implements Deserializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public Integer deserialize(String topic, byte[] data) { + if (data == null) + return null; + if (data.length != 4) { + throw new SerializationException("Size of data received by IntegerDeserializer is " + + "not 4"); + } + + int value = 0; + for (byte b : data) { + value <<= 8; + value |= b & 0xFF; + } + return value; + } + + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/IntegerSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/IntegerSerializer.java new file mode 100644 index 0000000000000..578bdd2f673a9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/IntegerSerializer.java @@ -0,0 +1,38 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import java.util.Map; + +public class IntegerSerializer implements Serializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public byte[] serialize(String topic, Integer data) { + if (data == null) + return null; + + return new byte[] { + (byte) (data >>> 24), + (byte) (data >>> 16), + (byte) (data >>> 8), + data.byteValue() + }; + } + + public void close() { + // nothing to do + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java new file mode 100644 index 0000000000000..88033b0817cc9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java @@ -0,0 +1,49 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import java.io.Closeable; +import java.util.Map; + +/** + * + * @param Type to be serialized from. + * + * A class that implements this interface is expected to have a constructor with no parameter. + */ +public interface Serializer extends Closeable { + + /** + * Configure this class. + * @param configs configs in key/value pairs + * @param isKey whether is for key or value + */ + public void configure(Map configs, boolean isKey); + + /** + * @param topic topic associated with data + * @param data typed data + * @return serialized bytes + */ + public byte[] serialize(String topic, T data); + + + /** + * Close this serializer. + * This method has to be idempotent if the serializer is used in KafkaProducer because it might be called + * multiple times. + */ + @Override + public void close(); +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java new file mode 100644 index 0000000000000..9783ea0c53110 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java @@ -0,0 +1,53 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import org.apache.kafka.common.errors.SerializationException; + +import java.io.UnsupportedEncodingException; +import java.util.Map; + +/** + * String encoding defaults to UTF8 and can be customized by setting the property key.deserializer.encoding, + * value.deserializer.encoding or deserializer.encoding. The first two take precedence over the last. + */ +public class StringDeserializer implements Deserializer { + private String encoding = "UTF8"; + + @Override + public void configure(Map configs, boolean isKey) { + String propertyName = isKey ? "key.deserializer.encoding" : "value.deserializer.encoding"; + Object encodingValue = configs.get(propertyName); + if (encodingValue == null) + encodingValue = configs.get("deserializer.encoding"); + if (encodingValue != null && encodingValue instanceof String) + encoding = (String) encodingValue; + } + + @Override + public String deserialize(String topic, byte[] data) { + try { + if (data == null) + return null; + else + return new String(data, encoding); + } catch (UnsupportedEncodingException e) { + throw new SerializationException("Error when deserializing byte[] to string due to unsupported encoding " + encoding); + } + } + + @Override + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java new file mode 100644 index 0000000000000..636d905ffc729 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java @@ -0,0 +1,53 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import org.apache.kafka.common.errors.SerializationException; + +import java.io.UnsupportedEncodingException; +import java.util.Map; + +/** + * String encoding defaults to UTF8 and can be customized by setting the property key.serializer.encoding, + * value.serializer.encoding or serializer.encoding. The first two take precedence over the last. + */ +public class StringSerializer implements Serializer { + private String encoding = "UTF8"; + + @Override + public void configure(Map configs, boolean isKey) { + String propertyName = isKey ? "key.serializer.encoding" : "value.serializer.encoding"; + Object encodingValue = configs.get(propertyName); + if (encodingValue == null) + encodingValue = configs.get("serializer.encoding"); + if (encodingValue != null && encodingValue instanceof String) + encoding = (String) encodingValue; + } + + @Override + public byte[] serialize(String topic, String data) { + try { + if (data == null) + return null; + else + return data.getBytes(encoding); + } catch (UnsupportedEncodingException e) { + throw new SerializationException("Error when serializing string to byte[] due to unsupported encoding " + encoding); + } + } + + @Override + public void close() { + // nothing to do + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java new file mode 100644 index 0000000000000..ba3863734391b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java @@ -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 org.apache.kafka.common.utils; + +import org.apache.kafka.common.TopicPartition; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class CollectionUtils { + /** + * group data by topic + * @param data Data to be partitioned + * @param Partition data type + * @return partitioned data + */ + public static Map> groupDataByTopic(Map data) { + Map> dataByTopic = new HashMap>(); + for (Map.Entry entry: data.entrySet()) { + String topic = entry.getKey().topic(); + int partition = entry.getKey().partition(); + Map topicData = dataByTopic.get(topic); + if (topicData == null) { + topicData = new HashMap(); + dataByTopic.put(topic, topicData); + } + topicData.put(partition, entry.getValue()); + } + return dataByTopic; + } + + /** + * group partitions by topic + * @param partitions + * @return partitions per topic + */ + public static Map> groupDataByTopic(List partitions) { + Map> partitionsByTopic = new HashMap>(); + for (TopicPartition tp: partitions) { + String topic = tp.topic(); + List topicData = partitionsByTopic.get(topic); + if (topicData == null) { + topicData = new ArrayList(); + partitionsByTopic.put(topic, topicData); + } + topicData.add(tp.partition()); + } + return partitionsByTopic; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java b/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java index 187d22fba5fe2..9c0e81ad80669 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.utils; @@ -24,7 +20,7 @@ import java.util.concurrent.ConcurrentMap; /** - * A simple read-optimized map implementation that synchronizes only writes and does a fully copy on each modification + * A simple read-optimized map implementation that synchronizes only writes and does a full copy on each modification */ public class CopyOnWriteMap implements ConcurrentMap { diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java index 153c5a6d34529..5b867001065df 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java @@ -28,6 +28,30 @@ */ public class Crc32 implements Checksum { + /** + * Compute the CRC32 of the byte array + * + * @param bytes The array to compute the checksum for + * @return The CRC32 + */ + public static long crc32(byte[] bytes) { + return crc32(bytes, 0, bytes.length); + } + + /** + * Compute the CRC32 of the segment of the byte array given by the specified size and offset + * + * @param bytes The bytes to checksum + * @param offset the offset at which to begin checksumming + * @param size the number of bytes to checksum + * @return The CRC32 + */ + public static long crc32(byte[] bytes, int offset, int size) { + Crc32 crc = new Crc32(); + crc.update(bytes, offset, size); + return crc.getValue(); + } + /** the current CRC value, bit-flipped */ private int crc; @@ -55,14 +79,14 @@ public void update(byte[] b, int off, int len) { 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]); + 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]); + 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; @@ -71,19 +95,19 @@ public void update(byte[] b, int off, int len) { /* loop unroll - duff's device style */ switch (len) { case 7: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 6: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 5: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 4: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 3: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 2: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 1: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; default: /* nothing */ } @@ -94,2076 +118,270 @@ public void update(byte[] b, int off, int len) { @Override final public void update(int b) { - crc = (crc >>> 8) ^ T[T8_0_start + ((crc ^ b) & 0xff)]; + crc = (crc >>> 8) ^ T[T8_0_START + ((crc ^ b) & 0xff)]; + } + + /** + * Update the CRC32 given an integer + */ + final public void updateInt(int input) { + update((byte) (input >> 24)); + update((byte) (input >> 16)); + update((byte) (input >> 8)); + update((byte) input /* >> 0 */); } /* * 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 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 }; + /* 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/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java b/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java index 9ff793f38d70d..57247c85ba13b 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java @@ -1,32 +1,33 @@ /** - * 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. + * 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 org.apache.kafka.common.utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * A wrapper for Thread that sets things up nicely */ public class KafkaThread extends Thread { - public KafkaThread(String name, Runnable runnable, boolean daemon) { + private final Logger log = LoggerFactory.getLogger(getClass()); + + public KafkaThread(final String name, Runnable runnable, boolean daemon) { super(runnable, name); setDaemon(daemon); setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { public void uncaughtException(Thread t, Throwable e) { - e.printStackTrace(); + log.error("Uncaught exception in " + name + ": ", e); } }); } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java index 6582c73dab634..18725de836838 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java @@ -26,6 +26,7 @@ public long milliseconds() { return System.currentTimeMillis(); } + @Override public long nanoseconds() { return System.nanoTime(); } @@ -35,7 +36,7 @@ public void sleep(long ms) { try { Thread.sleep(ms); } catch (InterruptedException e) { - // no stress + // just wake up early } } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java old mode 100644 new mode 100755 index 9c34e7dc82f33..af9993cf9b399 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -1,29 +1,51 @@ /** - * 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. + * 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 org.apache.kafka.common.utils; +import java.io.IOException; +import java.io.InputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.OutputStream; import java.io.UnsupportedEncodingException; +import java.io.FileNotFoundException; +import java.io.StringWriter; +import java.io.PrintWriter; import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.Properties; +import java.nio.channels.FileChannel; +import java.nio.charset.Charset; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.kafka.common.KafkaException; - public class Utils { + // This matches URIs of formats: host:port and protocol:\\host:port + // IPv6 is supported with [ip] pattern + private static final Pattern HOST_PORT_PATTERN = Pattern.compile(".*?\\[?([0-9a-z\\-.:]*)\\]?:([0-9]+)"); + + public static final String NL = System.getProperty("line.separator"); + + private static final Logger log = LoggerFactory.getLogger(Utils.class); + /** * Turn the given UTF8 byte array into a string * @@ -73,6 +95,34 @@ public static long readUnsignedInt(ByteBuffer buffer, int index) { return buffer.getInt(index) & 0xffffffffL; } + /** + * Read an unsigned integer stored in little-endian format from the {@link InputStream}. + * + * @param in The stream to read from + * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) + */ + public static int readUnsignedIntLE(InputStream in) throws IOException { + return (in.read() << 8 * 0) + | (in.read() << 8 * 1) + | (in.read() << 8 * 2) + | (in.read() << 8 * 3); + } + + /** + * Read an unsigned integer stored in little-endian format from a byte array + * at a given offset. + * + * @param buffer The byte array to read from + * @param offset The position in buffer to read from + * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) + */ + public static int readUnsignedIntLE(byte[] buffer, int offset) { + return (buffer[offset++] << 8 * 0) + | (buffer[offset++] << 8 * 1) + | (buffer[offset++] << 8 * 2) + | (buffer[offset] << 8 * 3); + } + /** * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. * @@ -95,35 +145,55 @@ public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) { } /** - * Compute the CRC32 of the byte array + * Write an unsigned integer in little-endian format to the {@link OutputStream}. * - * @param bytes The array to compute the checksum for - * @return The CRC32 + * @param out The stream to write to + * @param value The value to write */ - public static long crc32(byte[] bytes) { - return crc32(bytes, 0, bytes.length); + public static void writeUnsignedIntLE(OutputStream out, int value) throws IOException { + out.write(value >>> 8 * 0); + out.write(value >>> 8 * 1); + out.write(value >>> 8 * 2); + out.write(value >>> 8 * 3); } /** - * Compute the CRC32 of the segment of the byte array given by the specificed size and offset + * Write an unsigned integer in little-endian format to a byte array + * at a given offset. * - * @param bytes The bytes to checksum - * @param offset the offset at which to begin checksumming - * @param size the number of bytes to checksum - * @return The CRC32 + * @param buffer The byte array to write to + * @param offset The position in buffer to write to + * @param value The value to write */ - public static long crc32(byte[] bytes, int offset, int size) { - Crc32 crc = new Crc32(); - crc.update(bytes, offset, size); - return crc.getValue(); + public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) { + buffer[offset++] = (byte) (value >>> 8 * 0); + buffer[offset++] = (byte) (value >>> 8 * 1); + buffer[offset++] = (byte) (value >>> 8 * 2); + buffer[offset] = (byte) (value >>> 8 * 3); } + /** * Get the absolute value of the given number. If the number is Int.MinValue return 0. This is different from * java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!). */ public static int abs(int n) { - return n & 0x7fffffff; + return (n == Integer.MIN_VALUE) ? 0 : Math.abs(n); + } + + /** + * Get the minimum of some long values. + * @param first Used to ensure at least one value + * @param rest The rest of longs to compare + * @return The minimum of all passed argument. + */ + public static long min(long first, long ... rest) { + long min = first; + for (int i = 0; i < rest.length; i++) { + if (rest[i] < min) + min = rest[i]; + } + return min; } /** @@ -186,6 +256,18 @@ public static T notNull(T t) { return t; } + /** + * Sleep for a bit + * @param ms The duration of the sleep + */ + public static void sleep(long ms) { + try { + Thread.sleep(ms); + } catch (InterruptedException e) { + // this is okay, we just wake up early + } + } + /** * Instantiate the class */ @@ -233,7 +315,7 @@ public static int murmur2(final byte[] data) { case 2: h ^= (data[(length & ~3) + 1] & 0xff) << 8; case 1: - h ^= (data[length & ~3] & 0xff); + h ^= data[length & ~3] & 0xff; h *= m; } @@ -244,4 +326,172 @@ public static int murmur2(final byte[] data) { return h; } + /** + * Extracts the hostname from a "host:port" address string. + * @param address address string to parse + * @return hostname or null if the given address is incorrect + */ + public static String getHost(String address) { + Matcher matcher = HOST_PORT_PATTERN.matcher(address); + return matcher.matches() ? matcher.group(1) : null; + } + + /** + * Extracts the port number from a "host:port" address string. + * @param address address string to parse + * @return port number or null if the given address is incorrect + */ + public static Integer getPort(String address) { + Matcher matcher = HOST_PORT_PATTERN.matcher(address); + return matcher.matches() ? Integer.parseInt(matcher.group(2)) : null; + } + + /** + * Formats hostname and port number as a "host:port" address string, + * surrounding IPv6 addresses with braces '[', ']' + * @param host hostname + * @param port port number + * @return address string + */ + public static String formatAddress(String host, Integer port) { + return host.contains(":") + ? "[" + host + "]:" + port // IPv6 + : host + ":" + port; + } + + /** + * Create a string representation of an array joined by the given separator + * @param strs The array of items + * @param seperator The separator + * @return The string representation. + */ + public static String join(T[] strs, String seperator) { + return join(Arrays.asList(strs), seperator); + } + + /** + * Create a string representation of a list joined by the given separator + * @param list The list of items + * @param seperator The separator + * @return The string representation. + */ + public static String join(Collection list, String seperator) { + StringBuilder sb = new StringBuilder(); + Iterator iter = list.iterator(); + while (iter.hasNext()) { + sb.append(iter.next()); + if (iter.hasNext()) + sb.append(seperator); + } + return sb.toString(); + } + + /** + * Read a properties file from the given path + * @param filename The path of the file to read + */ + public static Properties loadProps(String filename) throws IOException, FileNotFoundException { + Properties props = new Properties(); + InputStream propStream = null; + try { + propStream = new FileInputStream(filename); + props.load(propStream); + } finally { + if (propStream != null) + propStream.close(); + } + return props; + } + + /** + * Get the stack trace from an exception as a string + */ + public static String stackTrace(Throwable e) { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + e.printStackTrace(pw); + return sw.toString(); + } + + /** + * Create a new thread + * @param name The name of the thread + * @param runnable The work for the thread to do + * @param daemon Should the thread block JVM shutdown? + * @return The unstarted thread + */ + public static Thread newThread(String name, Runnable runnable, Boolean daemon) { + Thread thread = new Thread(runnable, name); + thread.setDaemon(daemon); + thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { + public void uncaughtException(Thread t, Throwable e) { + log.error("Uncaught exception in thread '" + t.getName() + "':", e); + } + }); + return thread; + } + + /** + * Create a daemon thread + * @param name The name of the thread + * @param runnable The runnable to execute in the background + * @return The unstarted thread + */ + public static Thread daemonThread(String name, Runnable runnable) { + return newThread(name, runnable, true); + } + + /** + * Print an error message and shutdown the JVM + * @param message The error message + */ + public static void croak(String message) { + System.err.println(message); + System.exit(1); + } + + /** + * Read a buffer into a Byte array for the given offset and length + */ + public static byte[] readBytes(ByteBuffer buffer, int offset, int length) { + byte[] dest = new byte[length]; + if (buffer.hasArray()) { + System.arraycopy(buffer.array(), buffer.arrayOffset() + offset, dest, 0, length); + } else { + buffer.mark(); + buffer.position(offset); + buffer.get(dest, 0, length); + buffer.reset(); + } + return dest; + } + + /** + * Read the given byte buffer into a Byte array + */ + public static byte[] readBytes(ByteBuffer buffer) { + return Utils.readBytes(buffer, 0, buffer.limit()); + } + + /** + * Attempt to read a file as a string + * @throws IOException + */ + public static String readFileAsString(String path, Charset charset) throws IOException { + if (charset == null) charset = Charset.defaultCharset(); + FileInputStream stream = new FileInputStream(new File(path)); + String result = new String(); + try { + FileChannel fc = stream.getChannel(); + MappedByteBuffer bb = fc.map(FileChannel.MapMode.READ_ONLY, 0, fc.size()); + result = charset.decode(bb).toString(); + } finally { + stream.close(); + } + return result; + } + + public static String readFileAsString(String path) throws IOException { + return Utils.readFileAsString(path, Charset.defaultCharset()); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java new file mode 100644 index 0000000000000..13ce519f03d13 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java @@ -0,0 +1,42 @@ +/** + * 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 org.apache.kafka.clients; + +import org.apache.kafka.common.config.ConfigException; +import org.junit.Test; + +import java.util.Arrays; + +public class ClientUtilsTest { + + @Test + public void testParseAndValidateAddresses() { + check("127.0.0.1:8000"); + check("mydomain.com:8080"); + check("[::1]:8000"); + check("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:1234", "mydomain.com:10000"); + } + + @Test(expected = ConfigException.class) + public void testNoPort() { + check("127.0.0.1"); + } + + private void check(String... url) { + ClientUtils.parseAndValidateAddresses(Arrays.asList(url)); + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java new file mode 100644 index 0000000000000..249d6b8a65745 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -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 org.apache.kafka.clients; + +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class MetadataTest { + + private long refreshBackoffMs = 100; + private long metadataExpireMs = 1000; + private Metadata metadata = new Metadata(refreshBackoffMs, metadataExpireMs); + private AtomicBoolean backgroundError = new AtomicBoolean(false); + + @After + public void tearDown() { + assertFalse(backgroundError.get()); + } + + @Test + public void testMetadata() throws Exception { + long time = 0; + metadata.update(Cluster.empty(), time); + assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0); + metadata.requestUpdate(); + assertFalse("Still no updated needed due to backoff", metadata.timeToNextUpdate(time) == 0); + time += refreshBackoffMs; + assertTrue("Update needed now that backoff time expired", metadata.timeToNextUpdate(time) == 0); + String topic = "my-topic"; + Thread t1 = asyncFetch(topic); + Thread t2 = asyncFetch(topic); + assertTrue("Awaiting update", t1.isAlive()); + assertTrue("Awaiting update", t2.isAlive()); + metadata.update(TestUtils.singletonCluster(topic, 1), time); + t1.join(); + t2.join(); + assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0); + time += metadataExpireMs; + assertTrue("Update needed due to stale metadata.", metadata.timeToNextUpdate(time) == 0); + } + + /** + * Tests that {@link org.apache.kafka.clients.Metadata#awaitUpdate(int, long)} doesn't + * wait forever with a max timeout value of 0 + * + * @throws Exception + * @see https://issues.apache.org/jira/browse/KAFKA-1836 + */ + @Test + public void testMetadataUpdateWaitTime() throws Exception { + long time = 0; + metadata.update(Cluster.empty(), time); + assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0); + // first try with a max wait time of 0 and ensure that this returns back without waiting forever + try { + metadata.awaitUpdate(metadata.requestUpdate(), 0); + fail("Wait on metadata update was expected to timeout, but it didn't"); + } catch (TimeoutException te) { + // expected + } + // now try with a higher timeout value once + final long twoSecondWait = 2000; + try { + metadata.awaitUpdate(metadata.requestUpdate(), twoSecondWait); + fail("Wait on metadata update was expected to timeout, but it didn't"); + } catch (TimeoutException te) { + // expected + } + } + + @Test + public void testFailedUpdate() { + long time = 100; + metadata.update(Cluster.empty(), time); + + assertEquals(100, metadata.timeToNextUpdate(1000)); + metadata.failedUpdate(1100); + + assertEquals(100, metadata.timeToNextUpdate(1100)); + assertEquals(100, metadata.lastSuccessfulUpdate()); + + } + + + private Thread asyncFetch(final String topic) { + Thread thread = new Thread() { + public void run() { + while (metadata.fetch().partitionsForTopic(topic) == null) { + try { + metadata.awaitUpdate(metadata.requestUpdate(), refreshBackoffMs); + } catch (Exception e) { + backgroundError.set(true); + } + } + } + }; + thread.start(); + return thread; + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java new file mode 100644 index 0000000000000..d9c97e966c0e2 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -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 org.apache.kafka.clients; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Queue; +import java.util.Set; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.utils.Time; + +/** + * A mock network client for use testing code + */ +public class MockClient implements KafkaClient { + + private class FutureResponse { + public final Struct responseBody; + public final boolean disconnected; + + public FutureResponse(Struct responseBody, boolean disconnected) { + this.responseBody = responseBody; + this.disconnected = disconnected; + } + } + + private final Time time; + private int correlation = 0; + private Node node = null; + private final Set ready = new HashSet(); + private final Queue requests = new ArrayDeque(); + private final Queue responses = new ArrayDeque(); + private final Queue futureResponses = new ArrayDeque(); + + public MockClient(Time time) { + this.time = time; + } + + @Override + public boolean isReady(Node node, long now) { + return ready.contains(node.id()); + } + + @Override + public boolean ready(Node node, long now) { + ready.add(node.id()); + return true; + } + + @Override + public long connectionDelay(Node node, long now) { + return 0; + } + + @Override + public boolean connectionFailed(Node node) { + return false; + } + + public void disconnect(String node) { + Iterator iter = requests.iterator(); + while (iter.hasNext()) { + ClientRequest request = iter.next(); + if (request.request().destination() == node) { + responses.add(new ClientResponse(request, time.milliseconds(), true, null)); + iter.remove(); + } + } + ready.remove(node); + } + + @Override + public void send(ClientRequest request) { + if (!futureResponses.isEmpty()) { + FutureResponse futureResp = futureResponses.poll(); + ClientResponse resp = new ClientResponse(request, time.milliseconds(), futureResp.disconnected, futureResp.responseBody); + responses.add(resp); + } else { + this.requests.add(request); + } + } + + @Override + public List poll(long timeoutMs, long now) { + List copy = new ArrayList(this.responses); + + while (!this.responses.isEmpty()) { + ClientResponse response = this.responses.poll(); + if (response.request().hasCallback()) + response.request().callback().onComplete(response); + } + + return copy; + } + + @Override + public List completeAll(String node, long now) { + return completeAll(now); + } + + @Override + public List completeAll(long now) { + List responses = poll(0, now); + if (requests.size() > 0) + throw new IllegalStateException("Requests without responses remain."); + return responses; + } + + public Queue requests() { + return this.requests; + } + + public void respond(Struct body) { + respond(body, false); + } + + public void respond(Struct body, boolean disconnected) { + ClientRequest request = requests.remove(); + responses.add(new ClientResponse(request, time.milliseconds(), disconnected, body)); + } + + public void prepareResponse(Struct body) { + prepareResponse(body, false); + } + + public void prepareResponse(Struct body, boolean disconnected) { + futureResponses.add(new FutureResponse(body, disconnected)); + } + + public void setNode(Node node) { + this.node = node; + } + + @Override + public int inFlightRequestCount() { + return requests.size(); + } + + @Override + public int inFlightRequestCount(String nodeId) { + return requests.size(); + } + + @Override + public RequestHeader nextRequestHeader(ApiKeys key) { + return new RequestHeader(key.id, "mock", correlation++); + } + + @Override + public void wakeup() { + } + + @Override + public void close() { + } + + @Override + public Node leastLoadedNode(long now) { + return this.node; + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java new file mode 100644 index 0000000000000..43238ceaad032 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -0,0 +1,127 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.network.NetworkReceive; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.ProduceRequest; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.requests.ResponseHeader; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.MockSelector; +import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; + +public class NetworkClientTest { + + private MockTime time = new MockTime(); + private MockSelector selector = new MockSelector(time); + private Metadata metadata = new Metadata(0, Long.MAX_VALUE); + private int nodeId = 1; + private Cluster cluster = TestUtils.singletonCluster("test", nodeId); + private Node node = cluster.nodes().get(0); + private NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024); + + @Before + public void setup() { + metadata.update(cluster, time.milliseconds()); + } + + @Test + public void testReadyAndDisconnect() { + assertFalse("Client begins unready as it has no connection.", client.ready(node, time.milliseconds())); + assertEquals("The connection is established as a side-effect of the readiness check", 1, selector.connected().size()); + client.poll(1, time.milliseconds()); + selector.clear(); + assertTrue("Now the client is ready", client.ready(node, time.milliseconds())); + selector.disconnect(node.idString()); + client.poll(1, time.milliseconds()); + selector.clear(); + assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds())); + assertTrue("Metadata should get updated.", metadata.timeToNextUpdate(time.milliseconds()) == 0); + } + + @Test(expected = IllegalStateException.class) + public void testSendToUnreadyNode() { + RequestSend send = new RequestSend("5", + client.nextRequestHeader(ApiKeys.METADATA), + new MetadataRequest(Arrays.asList("test")).toStruct()); + ClientRequest request = new ClientRequest(time.milliseconds(), false, send, null); + client.send(request); + client.poll(1, time.milliseconds()); + } + + @Test + public void testSimpleRequestResponse() { + ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.emptyMap()); + RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE); + RequestSend send = new RequestSend(node.idString(), reqHeader, produceRequest.toStruct()); + TestCallbackHandler handler = new TestCallbackHandler(); + ClientRequest request = new ClientRequest(time.milliseconds(), true, send, handler); + awaitReady(client, node); + client.send(request); + client.poll(1, time.milliseconds()); + assertEquals(1, client.inFlightRequestCount()); + ResponseHeader respHeader = new ResponseHeader(reqHeader.correlationId()); + Struct resp = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); + resp.set("responses", new Object[0]); + int size = respHeader.sizeOf() + resp.sizeOf(); + ByteBuffer buffer = ByteBuffer.allocate(size); + respHeader.writeTo(buffer); + resp.writeTo(buffer); + buffer.flip(); + selector.completeReceive(new NetworkReceive(node.idString(), buffer)); + List responses = client.poll(1, time.milliseconds()); + assertEquals(1, responses.size()); + assertTrue("The handler should have executed.", handler.executed); + assertTrue("Should have a response body.", handler.response.hasResponse()); + assertEquals("Should be correlated to the original request", request, handler.response.request()); + } + + private void awaitReady(NetworkClient client, Node node) { + while (!client.ready(node, time.milliseconds())) + client.poll(1, time.milliseconds()); + } + + private static class TestCallbackHandler implements RequestCompletionHandler { + public boolean executed = false; + public ClientResponse response; + + public void onComplete(ClientResponse response) { + this.executed = true; + this.response = response; + } + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java new file mode 100644 index 0000000000000..738f3ed0757cf --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -0,0 +1,49 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.test.MockMetricsReporter; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Properties; + +public class KafkaConsumerTest { + + @Test + public void testConstructorClose() throws Exception { + Properties props = new Properties(); + props.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testConstructorClose"); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "some.invalid.hostname.foo.bar:9999"); + props.setProperty(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + + final int oldInitCount = MockMetricsReporter.INIT_COUNT.get(); + final int oldCloseCount = MockMetricsReporter.CLOSE_COUNT.get(); + try { + KafkaConsumer consumer = new KafkaConsumer( + props, null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } catch (KafkaException e) { + Assert.assertEquals(oldInitCount + 1, MockMetricsReporter.INIT_COUNT.get()); + Assert.assertEquals(oldCloseCount + 1, MockMetricsReporter.CLOSE_COUNT.get()); + Assert.assertEquals("Failed to construct kafka consumer", e.getMessage()); + return; + } + Assert.fail("should have caught an exception and returned"); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java new file mode 100644 index 0000000000000..26b6b409843b3 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -0,0 +1,48 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import static org.junit.Assert.*; + +import java.util.Iterator; + +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +public class MockConsumerTest { + + private MockConsumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST); + + @Test + public void testSimpleMock() { + consumer.subscribe("topic"); + assertEquals(0, consumer.poll(1000).count()); + ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, "key1", "value1"); + ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, "key2", "value2"); + consumer.addRecord(rec1); + consumer.addRecord(rec2); + ConsumerRecords recs = consumer.poll(1); + Iterator> iter = recs.iterator(); + assertEquals(rec1, iter.next()); + assertEquals(rec2, iter.next()); + assertFalse(iter.hasNext()); + assertEquals(1L, consumer.position(new TopicPartition("test", 0))); + consumer.commit(CommitType.SYNC); + assertEquals(1L, consumer.committed(new TopicPartition("test", 0))); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java new file mode 100644 index 0000000000000..9de1cee7bab9e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java @@ -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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.ConsumerWakeupException; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.HeartbeatRequest; +import org.apache.kafka.common.requests.HeartbeatResponse; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestUtils; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class ConsumerNetworkClientTest { + + private String topicName = "test"; + private MockTime time = new MockTime(); + private MockClient client = new MockClient(time); + private Cluster cluster = TestUtils.singletonCluster(topicName, 1); + private Node node = cluster.nodes().get(0); + private Metadata metadata = new Metadata(0, Long.MAX_VALUE); + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + + @Test + public void send() { + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + RequestFuture future = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + assertEquals(1, consumerClient.pendingRequestCount()); + assertEquals(1, consumerClient.pendingRequestCount(node)); + assertFalse(future.isDone()); + + consumerClient.poll(future); + assertTrue(future.isDone()); + assertTrue(future.succeeded()); + + ClientResponse clientResponse = future.value(); + HeartbeatResponse response = new HeartbeatResponse(clientResponse.responseBody()); + assertEquals(Errors.NONE.code(), response.errorCode()); + } + + @Test + public void multiSend() { + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + RequestFuture future1 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + RequestFuture future2 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + assertEquals(2, consumerClient.pendingRequestCount()); + assertEquals(2, consumerClient.pendingRequestCount(node)); + + consumerClient.awaitPendingRequests(node); + assertTrue(future1.succeeded()); + assertTrue(future2.succeeded()); + } + + @Test + public void schedule() { + TestDelayedTask task = new TestDelayedTask(); + consumerClient.schedule(task, time.milliseconds()); + consumerClient.poll(0); + assertEquals(1, task.executions); + + consumerClient.schedule(task, time.milliseconds() + 100); + consumerClient.poll(0); + assertEquals(1, task.executions); + + time.sleep(100); + consumerClient.poll(0); + assertEquals(2, task.executions); + } + + @Test + public void wakeup() { + RequestFuture future = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + consumerClient.wakeup(); + try { + consumerClient.poll(0); + fail(); + } catch (ConsumerWakeupException e) { + } + + client.respond(heartbeatResponse(Errors.NONE.code())); + consumerClient.poll(future); + assertTrue(future.isDone()); + } + + + private HeartbeatRequest heartbeatRequest() { + return new HeartbeatRequest("group", 1, "consumerId"); + } + + private Struct heartbeatResponse(short error) { + HeartbeatResponse response = new HeartbeatResponse(error); + return response.toStruct(); + } + + private static class TestDelayedTask implements DelayedTask { + int executions = 0; + @Override + public void run(long now) { + executions++; + } + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java new file mode 100644 index 0000000000000..ca832beeaab2b --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java @@ -0,0 +1,548 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.CommitType; +import org.apache.kafka.clients.consumer.ConsumerCommitCallback; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.ConsumerMetadataResponse; +import org.apache.kafka.common.requests.HeartbeatResponse; +import org.apache.kafka.common.requests.JoinGroupResponse; +import org.apache.kafka.common.requests.OffsetCommitResponse; +import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestUtils; + +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.Before; +import org.junit.Test; + + +public class CoordinatorTest { + + private String topicName = "test"; + private String groupId = "test-group"; + private TopicPartition tp = new TopicPartition(topicName, 0); + private int sessionTimeoutMs = 10; + private long retryBackoffMs = 100; + private long requestTimeoutMs = 5000; + private String rebalanceStrategy = "not-matter"; + private MockTime time; + private MockClient client; + private Cluster cluster = TestUtils.singletonCluster(topicName, 1); + private Node node = cluster.nodes().get(0); + private SubscriptionState subscriptions; + private Metadata metadata; + private Metrics metrics; + private Map metricTags = new LinkedHashMap(); + private ConsumerNetworkClient consumerClient; + private MockRebalanceCallback rebalanceCallback; + private Coordinator coordinator; + + @Before + public void setup() { + this.time = new MockTime(); + this.client = new MockClient(time); + this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); + this.metadata = new Metadata(0, Long.MAX_VALUE); + this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + this.metrics = new Metrics(time); + this.rebalanceCallback = new MockRebalanceCallback(); + + client.setNode(node); + + this.coordinator = new Coordinator(consumerClient, + groupId, + sessionTimeoutMs, + rebalanceStrategy, + subscriptions, + metrics, + "consumer" + groupId, + metricTags, + time, + requestTimeoutMs, + retryBackoffMs, + rebalanceCallback); + } + + @Test + public void testNormalHeartbeat() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // normal heartbeat + time.sleep(sessionTimeoutMs); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.succeeded()); + } + + @Test + public void testCoordinatorNotAvailable() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // consumer_coordinator_not_available will mark coordinator as unknown + time.sleep(sessionTimeoutMs); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())); + time.sleep(sessionTimeoutMs); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), future.exception()); + assertTrue(coordinator.coordinatorUnknown()); + } + + @Test + public void testNotCoordinator() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // not_coordinator will mark coordinator as unknown + time.sleep(sessionTimeoutMs); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.NOT_COORDINATOR_FOR_CONSUMER.code())); + time.sleep(sessionTimeoutMs); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), future.exception()); + assertTrue(coordinator.coordinatorUnknown()); + } + + @Test + public void testIllegalGeneration() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // illegal_generation will cause re-partition + subscriptions.subscribe(topicName); + subscriptions.changePartitionAssignment(Collections.singletonList(tp)); + + time.sleep(sessionTimeoutMs); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.ILLEGAL_GENERATION.code())); + time.sleep(sessionTimeoutMs); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.ILLEGAL_GENERATION.exception(), future.exception()); + assertTrue(subscriptions.partitionAssignmentNeeded()); + } + + @Test + public void testUnknownConsumerId() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // illegal_generation will cause re-partition + subscriptions.subscribe(topicName); + subscriptions.changePartitionAssignment(Collections.singletonList(tp)); + + time.sleep(sessionTimeoutMs); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.UNKNOWN_CONSUMER_ID.code())); + time.sleep(sessionTimeoutMs); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertEquals(Errors.UNKNOWN_CONSUMER_ID.exception(), future.exception()); + assertTrue(subscriptions.partitionAssignmentNeeded()); + } + + @Test + public void testCoordinatorDisconnect() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // coordinator disconnect will mark coordinator as unknown + time.sleep(sessionTimeoutMs); + RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat + assertEquals(1, consumerClient.pendingRequestCount()); + assertFalse(future.isDone()); + + client.prepareResponse(heartbeatResponse(Errors.NONE.code()), true); // return disconnected + time.sleep(sessionTimeoutMs); + consumerClient.poll(0); + + assertTrue(future.isDone()); + assertTrue(future.failed()); + assertTrue(future.exception() instanceof DisconnectException); + assertTrue(coordinator.coordinatorUnknown()); + } + + @Test + public void testNormalJoinGroup() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // normal join group + client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); + coordinator.ensurePartitionAssignment(); + + assertFalse(subscriptions.partitionAssignmentNeeded()); + assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertEquals(1, rebalanceCallback.revokedCount); + assertEquals(Collections.emptySet(), rebalanceCallback.revoked); + assertEquals(1, rebalanceCallback.assignedCount); + assertEquals(Collections.singleton(tp), rebalanceCallback.assigned); + } + + @Test + public void testReJoinGroup() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // disconnected from original coordinator will cause re-discover and join again + client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()), true); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code())); + coordinator.ensurePartitionAssignment(); + assertFalse(subscriptions.partitionAssignmentNeeded()); + assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertEquals(1, rebalanceCallback.revokedCount); + assertEquals(Collections.emptySet(), rebalanceCallback.revoked); + assertEquals(1, rebalanceCallback.assignedCount); + assertEquals(Collections.singleton(tp), rebalanceCallback.assigned); + } + + @Test(expected = ApiException.class) + public void testUnknownPartitionAssignmentStrategy() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // coordinator doesn't like our assignment strategy + client.prepareResponse(joinGroupResponse(0, "consumer", Collections.emptyList(), Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code())); + coordinator.ensurePartitionAssignment(); + } + + @Test(expected = ApiException.class) + public void testInvalidSessionTimeout() { + subscriptions.subscribe(topicName); + subscriptions.needReassignment(); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // coordinator doesn't like our assignment strategy + client.prepareResponse(joinGroupResponse(0, "consumer", Collections.emptyList(), Errors.INVALID_SESSION_TIMEOUT.code())); + coordinator.ensurePartitionAssignment(); + } + + @Test + public void testCommitOffsetNormal() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + + AtomicBoolean success = new AtomicBoolean(false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, callback(success)); + consumerClient.poll(0); + assertTrue(success.get()); + } + + @Test + public void testCommitOffsetAsyncCoordinatorNotAvailable() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // async commit with coordinator not available + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb); + consumerClient.poll(0); + + assertTrue(coordinator.coordinatorUnknown()); + assertEquals(1, cb.invoked); + assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), cb.exception); + } + + @Test + public void testCommitOffsetAsyncNotCoordinator() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // async commit with not coordinator + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb); + consumerClient.poll(0); + + assertTrue(coordinator.coordinatorUnknown()); + assertEquals(1, cb.invoked); + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), cb.exception); + } + + @Test + public void testCommitOffsetAsyncDisconnected() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // async commit with coordinator disconnected + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.ASYNC, cb); + consumerClient.poll(0); + + assertTrue(coordinator.coordinatorUnknown()); + assertEquals(1, cb.invoked); + assertTrue(cb.exception instanceof DisconnectException); + } + + @Test + public void testCommitOffsetSyncNotCoordinator() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code()))); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertEquals(1, cb.invoked); + assertNull(cb.exception); + } + + @Test + public void testCommitOffsetSyncCoordinatorNotAvailable() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()))); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertEquals(1, cb.invoked); + assertNull(cb.exception); + } + + @Test + public void testCommitOffsetSyncCoordinatorDisconnected() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertEquals(1, cb.invoked); + assertNull(cb.exception); + } + + @Test(expected = ApiException.class) + public void testCommitOffsetSyncThrowsNonRetriableException() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // sync commit with invalid partitions should throw if we have no callback + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code())), false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, null); + } + + @Test + public void testCommitOffsetSyncCallbackHandlesNonRetriableException() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // sync commit with invalid partitions should throw if we have no callback + MockCommitCallback cb = new MockCommitCallback(); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code())), false); + coordinator.commitOffsets(Collections.singletonMap(tp, 100L), CommitType.SYNC, cb); + assertTrue(cb.exception instanceof ApiException); + } + + @Test + public void testRefreshOffset() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + subscriptions.subscribe(tp); + subscriptions.needRefreshCommits(); + client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); + coordinator.refreshCommittedOffsetsIfNeeded(); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, (long) subscriptions.committed(tp)); + } + + @Test + public void testRefreshOffsetLoadInProgress() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + subscriptions.subscribe(tp); + subscriptions.needRefreshCommits(); + client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L)); + client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); + coordinator.refreshCommittedOffsetsIfNeeded(); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, (long) subscriptions.committed(tp)); + } + + @Test + public void testRefreshOffsetNotCoordinatorForConsumer() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + subscriptions.subscribe(tp); + subscriptions.needRefreshCommits(); + client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code(), "", 100L)); + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L)); + coordinator.refreshCommittedOffsetsIfNeeded(); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(100L, (long) subscriptions.committed(tp)); + } + + @Test + public void testRefreshOffsetWithNoFetchableOffsets() { + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + subscriptions.subscribe(tp); + subscriptions.needRefreshCommits(); + client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L)); + coordinator.refreshCommittedOffsetsIfNeeded(); + assertFalse(subscriptions.refreshCommitsNeeded()); + assertEquals(null, subscriptions.committed(tp)); + } + + private Struct consumerMetadataResponse(Node node, short error) { + ConsumerMetadataResponse response = new ConsumerMetadataResponse(error, node); + return response.toStruct(); + } + + private Struct heartbeatResponse(short error) { + HeartbeatResponse response = new HeartbeatResponse(error); + return response.toStruct(); + } + + private Struct joinGroupResponse(int generationId, String consumerId, List assignedPartitions, short error) { + JoinGroupResponse response = new JoinGroupResponse(error, generationId, consumerId, assignedPartitions); + return response.toStruct(); + } + + private Struct offsetCommitResponse(Map responseData) { + OffsetCommitResponse response = new OffsetCommitResponse(responseData); + return response.toStruct(); + } + + private Struct offsetFetchResponse(TopicPartition tp, Short error, String metadata, long offset) { + OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, metadata, error); + OffsetFetchResponse response = new OffsetFetchResponse(Collections.singletonMap(tp, data)); + return response.toStruct(); + } + + private ConsumerCommitCallback callback(final AtomicBoolean success) { + return new ConsumerCommitCallback() { + @Override + public void onComplete(Map offsets, Exception exception) { + if (exception == null) + success.set(true); + } + }; + } + + private static class MockCommitCallback implements ConsumerCommitCallback { + public int invoked = 0; + public Exception exception = null; + + @Override + public void onComplete(Map offsets, Exception exception) { + invoked++; + this.exception = exception; + } + } + + private static class MockRebalanceCallback implements Coordinator.RebalanceCallback { + public Collection revoked; + public Collection assigned; + public int revokedCount = 0; + public int assignedCount = 0; + + + @Override + public void onPartitionsAssigned(Collection partitions) { + this.assigned = partitions; + assignedCount++; + } + + @Override + public void onPartitionsRevoked(Collection partitions) { + this.revoked = partitions; + revokedCount++; + } + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java new file mode 100644 index 0000000000000..db87b6635c714 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.java @@ -0,0 +1,89 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; + +import static org.junit.Assert.assertEquals; + +public class DelayedTaskQueueTest { + private DelayedTaskQueue scheduler = new DelayedTaskQueue(); + private ArrayList executed = new ArrayList(); + + @Test + public void testScheduling() { + // Empty scheduler + assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(0)); + scheduler.poll(0); + assertEquals(Collections.emptyList(), executed); + + TestTask task1 = new TestTask(); + TestTask task2 = new TestTask(); + TestTask task3 = new TestTask(); + scheduler.add(task1, 20); + assertEquals(20, scheduler.nextTimeout(0)); + scheduler.add(task2, 10); + assertEquals(10, scheduler.nextTimeout(0)); + scheduler.add(task3, 30); + assertEquals(10, scheduler.nextTimeout(0)); + + scheduler.poll(5); + assertEquals(Collections.emptyList(), executed); + assertEquals(5, scheduler.nextTimeout(5)); + + scheduler.poll(10); + assertEquals(Arrays.asList(task2), executed); + assertEquals(10, scheduler.nextTimeout(10)); + + scheduler.poll(20); + assertEquals(Arrays.asList(task2, task1), executed); + assertEquals(20, scheduler.nextTimeout(10)); + + scheduler.poll(30); + assertEquals(Arrays.asList(task2, task1, task3), executed); + assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(30)); + } + + @Test + public void testRemove() { + TestTask task1 = new TestTask(); + TestTask task2 = new TestTask(); + TestTask task3 = new TestTask(); + scheduler.add(task1, 20); + scheduler.add(task2, 10); + scheduler.add(task3, 30); + scheduler.add(task1, 40); + assertEquals(10, scheduler.nextTimeout(0)); + + scheduler.remove(task2); + assertEquals(20, scheduler.nextTimeout(0)); + + scheduler.remove(task1); + assertEquals(30, scheduler.nextTimeout(0)); + + scheduler.remove(task3); + assertEquals(Long.MAX_VALUE, scheduler.nextTimeout(0)); + } + + private class TestTask implements DelayedTask { + @Override + public void run(long now) { + executed.add(this); + } + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java new file mode 100644 index 0000000000000..7a4e586c1a0c5 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -0,0 +1,172 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.requests.FetchResponse; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class FetcherTest { + + private String topicName = "test"; + private String groupId = "test-group"; + private TopicPartition tp = new TopicPartition(topicName, 0); + private int minBytes = 1; + private int maxWaitMs = 0; + private int fetchSize = 1000; + private long retryBackoffMs = 100; + private MockTime time = new MockTime(); + private MockClient client = new MockClient(time); + private Metadata metadata = new Metadata(0, Long.MAX_VALUE); + private Cluster cluster = TestUtils.singletonCluster(topicName, 1); + private Node node = cluster.nodes().get(0); + private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); + private Metrics metrics = new Metrics(time); + private Map metricTags = new LinkedHashMap(); + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + + private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); + + private Fetcher fetcher = new Fetcher(consumerClient, + minBytes, + maxWaitMs, + fetchSize, + true, // check crc + new ByteArrayDeserializer(), + new ByteArrayDeserializer(), + metadata, + subscriptions, + metrics, + "consumer" + groupId, + metricTags, + time, + retryBackoffMs); + + @Before + public void setup() throws Exception { + metadata.update(cluster, time.milliseconds()); + client.setNode(node); + + records.append(1L, "key".getBytes(), "value-1".getBytes()); + records.append(2L, "key".getBytes(), "value-2".getBytes()); + records.append(3L, "key".getBytes(), "value-3".getBytes()); + records.close(); + records.flip(); + } + + @Test + public void testFetchNormal() { + List> records; + subscriptions.subscribe(tp); + subscriptions.fetched(tp, 0); + subscriptions.consumed(tp, 0); + + // normal fetch + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L)); + consumerClient.poll(0); + records = fetcher.fetchedRecords().get(tp); + assertEquals(3, records.size()); + assertEquals(4L, (long) subscriptions.fetched(tp)); // this is the next fetching position + assertEquals(4L, (long) subscriptions.consumed(tp)); + long offset = 1; + for (ConsumerRecord record : records) { + assertEquals(offset, record.offset()); + offset += 1; + } + } + + @Test + public void testFetchFailed() { + List> records; + subscriptions.subscribe(tp); + subscriptions.fetched(tp, 0); + subscriptions.consumed(tp, 0); + + // fetch with not leader + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L)); + consumerClient.poll(0); + assertEquals(0, fetcher.fetchedRecords().size()); + assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); + + // fetch with unknown topic partition + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L)); + consumerClient.poll(0); + assertEquals(0, fetcher.fetchedRecords().size()); + assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); + + // fetch with out of range + subscriptions.fetched(tp, 5); + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); + consumerClient.poll(0); + assertTrue(subscriptions.isOffsetResetNeeded(tp)); + assertEquals(0, fetcher.fetchedRecords().size()); + assertEquals(null, subscriptions.fetched(tp)); + assertEquals(null, subscriptions.consumed(tp)); + } + + @Test + public void testFetchOutOfRange() { + List> records; + subscriptions.subscribe(tp); + subscriptions.fetched(tp, 5); + subscriptions.consumed(tp, 5); + + // fetch with out of range + fetcher.initFetches(cluster); + client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L)); + consumerClient.poll(0); + assertTrue(subscriptions.isOffsetResetNeeded(tp)); + assertEquals(0, fetcher.fetchedRecords().size()); + assertEquals(null, subscriptions.fetched(tp)); + assertEquals(null, subscriptions.consumed(tp)); + } + + private Struct fetchResponse(ByteBuffer buffer, short error, long hw) { + FetchResponse response = new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, buffer))); + return response.toStruct(); + } + + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java new file mode 100644 index 0000000000000..b587e1493755e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java @@ -0,0 +1,69 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.utils.MockTime; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class HeartbeatTest { + + private long timeout = 300L; + private MockTime time = new MockTime(); + private Heartbeat heartbeat = new Heartbeat(timeout, -1L); + + @Test + public void testShouldHeartbeat() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep((long) ((float) timeout / Heartbeat.HEARTBEATS_PER_SESSION_INTERVAL * 1.1)); + assertTrue(heartbeat.shouldHeartbeat(time.milliseconds())); + } + + @Test + public void testShouldNotHeartbeat() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep(timeout / (2 * Heartbeat.HEARTBEATS_PER_SESSION_INTERVAL)); + assertFalse(heartbeat.shouldHeartbeat(time.milliseconds())); + } + + @Test + public void testTimeToNextHeartbeat() { + heartbeat.sentHeartbeat(0); + assertEquals(100, heartbeat.timeToNextHeartbeat(0)); + assertEquals(0, heartbeat.timeToNextHeartbeat(100)); + assertEquals(0, heartbeat.timeToNextHeartbeat(200)); + } + + @Test + public void testSessionTimeoutExpired() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep(305); + assertTrue(heartbeat.sessionTimeoutExpired(time.milliseconds())); + } + + @Test + public void testResetSession() { + heartbeat.sentHeartbeat(time.milliseconds()); + time.sleep(305); + heartbeat.resetSessionTimeout(time.milliseconds()); + assertFalse(heartbeat.sessionTimeoutExpired(time.milliseconds())); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java new file mode 100644 index 0000000000000..73727541b5f99 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java @@ -0,0 +1,57 @@ +/** + * 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 org.apache.kafka.clients.consumer.internals; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class RequestFutureTest { + + @Test + public void testComposeSuccessCase() { + RequestFuture future = new RequestFuture(); + RequestFuture composed = future.compose(new RequestFutureAdapter() { + @Override + public void onSuccess(String value, RequestFuture future) { + future.complete(value.length()); + } + }); + + future.complete("hello"); + + assertTrue(composed.isDone()); + assertTrue(composed.succeeded()); + assertEquals(5, (int) composed.value()); + } + + @Test + public void testComposeFailureCase() { + RequestFuture future = new RequestFuture(); + RequestFuture composed = future.compose(new RequestFutureAdapter() { + @Override + public void onSuccess(String value, RequestFuture future) { + future.complete(value.length()); + } + }); + + RuntimeException e = new RuntimeException(); + future.raise(e); + + assertTrue(composed.isDone()); + assertTrue(composed.failed()); + assertEquals(e, composed.exception()); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java new file mode 100644 index 0000000000000..319751c374ccd --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -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 org.apache.kafka.clients.consumer.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static java.util.Arrays.asList; + +import java.util.Collections; + +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +public class SubscriptionStateTest { + + private final SubscriptionState state = new SubscriptionState(OffsetResetStrategy.EARLIEST); + private final TopicPartition tp0 = new TopicPartition("test", 0); + private final TopicPartition tp1 = new TopicPartition("test", 1); + + @Test + public void partitionSubscription() { + state.subscribe(tp0); + assertEquals(Collections.singleton(tp0), state.assignedPartitions()); + state.committed(tp0, 1); + state.fetched(tp0, 1); + state.consumed(tp0, 1); + assertAllPositions(tp0, 1L); + state.unsubscribe(tp0); + assertTrue(state.assignedPartitions().isEmpty()); + assertAllPositions(tp0, null); + } + + @Test + public void partitionReset() { + state.subscribe(tp0); + state.seek(tp0, 5); + assertEquals(5L, (long) state.fetched(tp0)); + assertEquals(5L, (long) state.consumed(tp0)); + state.needOffsetReset(tp0); + assertTrue(state.isOffsetResetNeeded()); + assertTrue(state.isOffsetResetNeeded(tp0)); + assertEquals(null, state.fetched(tp0)); + assertEquals(null, state.consumed(tp0)); + } + + @Test + public void topicSubscription() { + state.subscribe("test"); + assertEquals(1, state.subscribedTopics().size()); + assertTrue(state.assignedPartitions().isEmpty()); + assertTrue(state.partitionsAutoAssigned()); + state.changePartitionAssignment(asList(tp0)); + state.committed(tp0, 1); + state.fetched(tp0, 1); + state.consumed(tp0, 1); + assertAllPositions(tp0, 1L); + state.changePartitionAssignment(asList(tp1)); + assertAllPositions(tp0, null); + assertEquals(Collections.singleton(tp1), state.assignedPartitions()); + } + + @Test(expected = IllegalArgumentException.class) + public void cantChangeFetchPositionForNonAssignedPartition() { + state.fetched(tp0, 1); + } + + @Test(expected = IllegalArgumentException.class) + public void cantChangeConsumedPositionForNonAssignedPartition() { + state.consumed(tp0, 1); + } + + public void assertAllPositions(TopicPartition tp, Long offset) { + assertEquals(offset, state.committed(tp)); + assertEquals(offset, state.fetched(tp)); + assertEquals(offset, state.consumed(tp)); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java new file mode 100644 index 0000000000000..f3f8334f848be --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -0,0 +1,71 @@ +/** + * 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 org.apache.kafka.clients.producer; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.test.MockMetricsReporter; +import org.apache.kafka.test.MockSerializer; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Properties; + +public class KafkaProducerTest { + + + @Test + public void testConstructorFailureCloseResource() { + Properties props = new Properties(); + props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "testConstructorClose"); + props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "some.invalid.hostname.foo.bar:9999"); + props.setProperty(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + + final int oldInitCount = MockMetricsReporter.INIT_COUNT.get(); + final int oldCloseCount = MockMetricsReporter.CLOSE_COUNT.get(); + try { + KafkaProducer producer = new KafkaProducer( + props, new ByteArraySerializer(), new ByteArraySerializer()); + } catch (KafkaException e) { + Assert.assertEquals(oldInitCount + 1, MockMetricsReporter.INIT_COUNT.get()); + Assert.assertEquals(oldCloseCount + 1, MockMetricsReporter.CLOSE_COUNT.get()); + Assert.assertEquals("Failed to construct kafka producer", e.getMessage()); + return; + } + Assert.fail("should have caught an exception and returned"); + } + + @Test + public void testSerializerClose() { + Properties props = new Properties(); + props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "testConstructorClose"); + props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + props.setProperty(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + + final int oldInitCount = MockSerializer.INIT_COUNT.get(); + final int oldCloseCount = MockSerializer.CLOSE_COUNT.get(); + + KafkaProducer producer = new KafkaProducer( + props, new MockSerializer(), new MockSerializer()); + Assert.assertEquals(oldInitCount + 2, MockSerializer.INIT_COUNT.get()); + Assert.assertEquals(oldCloseCount, MockSerializer.CLOSE_COUNT.get()); + + producer.close(); + Assert.assertEquals(oldInitCount + 2, MockSerializer.INIT_COUNT.get()); + Assert.assertEquals(oldCloseCount + 2, MockSerializer.CLOSE_COUNT.get()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java deleted file mode 100644 index 09a5355d25a3b..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java +++ /dev/null @@ -1,65 +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 org.apache.kafka.clients.producer; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import org.apache.kafka.clients.producer.internals.Metadata; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.test.TestUtils; -import org.junit.Test; - -public class MetadataTest { - - private long refreshBackoffMs = 100; - private long metadataExpireMs = 1000; - private Metadata metadata = new Metadata(refreshBackoffMs, metadataExpireMs); - - @Test - public void testMetadata() throws Exception { - long time = 0; - metadata.update(Cluster.empty(), time); - assertFalse("No update needed.", metadata.needsUpdate(time)); - metadata.forceUpdate(); - assertFalse("Still no updated needed due to backoff", metadata.needsUpdate(time)); - time += refreshBackoffMs; - assertTrue("Update needed now that backoff time expired", metadata.needsUpdate(time)); - String topic = "my-topic"; - Thread t1 = asyncFetch(topic); - Thread t2 = asyncFetch(topic); - assertTrue("Awaiting update", t1.isAlive()); - assertTrue("Awaiting update", t2.isAlive()); - metadata.update(TestUtils.singletonCluster(topic, 1), time); - t1.join(); - t2.join(); - assertFalse("No update needed.", metadata.needsUpdate(time)); - time += metadataExpireMs; - assertTrue("Update needed due to stale metadata.", metadata.needsUpdate(time)); - } - - private Thread asyncFetch(final String topic) { - Thread thread = new Thread() { - public void run() { - metadata.fetch(topic, Integer.MAX_VALUE); - } - }; - thread.start(); - return thread; - } - -} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 9a9411fc900bf..7a46c561b70cd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -17,17 +17,22 @@ package org.apache.kafka.clients.producer; import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.ArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import org.apache.kafka.clients.producer.MockProducer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.clients.producer.internals.DefaultPartitioner; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.test.MockSerializer; import org.junit.Test; public class MockProducerTest { @@ -35,24 +40,38 @@ public class MockProducerTest { private String topic = "topic"; @Test + @SuppressWarnings("unchecked") public void testAutoCompleteMock() throws Exception { - MockProducer producer = new MockProducer(true); - ProducerRecord record = new ProducerRecord(topic, "key".getBytes(), "value".getBytes()); + MockProducer producer = new MockProducer(true, new MockSerializer(), new MockSerializer()); + ProducerRecord record = new ProducerRecord(topic, "key".getBytes(), "value".getBytes()); Future metadata = producer.send(record); assertTrue("Send should be immediately complete", metadata.isDone()); assertFalse("Send should be successful", isError(metadata)); - assertEquals("Offset should be 0", 0, metadata.get().offset()); + assertEquals("Offset should be 0", 0L, metadata.get().offset()); assertEquals(topic, metadata.get().topic()); - assertEquals("We should have the record in our history", asList(record), producer.history()); + assertEquals("We should have the record in our history", singletonList(record), producer.history()); + producer.clear(); + assertEquals("Clear should erase our history", 0, producer.history().size()); + } + + @Test + public void testPartitioner() throws Exception { + PartitionInfo partitionInfo0 = new PartitionInfo(topic, 0, null, null, null); + PartitionInfo partitionInfo1 = new PartitionInfo(topic, 1, null, null, null); + Cluster cluster = new Cluster(new ArrayList(0), asList(partitionInfo0, partitionInfo1)); + MockProducer producer = new MockProducer(cluster, true, new DefaultPartitioner(), new StringSerializer(), new StringSerializer()); + ProducerRecord record = new ProducerRecord(topic, "key", "value"); + Future metadata = producer.send(record); + assertEquals("Partition should be correct", 1, metadata.get().partition()); producer.clear(); assertEquals("Clear should erase our history", 0, producer.history().size()); } @Test public void testManualCompletion() throws Exception { - MockProducer producer = new MockProducer(false); - ProducerRecord record1 = new ProducerRecord("topic", "key1".getBytes(), "value1".getBytes()); - ProducerRecord record2 = new ProducerRecord("topic", "key2".getBytes(), "value2".getBytes()); + MockProducer producer = new MockProducer(false, new MockSerializer(), new MockSerializer()); + ProducerRecord record1 = new ProducerRecord(topic, "key1".getBytes(), "value1".getBytes()); + ProducerRecord record2 = new ProducerRecord(topic, "key2".getBytes(), "value2".getBytes()); Future md1 = producer.send(record1); assertFalse("Send shouldn't have completed", md1.isDone()); Future md2 = producer.send(record2); @@ -69,6 +88,12 @@ public void testManualCompletion() throws Exception { assertEquals(e, err.getCause()); } assertFalse("No more requests to complete", producer.completeNext()); + + Future md3 = producer.send(record1); + Future md4 = producer.send(record2); + assertTrue("Requests should not be completed.", !md3.isDone() && !md4.isDone()); + producer.flush(); + assertTrue("Requests should be completed.", md3.isDone() && md4.isDone()); } private boolean isError(Future future) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java deleted file mode 100644 index c78da64b5ad7b..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java +++ /dev/null @@ -1,71 +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 org.apache.kafka.clients.producer; - -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.util.List; - - -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.internals.Partitioner; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; -import org.junit.Test; - -public class PartitionerTest { - - private byte[] key = "key".getBytes(); - private byte[] value = "value".getBytes(); - private Partitioner partitioner = new Partitioner(); - private Node node0 = new Node(0, "localhost", 99); - private Node node1 = new Node(1, "localhost", 100); - private Node node2 = new Node(2, "localhost", 101); - private Node[] nodes = new Node[] { node0, node1, node2 }; - private String topic = "test"; - private List partitions = asList(new PartitionInfo(topic, 0, node0, nodes, nodes), - new PartitionInfo(topic, 1, node1, nodes, nodes), - new PartitionInfo(topic, 2, null, nodes, nodes)); - private Cluster cluster = new Cluster(asList(node0, node1, node2), partitions); - - @Test - public void testUserSuppliedPartitioning() { - assertEquals("If the user supplies a partition we should use it.", - 0, - partitioner.partition(new ProducerRecord("test", 0, key, value), cluster)); - } - - @Test - public void testKeyPartitionIsStable() { - int partition = partitioner.partition(new ProducerRecord("test", key, value), cluster); - assertEquals("Same key should yield same partition", - partition, - partitioner.partition(new ProducerRecord("test", key, "value2".getBytes()), cluster)); - } - - @Test - public void testRoundRobinWithDownNode() { - for (int i = 0; i < partitions.size(); i++) { - int part = partitioner.partition(new ProducerRecord("test", value), cluster); - assertTrue("We should never choose a leader-less node in round robin", part >= 0 && part < 2); - - } - } -} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java new file mode 100644 index 0000000000000..7bb181e46c474 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java @@ -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 org.apache.kafka.clients.producer; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class ProducerRecordTest { + + @Test + public void testEqualsAndHashCode() { + ProducerRecord producerRecord = new ProducerRecord("test", 1 , "key", 1); + assertEquals(producerRecord, producerRecord); + assertEquals(producerRecord.hashCode(), producerRecord.hashCode()); + + ProducerRecord equalRecord = new ProducerRecord("test", 1 , "key", 1); + assertEquals(producerRecord, equalRecord); + assertEquals(producerRecord.hashCode(), equalRecord.hashCode()); + + ProducerRecord topicMisMatch = new ProducerRecord("test-1", 1 , "key", 1); + assertFalse(producerRecord.equals(topicMisMatch)); + + ProducerRecord partitionMismatch = new ProducerRecord("test", 2 , "key", 1); + assertFalse(producerRecord.equals(partitionMismatch)); + + ProducerRecord keyMisMatch = new ProducerRecord("test", 1 , "key-1", 1); + assertFalse(producerRecord.equals(keyMisMatch)); + + ProducerRecord valueMisMatch = new ProducerRecord("test", 1 , "key", 2); + assertFalse(producerRecord.equals(valueMisMatch)); + + ProducerRecord nullFieldsRecord = new ProducerRecord("topic", null, null, null); + assertEquals(nullFieldsRecord, nullFieldsRecord); + assertEquals(nullFieldsRecord.hashCode(), nullFieldsRecord.hashCode()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java deleted file mode 100644 index 1bbe83c1bfd75..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java +++ /dev/null @@ -1,151 +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 org.apache.kafka.clients.producer; - -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - - -import org.apache.kafka.clients.producer.internals.RecordAccumulator; -import org.apache.kafka.clients.producer.internals.RecordBatch; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.LogEntry; -import org.apache.kafka.common.record.Record; -import org.apache.kafka.common.record.Records; -import org.apache.kafka.common.utils.MockTime; -import org.junit.Test; - -public class RecordAccumulatorTest { - - private TopicPartition tp = new TopicPartition("test", 0); - private MockTime time = new MockTime(); - private byte[] key = "key".getBytes(); - private byte[] value = "value".getBytes(); - private int msgSize = Records.LOG_OVERHEAD + Record.recordSize(key, value); - private Metrics metrics = new Metrics(time); - - @Test - public void testFull() throws Exception { - long now = time.milliseconds(); - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, false, metrics, time); - int appends = 1024 / msgSize; - for (int i = 0; i < appends; i++) { - accum.append(tp, key, value, CompressionType.NONE, null); - assertEquals("No partitions should be ready.", 0, accum.ready(now).size()); - } - accum.append(tp, key, value, CompressionType.NONE, null); - assertEquals("Our partition should be ready", asList(tp), accum.ready(time.milliseconds())); - List batches = accum.drain(asList(tp), Integer.MAX_VALUE); - assertEquals(1, batches.size()); - RecordBatch batch = batches.get(0); - Iterator iter = batch.records.iterator(); - for (int i = 0; i < appends; i++) { - LogEntry entry = iter.next(); - assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key()); - assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value()); - } - assertFalse("No more records", iter.hasNext()); - } - - @Test - public void testAppendLarge() throws Exception { - int batchSize = 512; - RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, 0L, false, metrics, time); - accum.append(tp, key, new byte[2 * batchSize], CompressionType.NONE, null); - assertEquals("Our partition should be ready", asList(tp), accum.ready(time.milliseconds())); - } - - @Test - public void testLinger() throws Exception { - long lingerMs = 10L; - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, false, metrics, time); - accum.append(tp, key, value, CompressionType.NONE, null); - assertEquals("No partitions should be ready", 0, accum.ready(time.milliseconds()).size()); - time.sleep(10); - assertEquals("Our partition should be ready", asList(tp), accum.ready(time.milliseconds())); - List batches = accum.drain(asList(tp), Integer.MAX_VALUE); - assertEquals(1, batches.size()); - RecordBatch batch = batches.get(0); - Iterator iter = batch.records.iterator(); - LogEntry entry = iter.next(); - assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key()); - assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value()); - assertFalse("No more records", iter.hasNext()); - } - - @Test - public void testPartialDrain() throws Exception { - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, false, metrics, time); - int appends = 1024 / msgSize + 1; - List partitions = asList(new TopicPartition("test", 0), new TopicPartition("test", 1)); - for (TopicPartition tp : partitions) { - for (int i = 0; i < appends; i++) - accum.append(tp, key, value, CompressionType.NONE, null); - } - assertEquals("Both partitions should be ready", 2, accum.ready(time.milliseconds()).size()); - - List batches = accum.drain(partitions, 1024); - assertEquals("But due to size bound only one partition should have been retrieved", 1, batches.size()); - } - - @Test - public void testStressfulSituation() throws Exception { - final int numThreads = 5; - final int msgs = 10000; - final int numParts = 10; - final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 0L, true, metrics, time); - List threads = new ArrayList(); - for (int i = 0; i < numThreads; i++) { - threads.add(new Thread() { - public void run() { - for (int i = 0; i < msgs; i++) { - try { - accum.append(new TopicPartition("test", i % numParts), key, value, CompressionType.NONE, null); - } catch (Exception e) { - e.printStackTrace(); - } - } - } - }); - } - for (Thread t : threads) - t.start(); - int read = 0; - long now = time.milliseconds(); - while (read < numThreads * msgs) { - List tps = accum.ready(now); - List batches = accum.drain(tps, 5 * 1024); - for (RecordBatch batch : batches) { - for (LogEntry entry : batch.records) - read++; - } - accum.deallocate(batches); - } - - for (Thread t : threads) - t.join(); - } - -} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java index a3700a6a10a9a..1e5d1c2d6159b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java @@ -78,7 +78,7 @@ public void testBlocking() throws Exception { /* create a new request result that will be completed after the given timeout */ public ProduceRequestResult asyncRequest(final long baseOffset, final RuntimeException error, final long timeout) { final ProduceRequestResult request = new ProduceRequestResult(); - new Thread() { + Thread thread = new Thread() { public void run() { try { sleep(timeout); @@ -87,7 +87,8 @@ public void run() { e.printStackTrace(); } } - }.start(); + }; + thread.start(); return request; } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java deleted file mode 100644 index 41c028bffbda1..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java +++ /dev/null @@ -1,104 +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 org.apache.kafka.clients.producer; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.nio.ByteBuffer; -import java.util.concurrent.Future; - - -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.clients.producer.internals.Metadata; -import org.apache.kafka.clients.producer.internals.RecordAccumulator; -import org.apache.kafka.clients.producer.internals.Sender; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.network.NetworkReceive; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.requests.RequestSend; -import org.apache.kafka.common.requests.ResponseHeader; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.test.MockSelector; -import org.apache.kafka.test.TestUtils; -import org.junit.Before; -import org.junit.Test; - -public class SenderTest { - - private MockTime time = new MockTime(); - private MockSelector selector = new MockSelector(time); - private int batchSize = 16 * 1024; - private Metadata metadata = new Metadata(0, Long.MAX_VALUE); - private Cluster cluster = TestUtils.singletonCluster("test", 1); - private Metrics metrics = new Metrics(time); - private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, 0L, false, metrics, time); - private Sender sender = new Sender(selector, metadata, this.accumulator, "", 1024 * 1024, 0L, (short) -1, 10000, time); - - @Before - public void setup() { - metadata.update(cluster, time.milliseconds()); - } - - @Test - public void testSimple() throws Exception { - TopicPartition tp = new TopicPartition("test", 0); - Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); - sender.run(time.milliseconds()); - assertEquals("We should have connected", 1, selector.connected().size()); - selector.clear(); - sender.run(time.milliseconds()); - assertEquals("Single request should be sent", 1, selector.completedSends().size()); - RequestSend request = (RequestSend) selector.completedSends().get(0); - selector.clear(); - long offset = 42; - selector.completeReceive(produceResponse(request.header().correlationId(), - cluster.leaderFor(tp).id(), - tp.topic(), - tp.partition(), - offset, - Errors.NONE.code())); - sender.run(time.milliseconds()); - assertTrue("Request should be completed", future.isDone()); - assertEquals(offset, future.get().offset()); - } - - private NetworkReceive produceResponse(int correlation, int source, String topic, int part, long offset, int error) { - Struct struct = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); - Struct response = struct.instance("responses"); - response.set("topic", topic); - Struct partResp = response.instance("partition_responses"); - partResp.set("partition", part); - partResp.set("error_code", (short) error); - partResp.set("base_offset", offset); - response.set("partition_responses", new Object[] { partResp }); - struct.set("responses", new Object[] { response }); - ResponseHeader header = new ResponseHeader(correlation); - ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + struct.sizeOf()); - header.writeTo(buffer); - struct.writeTo(buffer); - buffer.rewind(); - return new NetworkReceive(source, buffer); - } - -} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java similarity index 84% rename from clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java rename to clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java index f227b5c3ff365..2c693824fa53d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java @@ -14,34 +14,38 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.clients.producer; +package org.apache.kafka.clients.producer.internals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import org.apache.kafka.clients.producer.BufferExhaustedException; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestUtils; +import org.junit.Test; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.kafka.clients.producer.BufferExhaustedException; -import org.apache.kafka.clients.producer.internals.BufferPool; -import org.apache.kafka.test.TestUtils; -import org.junit.Test; +import static org.junit.Assert.*; public class BufferPoolTest { + private MockTime time = new MockTime(); + private Metrics metrics = new Metrics(time); + String metricGroup = "TestMetrics"; + Map metricTags = new LinkedHashMap(); /** * Test the simple non-blocking allocation paths */ @Test public void testSimple() throws Exception { - int totalMemory = 64 * 1024; + long totalMemory = 64 * 1024; int size = 1024; - BufferPool pool = new BufferPool(totalMemory, size, false); + BufferPool pool = new BufferPool(totalMemory, size, false, metrics, time, metricGroup, metricTags); ByteBuffer buffer = pool.allocate(size); assertEquals("Buffer size should equal requested size.", size, buffer.limit()); assertEquals("Unallocated memory should have shrunk", totalMemory - size, pool.unallocatedMemory()); @@ -68,7 +72,7 @@ public void testSimple() throws Exception { */ @Test(expected = IllegalArgumentException.class) public void testCantAllocateMoreMemoryThanWeHave() throws Exception { - BufferPool pool = new BufferPool(1024, 512, true); + BufferPool pool = new BufferPool(1024, 512, true, metrics, time, metricGroup, metricTags); ByteBuffer buffer = pool.allocate(1024); assertEquals(1024, buffer.limit()); pool.deallocate(buffer); @@ -77,7 +81,7 @@ public void testCantAllocateMoreMemoryThanWeHave() throws Exception { @Test public void testNonblockingMode() throws Exception { - BufferPool pool = new BufferPool(2, 1, false); + BufferPool pool = new BufferPool(2, 1, false, metrics, time, metricGroup, metricTags); pool.allocate(1); try { pool.allocate(2); @@ -92,18 +96,18 @@ public void testNonblockingMode() throws Exception { */ @Test public void testDelayedAllocation() throws Exception { - BufferPool pool = new BufferPool(5 * 1024, 1024, true); + BufferPool pool = new BufferPool(5 * 1024, 1024, true, metrics, time, metricGroup, metricTags); ByteBuffer buffer = pool.allocate(1024); CountDownLatch doDealloc = asyncDeallocate(pool, buffer); CountDownLatch allocation = asyncAllocate(pool, 5 * 1024); - assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1, allocation.getCount()); + assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1L, allocation.getCount()); doDealloc.countDown(); // return the memory allocation.await(); } private CountDownLatch asyncDeallocate(final BufferPool pool, final ByteBuffer buffer) { final CountDownLatch latch = new CountDownLatch(1); - new Thread() { + Thread thread = new Thread() { public void run() { try { latch.await(); @@ -112,13 +116,14 @@ public void run() { } pool.deallocate(buffer); } - }.start(); + }; + thread.start(); return latch; } private CountDownLatch asyncAllocate(final BufferPool pool, final int size) { final CountDownLatch completed = new CountDownLatch(1); - new Thread() { + Thread thread = new Thread() { public void run() { try { pool.allocate(size); @@ -128,7 +133,8 @@ public void run() { completed.countDown(); } } - }.start(); + }; + thread.start(); return completed; } @@ -140,8 +146,8 @@ public void testStressfulSituation() throws Exception { int numThreads = 10; final int iterations = 50000; final int poolableSize = 1024; - final int totalMemory = numThreads / 2 * poolableSize; - final BufferPool pool = new BufferPool(totalMemory, poolableSize, true); + final long totalMemory = numThreads / 2 * poolableSize; + final BufferPool pool = new BufferPool(totalMemory, poolableSize, true, metrics, time, metricGroup, metricTags); List threads = new ArrayList(); for (int i = 0; i < numThreads; i++) threads.add(new StressTestThread(pool, iterations)); @@ -168,12 +174,12 @@ public void run() { try { for (int i = 0; i < iterations; i++) { int size; - if (TestUtils.random.nextBoolean()) + if (TestUtils.RANDOM.nextBoolean()) // allocate poolable size size = pool.poolableSize(); else // allocate a random size - size = TestUtils.random.nextInt((int) pool.totalMemory()); + size = TestUtils.RANDOM.nextInt((int) pool.totalMemory()); ByteBuffer buffer = pool.allocate(size); pool.deallocate(buffer); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java new file mode 100644 index 0000000000000..977fa933d8316 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java @@ -0,0 +1,63 @@ +/** + * 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 org.apache.kafka.clients.producer.internals; + +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.kafka.clients.producer.Partitioner; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.junit.Test; + +public class DefaultPartitionerTest { + private byte[] keyBytes = "key".getBytes(); + private Partitioner partitioner = new DefaultPartitioner(); + private Node node0 = new Node(0, "localhost", 99); + private Node node1 = new Node(1, "localhost", 100); + private Node node2 = new Node(2, "localhost", 101); + private Node[] nodes = new Node[] {node0, node1, node2}; + private String topic = "test"; + // Intentionally make the partition list not in partition order to test the edge cases. + private List partitions = asList(new PartitionInfo(topic, 1, null, nodes, nodes), + new PartitionInfo(topic, 2, node1, nodes, nodes), + new PartitionInfo(topic, 0, node0, nodes, nodes)); + private Cluster cluster = new Cluster(asList(node0, node1, node2), partitions); + + @Test + public void testKeyPartitionIsStable() { + int partition = partitioner.partition("test", null, keyBytes, null, null, cluster); + assertEquals("Same key should yield same partition", partition, partitioner.partition("test", null, keyBytes, null, null, cluster)); + } + + @Test + public void testRoundRobinWithUnavailablePartitions() { + // When there are some unavailable partitions, we want to make sure that (1) we always pick an available partition, + // and (2) the available partitions are selected in a round robin way. + int countForPart0 = 0; + int countForPart2 = 0; + for (int i = 1; i <= 100; i++) { + int part = partitioner.partition("test", null, null, null, null, cluster); + assertTrue("We should never choose a leader-less node in round robin", part == 0 || part == 2); + if (part == 0) + countForPart0++; + else + countForPart2++; + } + assertEquals("The distribution between two available partitions should be even", countForPart0, countForPart2); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java new file mode 100644 index 0000000000000..5b2e4ffaeab71 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -0,0 +1,294 @@ +/** + * 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 org.apache.kafka.clients.producer.internals; + +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.LogEntry; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.utils.MockTime; +import org.junit.Test; + +public class RecordAccumulatorTest { + + private String topic = "test"; + private int partition1 = 0; + private int partition2 = 1; + private int partition3 = 2; + private Node node1 = new Node(0, "localhost", 1111); + private Node node2 = new Node(1, "localhost", 1112); + private TopicPartition tp1 = new TopicPartition(topic, partition1); + private TopicPartition tp2 = new TopicPartition(topic, partition2); + private TopicPartition tp3 = new TopicPartition(topic, partition3); + private PartitionInfo part1 = new PartitionInfo(topic, partition1, node1, null, null); + private PartitionInfo part2 = new PartitionInfo(topic, partition2, node1, null, null); + private PartitionInfo part3 = new PartitionInfo(topic, partition3, node2, null, null); + private MockTime time = new MockTime(); + private byte[] key = "key".getBytes(); + private byte[] value = "value".getBytes(); + private int msgSize = Records.LOG_OVERHEAD + Record.recordSize(key, value); + private Cluster cluster = new Cluster(Arrays.asList(node1, node2), Arrays.asList(part1, part2, part3)); + private Metrics metrics = new Metrics(time); + Map metricTags = new LinkedHashMap(); + + @Test + public void testFull() throws Exception { + long now = time.milliseconds(); + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10L, 100L, false, metrics, time, metricTags); + int appends = 1024 / msgSize; + for (int i = 0; i < appends; i++) { + accum.append(tp1, key, value, null); + assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); + } + accum.append(tp1, key, value, null); + assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); + List batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); + assertEquals(1, batches.size()); + RecordBatch batch = batches.get(0); + batch.records.flip(); + Iterator iter = batch.records.iterator(); + for (int i = 0; i < appends; i++) { + LogEntry entry = iter.next(); + assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key()); + assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value()); + } + assertFalse("No more records", iter.hasNext()); + } + + @Test + public void testAppendLarge() throws Exception { + int batchSize = 512; + RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, CompressionType.NONE, 0L, 100L, false, metrics, time, metricTags); + accum.append(tp1, key, new byte[2 * batchSize], null); + assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); + } + + @Test + public void testLinger() throws Exception { + long lingerMs = 10L; + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, 100L, false, metrics, time, metricTags); + accum.append(tp1, key, value, null); + assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size()); + time.sleep(10); + assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); + List batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); + assertEquals(1, batches.size()); + RecordBatch batch = batches.get(0); + batch.records.flip(); + Iterator iter = batch.records.iterator(); + LogEntry entry = iter.next(); + assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key()); + assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value()); + assertFalse("No more records", iter.hasNext()); + } + + @Test + public void testPartialDrain() throws Exception { + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10L, 100L, false, metrics, time, metricTags); + int appends = 1024 / msgSize + 1; + List partitions = asList(tp1, tp2); + for (TopicPartition tp : partitions) { + for (int i = 0; i < appends; i++) + accum.append(tp, key, value, null); + } + assertEquals("Partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); + + List batches = accum.drain(cluster, Collections.singleton(node1), 1024, 0).get(node1.id()); + assertEquals("But due to size bound only one partition should have been retrieved", 1, batches.size()); + } + + @SuppressWarnings("unused") + @Test + public void testStressfulSituation() throws Exception { + final int numThreads = 5; + final int msgs = 10000; + final int numParts = 2; + final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 0L, 100L, true, metrics, time, metricTags); + List threads = new ArrayList(); + for (int i = 0; i < numThreads; i++) { + threads.add(new Thread() { + public void run() { + for (int i = 0; i < msgs; i++) { + try { + accum.append(new TopicPartition(topic, i % numParts), key, value, null); + } catch (Exception e) { + e.printStackTrace(); + } + } + } + }); + } + for (Thread t : threads) + t.start(); + int read = 0; + long now = time.milliseconds(); + while (read < numThreads * msgs) { + Set nodes = accum.ready(cluster, now).readyNodes; + List batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id()); + if (batches != null) { + for (RecordBatch batch : batches) { + batch.records.flip(); + for (LogEntry entry : batch.records) + read++; + accum.deallocate(batch); + } + } + } + + for (Thread t : threads) + t.join(); + } + + + @Test + public void testNextReadyCheckDelay() throws Exception { + // Next check time will use lingerMs since this test won't trigger any retries/backoff + long lingerMs = 10L; + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, 100L, false, metrics, time, metricTags); + // Just short of going over the limit so we trigger linger time + int appends = 1024 / msgSize; + + // Partition on node1 only + for (int i = 0; i < appends; i++) + accum.append(tp1, key, value, null); + RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds()); + assertEquals("No nodes should be ready.", 0, result.readyNodes.size()); + assertEquals("Next check time should be the linger time", lingerMs, result.nextReadyCheckDelayMs); + + time.sleep(lingerMs / 2); + + // Add partition on node2 only + for (int i = 0; i < appends; i++) + accum.append(tp3, key, value, null); + result = accum.ready(cluster, time.milliseconds()); + assertEquals("No nodes should be ready.", 0, result.readyNodes.size()); + assertEquals("Next check time should be defined by node1, half remaining linger time", lingerMs / 2, result.nextReadyCheckDelayMs); + + // Add data for another partition on node1, enough to make data sendable immediately + for (int i = 0; i < appends + 1; i++) + accum.append(tp2, key, value, null); + result = accum.ready(cluster, time.milliseconds()); + assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes); + // Note this can actually be < linger time because it may use delays from partitions that aren't sendable + // but have leaders with other sendable data. + assertTrue("Next check time should be defined by node2, at most linger time", result.nextReadyCheckDelayMs <= lingerMs); + } + + @Test + public void testRetryBackoff() throws Exception { + long lingerMs = Long.MAX_VALUE / 4; + long retryBackoffMs = Long.MAX_VALUE / 2; + final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, retryBackoffMs, false, metrics, time, metricTags); + + long now = time.milliseconds(); + accum.append(tp1, key, value, null); + RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, now + lingerMs + 1); + assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes); + Map> batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + lingerMs + 1); + assertEquals("Node1 should be the only ready node.", 1, batches.size()); + assertEquals("Partition 0 should only have one batch drained.", 1, batches.get(0).size()); + + // Reenqueue the batch + now = time.milliseconds(); + accum.reenqueue(batches.get(0).get(0), now); + + // Put message for partition 1 into accumulator + accum.append(tp2, key, value, null); + result = accum.ready(cluster, now + lingerMs + 1); + assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes); + + // tp1 should backoff while tp2 should not + batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + lingerMs + 1); + assertEquals("Node1 should be the only ready node.", 1, batches.size()); + assertEquals("Node1 should only have one batch drained.", 1, batches.get(0).size()); + assertEquals("Node1 should only have one batch for partition 1.", tp2, batches.get(0).get(0).topicPartition); + + // Partition 0 can be drained after retry backoff + result = accum.ready(cluster, now + retryBackoffMs + 1); + assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes); + batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + retryBackoffMs + 1); + assertEquals("Node1 should be the only ready node.", 1, batches.size()); + assertEquals("Node1 should only have one batch drained.", 1, batches.get(0).size()); + assertEquals("Node1 should only have one batch for partition 0.", tp1, batches.get(0).get(0).topicPartition); + } + + @Test + public void testFlush() throws Exception { + long lingerMs = Long.MAX_VALUE; + final RecordAccumulator accum = new RecordAccumulator(4 * 1024, 64 * 1024, CompressionType.NONE, lingerMs, 100L, false, metrics, time, metricTags); + for (int i = 0; i < 100; i++) + accum.append(new TopicPartition(topic, i % 3), key, value, null); + RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds()); + assertEquals("No nodes should be ready.", 0, result.readyNodes.size()); + + accum.beginFlush(); + result = accum.ready(cluster, time.milliseconds()); + + // drain and deallocate all batches + Map> results = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); + for (List batches: results.values()) + for (RecordBatch batch: batches) + accum.deallocate(batch); + + // should be complete with no unsent records. + accum.awaitFlushCompletion(); + assertFalse(accum.hasUnsent()); + } + + @Test + public void testAbortIncompleteBatches() throws Exception { + long lingerMs = Long.MAX_VALUE; + final AtomicInteger numExceptionReceivedInCallback = new AtomicInteger(0); + final RecordAccumulator accum = new RecordAccumulator(4 * 1024, 64 * 1024, CompressionType.NONE, lingerMs, 100L, false, metrics, time, metricTags); + class TestCallback implements Callback { + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + assertTrue(exception.getMessage().equals("Producer is closed forcefully.")); + numExceptionReceivedInCallback.incrementAndGet(); + } + } + for (int i = 0; i < 100; i++) + accum.append(new TopicPartition(topic, i % 3), key, value, new TestCallback()); + RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds()); + assertEquals("No nodes should be ready.", 0, result.readyNodes.size()); + + accum.abortIncompleteBatches(); + assertEquals(numExceptionReceivedInCallback.get(), 100); + assertFalse(accum.hasUnsent()); + + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java new file mode 100644 index 0000000000000..8b1805d3d2bcb --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -0,0 +1,148 @@ +/** + * 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 org.apache.kafka.clients.producer.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.requests.ProduceResponse; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; + +public class SenderTest { + + private static final int MAX_REQUEST_SIZE = 1024 * 1024; + private static final short ACKS_ALL = -1; + private static final int MAX_RETRIES = 0; + private static final int REQUEST_TIMEOUT_MS = 10000; + + private TopicPartition tp = new TopicPartition("test", 0); + private MockTime time = new MockTime(); + private MockClient client = new MockClient(time); + private int batchSize = 16 * 1024; + private Metadata metadata = new Metadata(0, Long.MAX_VALUE); + private Cluster cluster = TestUtils.singletonCluster("test", 1); + private Metrics metrics = new Metrics(time); + Map metricTags = new LinkedHashMap(); + private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, CompressionType.NONE, 0L, 0L, false, metrics, time, metricTags); + private Sender sender = new Sender(client, + metadata, + this.accumulator, + MAX_REQUEST_SIZE, + ACKS_ALL, + MAX_RETRIES, + REQUEST_TIMEOUT_MS, + metrics, + time, + "clientId"); + + @Before + public void setup() { + metadata.update(cluster, time.milliseconds()); + } + + @Test + public void testSimple() throws Exception { + long offset = 0; + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null).future; + sender.run(time.milliseconds()); // connect + sender.run(time.milliseconds()); // send produce request + assertEquals("We should have a single produce request in flight.", 1, client.inFlightRequestCount()); + client.respond(produceResponse(tp, offset, Errors.NONE.code())); + sender.run(time.milliseconds()); + assertEquals("All requests completed.", offset, (long) client.inFlightRequestCount()); + sender.run(time.milliseconds()); + assertTrue("Request should be completed", future.isDone()); + assertEquals(offset, future.get().offset()); + } + + @Test + public void testRetries() throws Exception { + // create a sender with retries = 1 + int maxRetries = 1; + Sender sender = new Sender(client, + metadata, + this.accumulator, + MAX_REQUEST_SIZE, + ACKS_ALL, + maxRetries, + REQUEST_TIMEOUT_MS, + new Metrics(), + time, + "clientId"); + // do a successful retry + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null).future; + sender.run(time.milliseconds()); // connect + sender.run(time.milliseconds()); // send produce request + assertEquals(1, client.inFlightRequestCount()); + client.disconnect(client.requests().peek().request().destination()); + assertEquals(0, client.inFlightRequestCount()); + sender.run(time.milliseconds()); // receive error + sender.run(time.milliseconds()); // reconnect + sender.run(time.milliseconds()); // resend + assertEquals(1, client.inFlightRequestCount()); + long offset = 0; + client.respond(produceResponse(tp, offset, Errors.NONE.code())); + sender.run(time.milliseconds()); + assertTrue("Request should have retried and completed", future.isDone()); + assertEquals(offset, future.get().offset()); + + // do an unsuccessful retry + future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null).future; + sender.run(time.milliseconds()); // send produce request + for (int i = 0; i < maxRetries + 1; i++) { + client.disconnect(client.requests().peek().request().destination()); + sender.run(time.milliseconds()); // receive error + sender.run(time.milliseconds()); // reconnect + sender.run(time.milliseconds()); // resend + } + sender.run(time.milliseconds()); + completedWithError(future, Errors.NETWORK_EXCEPTION); + } + + private void completedWithError(Future future, Errors error) throws Exception { + assertTrue("Request should be completed", future.isDone()); + try { + future.get(); + fail("Should have thrown an exception."); + } catch (ExecutionException e) { + assertEquals(error.exception().getClass(), e.getCause().getClass()); + } + } + + private Struct produceResponse(TopicPartition tp, long offset, int error) { + ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse((short) error, offset); + Map partResp = Collections.singletonMap(tp, resp); + ProduceResponse response = new ProduceResponse(partResp); + return response.toStruct(); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java new file mode 100644 index 0000000000000..db1b0ee911321 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java @@ -0,0 +1,80 @@ +/** + * 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 org.apache.kafka.common.config; + +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.metrics.MetricsReporter; +import org.junit.Test; + +import java.util.Map; +import java.util.Properties; + +import static org.junit.Assert.fail; + +public class AbstractConfigTest { + + @Test + public void testConfiguredInstances() { + testValidInputs(""); + testValidInputs("org.apache.kafka.common.metrics.FakeMetricsReporter"); + testValidInputs("org.apache.kafka.common.metrics.FakeMetricsReporter, org.apache.kafka.common.metrics.FakeMetricsReporter"); + testInvalidInputs(","); + testInvalidInputs("org.apache.kafka.clients.producer.unknown-metrics-reporter"); + testInvalidInputs("test1,test2"); + testInvalidInputs("org.apache.kafka.common.metrics.FakeMetricsReporter,"); + } + + private void testValidInputs(String configValue) { + Properties props = new Properties(); + props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, configValue); + TestConfig config = new TestConfig(props); + try { + config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); + } catch (ConfigException e) { + fail("No exceptions are expected here, valid props are :" + props); + } + } + + private void testInvalidInputs(String configValue) { + Properties props = new Properties(); + props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, configValue); + TestConfig config = new TestConfig(props); + try { + config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); + fail("Expected a config exception due to invalid props :" + props); + } catch (ConfigException e) { + // this is good + } + } + + private static class TestConfig extends AbstractConfig { + + private static final ConfigDef CONFIG; + + public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; + private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters."; + + static { + CONFIG = new ConfigDef().define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, + "", + Importance.LOW, + METRIC_REPORTER_CLASSES_DOC); + } + + public TestConfig(Map props) { + super(CONFIG, props); + } + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 29543dfa34d5f..44c2ef0a859de 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.config; @@ -24,10 +20,10 @@ import java.util.Map; import java.util.Properties; - -import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Validator; import org.apache.kafka.common.config.ConfigDef.Range; +import org.apache.kafka.common.config.ConfigDef.ValidString; import org.apache.kafka.common.config.ConfigDef.Type; import org.junit.Test; @@ -35,13 +31,15 @@ public class ConfigDefTest { @Test public void testBasicTypes() { - ConfigDef def = new ConfigDef().define("a", Type.INT, 5, Range.between(0, 14), "docs") - .define("b", Type.LONG, "docs") - .define("c", Type.STRING, "hello", "docs") - .define("d", Type.LIST, "docs") - .define("e", Type.DOUBLE, "docs") - .define("f", Type.CLASS, "docs") - .define("g", Type.BOOLEAN, "docs"); + ConfigDef def = new ConfigDef().define("a", Type.INT, 5, Range.between(0, 14), Importance.HIGH, "docs") + .define("b", Type.LONG, Importance.HIGH, "docs") + .define("c", Type.STRING, "hello", Importance.HIGH, "docs") + .define("d", Type.LIST, Importance.HIGH, "docs") + .define("e", Type.DOUBLE, Importance.HIGH, "docs") + .define("f", Type.CLASS, Importance.HIGH, "docs") + .define("g", Type.BOOLEAN, Importance.HIGH, "docs") + .define("h", Type.BOOLEAN, Importance.HIGH, "docs") + .define("i", Type.BOOLEAN, Importance.HIGH, "docs"); Properties props = new Properties(); props.put("a", "1 "); @@ -50,6 +48,8 @@ public void testBasicTypes() { props.put("e", 42.5d); props.put("f", String.class.getName()); props.put("g", "true"); + props.put("h", "FalSE"); + props.put("i", "TRUE"); Map vals = def.parse(props); assertEquals(1, vals.get("a")); @@ -59,26 +59,28 @@ public void testBasicTypes() { assertEquals(42.5d, vals.get("e")); assertEquals(String.class, vals.get("f")); assertEquals(true, vals.get("g")); + assertEquals(false, vals.get("h")); + assertEquals(true, vals.get("i")); } @Test(expected = ConfigException.class) public void testInvalidDefault() { - new ConfigDef().define("a", Type.INT, "hello", "docs"); + new ConfigDef().define("a", Type.INT, "hello", Importance.HIGH, "docs"); } @Test(expected = ConfigException.class) public void testNullDefault() { - new ConfigDef().define("a", Type.INT, null, null, "docs"); + new ConfigDef().define("a", Type.INT, null, null, null, "docs"); } @Test(expected = ConfigException.class) public void testMissingRequired() { - new ConfigDef().define("a", Type.INT, "docs").parse(new HashMap()); + new ConfigDef().define("a", Type.INT, Importance.HIGH, "docs").parse(new HashMap()); } @Test(expected = ConfigException.class) public void testDefinedTwice() { - new ConfigDef().define("a", Type.STRING, "docs").define("a", Type.INT, "docs"); + new ConfigDef().define("a", Type.STRING, Importance.HIGH, "docs").define("a", Type.INT, Importance.HIGH, "docs"); } @Test @@ -88,13 +90,14 @@ public void testBadInputs() { testBadInputs(Type.DOUBLE, "hello", null, new Object()); testBadInputs(Type.STRING, new Object()); testBadInputs(Type.LIST, 53, new Object()); + testBadInputs(Type.BOOLEAN, "hello", "truee", "fals"); } private void testBadInputs(Type type, Object... values) { for (Object value : values) { Map m = new HashMap(); m.put("name", value); - ConfigDef def = new ConfigDef().define("name", type, "docs"); + ConfigDef def = new ConfigDef().define("name", type, Importance.HIGH, "docs"); try { def.parse(m); fail("Expected a config exception on bad input for value " + value); @@ -103,4 +106,42 @@ private void testBadInputs(Type type, Object... values) { } } } + + @Test(expected = ConfigException.class) + public void testInvalidDefaultRange() { + new ConfigDef().define("name", Type.INT, -1, Range.between(0, 10), Importance.HIGH, "docs"); + } + + @Test(expected = ConfigException.class) + public void testInvalidDefaultString() { + new ConfigDef().define("name", Type.STRING, "bad", ValidString.in("valid", "values"), Importance.HIGH, "docs"); + } + + @Test + public void testValidators() { + testValidators(Type.INT, Range.between(0, 10), 5, new Object[]{1, 5, 9}, new Object[]{-1, 11}); + testValidators(Type.STRING, ValidString.in("good", "values", "default"), "default", + new Object[]{"good", "values", "default"}, new Object[]{"bad", "inputs"}); + } + + private void testValidators(Type type, Validator validator, Object defaultVal, Object[] okValues, Object[] badValues) { + ConfigDef def = new ConfigDef().define("name", type, defaultVal, validator, Importance.HIGH, "docs"); + + for (Object value : okValues) { + Map m = new HashMap(); + m.put("name", value); + def.parse(m); + } + + for (Object value : badValues) { + Map m = new HashMap(); + m.put("name", value); + try { + def.parse(m); + fail("Expected a config exception due to invalid value " + value); + } catch (ConfigException e) { + // this is good + } + } + } } diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java b/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java new file mode 100644 index 0000000000000..7c7ead11d211e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java @@ -0,0 +1,32 @@ +/** + * 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 org.apache.kafka.common.metrics; + +import java.util.List; +import java.util.Map; + +public class FakeMetricsReporter implements MetricsReporter { + + @Override + public void configure(Map configs) {} + + @Override + public void init(List metrics) {} + + @Override + public void metricChange(KafkaMetric metric) {} + + @Override + public void close() {} + +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java index 2f43c49450e1a..07b1b60d3a9cb 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java @@ -16,10 +16,7 @@ */ package org.apache.kafka.common.metrics; - -import org.apache.kafka.common.metrics.JmxReporter; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Total; import org.junit.Test; @@ -31,10 +28,10 @@ public void testJmxRegistration() throws Exception { Metrics metrics = new Metrics(); metrics.addReporter(new JmxReporter()); Sensor sensor = metrics.sensor("kafka.requests"); - sensor.add("pack.bean1.avg", new Avg()); - sensor.add("pack.bean2.total", new Total()); + sensor.add(new MetricName("pack.bean1.avg", "grp1"), new Avg()); + sensor.add(new MetricName("pack.bean2.total", "grp2"), new Total()); Sensor sensor2 = metrics.sensor("kafka.blah"); - sensor2.add("pack.bean1.some", new Total()); - sensor2.add("pack.bean2.some", new Total()); + sensor2.add(new MetricName("pack.bean1.some", "grp1"), new Total()); + sensor2.add(new MetricName("pack.bean2.some", "grp1"), new Total()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index fdd89141579b6..544e120594de7 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.common.metrics; @@ -20,53 +16,65 @@ import static org.junit.Assert.fail; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.TimeUnit; - import org.apache.kafka.common.Metric; -import org.apache.kafka.common.metrics.JmxReporter; -import org.apache.kafka.common.metrics.Measurable; -import org.apache.kafka.common.metrics.MetricConfig; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.metrics.MetricsReporter; -import org.apache.kafka.common.metrics.Quota; -import org.apache.kafka.common.metrics.QuotaViolationException; -import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Count; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Min; import org.apache.kafka.common.metrics.stats.Percentile; import org.apache.kafka.common.metrics.stats.Percentiles; +import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.metrics.stats.Total; -import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing; import org.apache.kafka.common.utils.MockTime; import org.junit.Test; public class MetricsTest { - private static double EPS = 0.000001; + private static final double EPS = 0.000001; MockTime time = new MockTime(); Metrics metrics = new Metrics(new MetricConfig(), Arrays.asList((MetricsReporter) new JmxReporter()), time); + @Test + public void testMetricName() { + MetricName n1 = new MetricName("name", "group", "description", "key1", "value1"); + Map tags = new HashMap(); + tags.put("key1", "value1"); + MetricName n2 = new MetricName("name", "group", "description", tags); + assertEquals("metric names created in two different ways should be equal", n1, n2); + + try { + new MetricName("name", "group", "description", "key1"); + fail("Creating MetricName with an old number of keyValue should fail"); + } catch (IllegalArgumentException e) { + // this is expected + } + } + @Test public void testSimpleStats() throws Exception { ConstantMeasurable measurable = new ConstantMeasurable(); - metrics.addMetric("direct.measurable", measurable); + + metrics.addMetric(new MetricName("direct.measurable", "grp1", "The fraction of time an appender waits for space allocation."), measurable); Sensor s = metrics.sensor("test.sensor"); - s.add("test.avg", new Avg()); - s.add("test.max", new Max()); - s.add("test.min", new Min()); - s.add("test.rate", new Rate(TimeUnit.SECONDS)); - s.add("test.occurences", new Rate(TimeUnit.SECONDS, new Count())); - s.add("test.count", new Count()); - s.add(new Percentiles(100, -100, 100, BucketSizing.CONSTANT, new Percentile("test.median", 50.0), new Percentile("test.perc99_9", - 99.9))); + s.add(new MetricName("test.avg", "grp1"), new Avg()); + s.add(new MetricName("test.max", "grp1"), new Max()); + s.add(new MetricName("test.min", "grp1"), new Min()); + s.add(new MetricName("test.rate", "grp1"), new Rate(TimeUnit.SECONDS)); + s.add(new MetricName("test.occurences", "grp1"), new Rate(TimeUnit.SECONDS, new Count())); + s.add(new MetricName("test.count", "grp1"), new Count()); + s.add(new Percentiles(100, -100, 100, BucketSizing.CONSTANT, + new Percentile(new MetricName("test.median", "grp1"), 50.0), + new Percentile(new MetricName("test.perc99_9", "grp1"), 99.9))); Sensor s2 = metrics.sensor("test.sensor2"); - s2.add("s2.total", new Total()); + s2.add(new MetricName("s2.total", "grp1"), new Total()); s2.record(5.0); for (int i = 0; i < 10; i++) @@ -75,27 +83,27 @@ public void testSimpleStats() throws Exception { // pretend 2 seconds passed... time.sleep(2000); - assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get("s2.total").value(), EPS); - assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get("test.avg").value(), EPS); - assertEquals("Max(0...9) = 9", 9.0, metrics.metrics().get("test.max").value(), EPS); - assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get("test.min").value(), EPS); - assertEquals("Rate(0...9) = 22.5", 22.5, metrics.metrics().get("test.rate").value(), EPS); - assertEquals("Occurences(0...9) = 5", 5.0, metrics.metrics().get("test.occurences").value(), EPS); - assertEquals("Count(0...9) = 10", 10.0, metrics.metrics().get("test.count").value(), EPS); + assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get(new MetricName("s2.total", "grp1")).value(), EPS); + assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get(new MetricName("test.avg", "grp1")).value(), EPS); + assertEquals("Max(0...9) = 9", 9.0, metrics.metrics().get(new MetricName("test.max", "grp1")).value(), EPS); + assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get(new MetricName("test.min", "grp1")).value(), EPS); + assertEquals("Rate(0...9) = 22.5", 22.5, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); + assertEquals("Occurences(0...9) = 5", 5.0, metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); + assertEquals("Count(0...9) = 10", 10.0, metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS); } @Test public void testHierarchicalSensors() { Sensor parent1 = metrics.sensor("test.parent1"); - parent1.add("test.parent1.count", new Count()); + parent1.add(new MetricName("test.parent1.count", "grp1"), new Count()); Sensor parent2 = metrics.sensor("test.parent2"); - parent2.add("test.parent2.count", new Count()); + parent2.add(new MetricName("test.parent2.count", "grp1"), new Count()); Sensor child1 = metrics.sensor("test.child1", parent1, parent2); - child1.add("test.child1.count", new Count()); + child1.add(new MetricName("test.child1.count", "grp1"), new Count()); Sensor child2 = metrics.sensor("test.child2", parent1); - child2.add("test.child2.count", new Count()); + child2.add(new MetricName("test.child2.count", "grp1"), new Count()); Sensor grandchild = metrics.sensor("test.grandchild", child1); - grandchild.add("test.grandchild.count", new Count()); + grandchild.add(new MetricName("test.grandchild.count", "grp1"), new Count()); /* increment each sensor one time */ parent1.record(); @@ -130,47 +138,48 @@ public void testBadSensorHiearchy() { public void testEventWindowing() { Count count = new Count(); MetricConfig config = new MetricConfig().eventWindow(1).samples(2); - count.record(config, 1.0, time.nanoseconds()); - count.record(config, 1.0, time.nanoseconds()); - assertEquals(2.0, count.measure(config, time.nanoseconds()), EPS); - count.record(config, 1.0, time.nanoseconds()); // first event times out - assertEquals(2.0, count.measure(config, time.nanoseconds()), EPS); + count.record(config, 1.0, time.milliseconds()); + count.record(config, 1.0, time.milliseconds()); + assertEquals(2.0, count.measure(config, time.milliseconds()), EPS); + count.record(config, 1.0, time.milliseconds()); // first event times out + assertEquals(2.0, count.measure(config, time.milliseconds()), EPS); } @Test public void testTimeWindowing() { Count count = new Count(); MetricConfig config = new MetricConfig().timeWindow(1, TimeUnit.MILLISECONDS).samples(2); - count.record(config, 1.0, time.nanoseconds()); + count.record(config, 1.0, time.milliseconds()); time.sleep(1); - count.record(config, 1.0, time.nanoseconds()); - assertEquals(2.0, count.measure(config, time.nanoseconds()), EPS); + count.record(config, 1.0, time.milliseconds()); + assertEquals(2.0, count.measure(config, time.milliseconds()), EPS); time.sleep(1); - count.record(config, 1.0, time.nanoseconds()); // oldest event times out - assertEquals(2.0, count.measure(config, time.nanoseconds()), EPS); + count.record(config, 1.0, time.milliseconds()); // oldest event times out + assertEquals(2.0, count.measure(config, time.milliseconds()), EPS); } @Test public void testOldDataHasNoEffect() { Max max = new Max(); long windowMs = 100; - MetricConfig config = new MetricConfig().timeWindow(windowMs, TimeUnit.MILLISECONDS); - max.record(config, 50, time.nanoseconds()); - time.sleep(windowMs); - assertEquals(Double.NEGATIVE_INFINITY, max.measure(config, time.nanoseconds()), EPS); + int samples = 2; + MetricConfig config = new MetricConfig().timeWindow(windowMs, TimeUnit.MILLISECONDS).samples(samples); + max.record(config, 50, time.milliseconds()); + time.sleep(samples * windowMs); + assertEquals(Double.NEGATIVE_INFINITY, max.measure(config, time.milliseconds()), EPS); } @Test(expected = IllegalArgumentException.class) public void testDuplicateMetricName() { - metrics.sensor("test").add("test", new Avg()); - metrics.sensor("test2").add("test", new Total()); + metrics.sensor("test").add(new MetricName("test", "grp1"), new Avg()); + metrics.sensor("test2").add(new MetricName("test", "grp1"), new Total()); } @Test public void testQuotas() { Sensor sensor = metrics.sensor("test"); - sensor.add("test1.total", new Total(), new MetricConfig().quota(Quota.lessThan(5.0))); - sensor.add("test2.total", new Total(), new MetricConfig().quota(Quota.moreThan(0.0))); + sensor.add(new MetricName("test1.total", "grp1"), new Total(), new MetricConfig().quota(Quota.lessThan(5.0))); + sensor.add(new MetricName("test2.total", "grp1"), new Total(), new MetricConfig().quota(Quota.moreThan(0.0))); sensor.record(5.0); try { sensor.record(1.0); @@ -178,7 +187,7 @@ public void testQuotas() { } catch (QuotaViolationException e) { // this is good } - assertEquals(6.0, metrics.metrics().get("test1.total").value(), EPS); + assertEquals(6.0, metrics.metrics().get(new MetricName("test1.total", "grp1")).value(), EPS); sensor.record(-6.0); try { sensor.record(-1.0); @@ -195,15 +204,15 @@ public void testPercentiles() { 0.0, 100.0, BucketSizing.CONSTANT, - new Percentile("test.p25", 25), - new Percentile("test.p50", 50), - new Percentile("test.p75", 75)); + new Percentile(new MetricName("test.p25", "grp1"), 25), + new Percentile(new MetricName("test.p50", "grp1"), 50), + new Percentile(new MetricName("test.p75", "grp1"), 75)); MetricConfig config = new MetricConfig().eventWindow(50).samples(2); Sensor sensor = metrics.sensor("test", config); sensor.add(percs); - Metric p25 = this.metrics.metrics().get("test.p25"); - Metric p50 = this.metrics.metrics().get("test.p50"); - Metric p75 = this.metrics.metrics().get("test.p75"); + Metric p25 = this.metrics.metrics().get(new MetricName("test.p25", "grp1")); + Metric p50 = this.metrics.metrics().get(new MetricName("test.p50", "grp1")); + Metric p75 = this.metrics.metrics().get(new MetricName("test.p75", "grp1")); // record two windows worth of sequential values for (int i = 0; i < buckets; i++) diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java index 3be6b2d5c718f..a55cc3226c68e 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java @@ -22,7 +22,6 @@ import java.util.Random; -import org.apache.kafka.common.metrics.stats.Histogram; import org.apache.kafka.common.metrics.stats.Histogram.BinScheme; import org.apache.kafka.common.metrics.stats.Histogram.ConstantBinScheme; import org.apache.kafka.common.metrics.stats.Histogram.LinearBinScheme; diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index 865996c1fb5d8..158f9829ff64a 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -1,22 +1,17 @@ /** - * 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. + * 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 org.apache.kafka.common.network; -import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -27,16 +22,11 @@ import java.net.ServerSocket; import java.net.Socket; import java.nio.ByteBuffer; -import java.nio.channels.UnresolvedAddressException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; +import java.util.*; - -import org.apache.kafka.common.network.NetworkReceive; -import org.apache.kafka.common.network.NetworkSend; -import org.apache.kafka.common.network.Selectable; -import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; import org.junit.After; @@ -48,17 +38,18 @@ */ public class SelectorTest { - private static final List EMPTY = new ArrayList(); private static final int BUFFER_SIZE = 4 * 1024; private EchoServer server; + private Time time; private Selectable selector; @Before public void setup() throws Exception { this.server = new EchoServer(); this.server.start(); - this.selector = new Selector(); + this.time = new MockTime(); + this.selector = new Selector(5000, new Metrics(), time, "MetricGroup", new LinkedHashMap()); } @After @@ -72,7 +63,7 @@ public void teardown() throws Exception { */ @Test public void testServerDisconnect() throws Exception { - int node = 0; + String node = "0"; // connect and do a simple request blockingConnect(node); @@ -81,7 +72,7 @@ public void testServerDisconnect() throws Exception { // disconnect this.server.closeConnections(); while (!selector.disconnected().contains(node)) - selector.poll(1000L, EMPTY); + selector.poll(1000L); // reconnect and do another request blockingConnect(node); @@ -93,10 +84,11 @@ public void testServerDisconnect() throws Exception { */ @Test public void testClientDisconnect() throws Exception { - int node = 0; + String node = "0"; blockingConnect(node); selector.disconnect(node); - selector.poll(10, asList(createSend(node, "hello1"))); + selector.send(createSend(node, "hello1")); + selector.poll(10); assertEquals("Request should not have succeeded", 0, selector.completedSends().size()); assertEquals("There should be a disconnect", 1, selector.disconnected().size()); assertTrue("The disconnect should be from our node", selector.disconnected().contains(node)); @@ -109,9 +101,11 @@ public void testClientDisconnect() throws Exception { */ @Test(expected = IllegalStateException.class) public void testCantSendWithInProgress() throws Exception { - int node = 0; + String node = "0"; blockingConnect(node); - selector.poll(1000L, asList(createSend(node, "test1"), createSend(node, "test2"))); + selector.send(createSend(node, "test1")); + selector.send(createSend(node, "test2")); + selector.poll(1000L); } /** @@ -119,15 +113,16 @@ public void testCantSendWithInProgress() throws Exception { */ @Test(expected = IllegalStateException.class) public void testCantSendWithoutConnecting() throws Exception { - selector.poll(1000L, asList(createSend(0, "test"))); + selector.send(createSend("0", "test")); + selector.poll(1000L); } /** * Sending a request to a node with a bad hostname should result in an exception during connect */ - @Test(expected = UnresolvedAddressException.class) + @Test(expected = IOException.class) public void testNoRouteToHost() throws Exception { - selector.connect(0, new InetSocketAddress("asdf.asdf.dsc", server.port), BUFFER_SIZE, BUFFER_SIZE); + selector.connect("0", new InetSocketAddress("asdf.asdf.dsc", server.port), BUFFER_SIZE, BUFFER_SIZE); } /** @@ -135,10 +130,13 @@ public void testNoRouteToHost() throws Exception { */ @Test public void testConnectionRefused() throws Exception { - int node = 0; - selector.connect(node, new InetSocketAddress("localhost", TestUtils.choosePort()), BUFFER_SIZE, BUFFER_SIZE); + String node = "0"; + ServerSocket nonListeningSocket = new ServerSocket(0); + int nonListeningPort = nonListeningSocket.getLocalPort(); + selector.connect(node, new InetSocketAddress("localhost", nonListeningPort), BUFFER_SIZE, BUFFER_SIZE); while (selector.disconnected().contains(node)) - selector.poll(1000L, EMPTY); + selector.poll(1000L); + nonListeningSocket.close(); } /** @@ -153,20 +151,20 @@ public void testNormalOperation() throws Exception { // create connections InetSocketAddress addr = new InetSocketAddress("localhost", server.port); for (int i = 0; i < conns; i++) - selector.connect(i, addr, BUFFER_SIZE, BUFFER_SIZE); - + selector.connect(Integer.toString(i), addr, BUFFER_SIZE, BUFFER_SIZE); // send echo requests and receive responses - int[] requests = new int[conns]; - int[] responses = new int[conns]; + Map requests = new HashMap(); + Map responses = new HashMap(); int responseCount = 0; - List sends = new ArrayList(); - for (int i = 0; i < conns; i++) - sends.add(createSend(i, i + "-" + 0)); + for (int i = 0; i < conns; i++) { + String node = Integer.toString(i); + selector.send(createSend(node, node + "-0")); + } // loop until we complete all requests while (responseCount < conns * reqs) { // do the i/o - selector.poll(0L, sends); + selector.poll(0L); assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size()); @@ -174,20 +172,27 @@ public void testNormalOperation() throws Exception { for (NetworkReceive receive : selector.completedReceives()) { String[] pieces = asString(receive).split("-"); assertEquals("Should be in the form 'conn-counter'", 2, pieces.length); - assertEquals("Check the source", receive.source(), Integer.parseInt(pieces[0])); + assertEquals("Check the source", receive.source(), pieces[0]); assertEquals("Check that the receive has kindly been rewound", 0, receive.payload().position()); - assertEquals("Check the request counter", responses[receive.source()], Integer.parseInt(pieces[1])); - responses[receive.source()]++; // increment the expected counter + if (responses.containsKey(receive.source())) { + assertEquals("Check the request counter", (int) responses.get(receive.source()), Integer.parseInt(pieces[1])); + responses.put(receive.source(), responses.get(receive.source()) + 1); + } else { + assertEquals("Check the request counter", 0, Integer.parseInt(pieces[1])); + responses.put(receive.source(), 1); + } responseCount++; } // prepare new sends for the next round - sends.clear(); - for (NetworkSend send : selector.completedSends()) { - int dest = send.destination(); - requests[dest]++; - if (requests[dest] < reqs) - sends.add(createSend(dest, dest + "-" + requests[dest])); + for (Send send : selector.completedSends()) { + String dest = send.destination(); + if (requests.containsKey(dest)) + requests.put(dest, requests.get(dest) + 1); + else + requests.put(dest, 1); + if (requests.get(dest) < reqs) + selector.send(createSend(dest, dest + "-" + requests.get(dest))); } } } @@ -197,7 +202,7 @@ public void testNormalOperation() throws Exception { */ @Test public void testSendLargeRequest() throws Exception { - int node = 0; + String node = "0"; blockingConnect(node); String big = TestUtils.randomString(10 * BUFFER_SIZE); assertEquals(big, blockingRequest(node, big)); @@ -208,15 +213,57 @@ public void testSendLargeRequest() throws Exception { */ @Test public void testEmptyRequest() throws Exception { - int node = 0; + String node = "0"; blockingConnect(node); assertEquals("", blockingRequest(node, "")); } - private String blockingRequest(int node, String s) throws IOException { - selector.poll(1000L, asList(createSend(node, s))); + @Test(expected = IllegalStateException.class) + public void testExistingConnectionId() throws IOException { + blockingConnect("0"); + blockingConnect("0"); + } + + @Test + public void testMute() throws Exception { + blockingConnect("0"); + blockingConnect("1"); + + selector.send(createSend("0", "hello")); + selector.send(createSend("1", "hi")); + + selector.mute("1"); + + while (selector.completedReceives().isEmpty()) + selector.poll(5); + assertEquals("We should have only one response", 1, selector.completedReceives().size()); + assertEquals("The response should not be from the muted node", "0", selector.completedReceives().get(0).source()); + + selector.unmute("1"); + do { + selector.poll(5); + } while (selector.completedReceives().isEmpty()); + assertEquals("We should have only one response", 1, selector.completedReceives().size()); + assertEquals("The response should be from the previously muted node", "1", selector.completedReceives().get(0).source()); + } + + + @Test + public void testCloseOldestConnection() throws Exception { + String id = "0"; + blockingConnect(id); + + time.sleep(6000); // The max idle time is 5000ms + selector.poll(0); + + assertTrue("The idle connection should have been closed", selector.disconnected().contains(id)); + } + + private String blockingRequest(String node, String s) throws IOException { + selector.send(createSend(node, s)); + selector.poll(1000L); while (true) { - selector.poll(1000L, EMPTY); + selector.poll(1000L); for (NetworkReceive receive : selector.completedReceives()) if (receive.source() == node) return asString(receive); @@ -224,13 +271,13 @@ private String blockingRequest(int node, String s) throws IOException { } /* connect and wait for the connection to complete */ - private void blockingConnect(int node) throws IOException { + private void blockingConnect(String node) throws IOException { selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE); while (!selector.connected().contains(node)) - selector.poll(10000L, EMPTY); + selector.poll(10000L); } - private NetworkSend createSend(int node, String s) { + private NetworkSend createSend(String node, String s) { return new NetworkSend(node, ByteBuffer.wrap(s.getBytes())); } @@ -248,8 +295,8 @@ static class EchoServer extends Thread { private final List sockets; public EchoServer() throws Exception { - this.port = TestUtils.choosePort(); - this.serverSocket = new ServerSocket(port); + this.serverSocket = new ServerSocket(0); + this.port = this.serverSocket.getLocalPort(); this.threads = Collections.synchronizedList(new ArrayList()); this.sockets = Collections.synchronizedList(new ArrayList()); } diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java index 4480e9b2aafe6..6c335a1202487 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java @@ -23,12 +23,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; -import org.apache.kafka.common.protocol.types.ArrayOf; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.SchemaException; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.protocol.types.Type; import org.junit.Before; import org.junit.Test; @@ -49,12 +43,12 @@ public void setup() { new Field("struct", new Schema(new Field("field", Type.INT32)))); this.struct = new Struct(this.schema).set("int8", (byte) 1) .set("int16", (short) 1) - .set("int32", (int) 1) - .set("int64", (long) 1) + .set("int32", 1) + .set("int64", 1L) .set("string", "1") .set("bytes", "1".getBytes()) - .set("array", new Object[] { 1 }); - this.struct.set("struct", this.struct.instance("struct").set("field", new Object[] { 1, 2, 3 })); + .set("array", new Object[] {1}); + this.struct.set("struct", this.struct.instance("struct").set("field", new Object[] {1, 2, 3})); } @Test @@ -68,9 +62,9 @@ public void testSimple() { check(Type.STRING, "A\u00ea\u00f1\u00fcC"); check(Type.BYTES, ByteBuffer.allocate(0)); check(Type.BYTES, ByteBuffer.wrap("abcd".getBytes())); - check(new ArrayOf(Type.INT32), new Object[] { 1, 2, 3, 4 }); + check(new ArrayOf(Type.INT32), new Object[] {1, 2, 3, 4}); check(new ArrayOf(Type.STRING), new Object[] {}); - check(new ArrayOf(Type.STRING), new Object[] { "hello", "there", "beautiful" }); + check(new ArrayOf(Type.STRING), new Object[] {"hello", "there", "beautiful"}); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index b0745b528cef9..8ec610ae4693d 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -22,29 +22,37 @@ import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; +import java.util.*; -import org.apache.kafka.common.record.LogEntry; -import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.Record; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +@RunWith(value = Parameterized.class) public class MemoryRecordsTest { + private CompressionType compression; + + public MemoryRecordsTest(CompressionType compression) { + this.compression = compression; + } + @Test public void testIterator() { - MemoryRecords recs1 = new MemoryRecords(ByteBuffer.allocate(1024)); - MemoryRecords recs2 = new MemoryRecords(ByteBuffer.allocate(1024)); + MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression); + MemoryRecords recs2 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression); List list = Arrays.asList(new Record("a".getBytes(), "1".getBytes()), new Record("b".getBytes(), "2".getBytes()), new Record("c".getBytes(), "3".getBytes())); for (int i = 0; i < list.size(); i++) { Record r = list.get(i); recs1.append(i, r); - recs2.append(i, toArray(r.key()), toArray(r.value()), r.compressionType()); + recs2.append(i, toArray(r.key()), toArray(r.value())); } + recs1.close(); + recs1.flip(); + recs2.close(); + recs2.flip(); for (int iteration = 0; iteration < 2; iteration++) { for (MemoryRecords recs : Arrays.asList(recs1, recs2)) { @@ -54,10 +62,18 @@ public void testIterator() { LogEntry entry = iter.next(); assertEquals((long) i, entry.offset()); assertEquals(list.get(i), entry.record()); + entry.record().ensureValid(); } assertFalse(iter.hasNext()); } } } + @Parameterized.Parameters + public static Collection data() { + List values = new ArrayList(); + for (CompressionType type: CompressionType.values()) + values.add(new Object[] {type}); + return values; + } } diff --git a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java index ae54d67da9907..957fc8fa3999c 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java @@ -27,9 +27,6 @@ import java.util.Collection; import java.util.List; -import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.InvalidRecordException; -import org.apache.kafka.common.record.Record; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -66,6 +63,10 @@ public void testFields() { @Test public void testChecksum() { assertEquals(record.checksum(), record.computeChecksum()); + assertEquals(record.checksum(), Record.computeChecksum( + this.key == null ? null : this.key.array(), + this.value == null ? null : this.value.array(), + this.compression, 0, -1)); assertTrue(record.isValid()); for (int i = Record.CRC_OFFSET + Record.CRC_LENGTH; i < record.size(); i++) { Record copy = copyOf(record); @@ -95,11 +96,13 @@ public void testEquality() { @Parameters public static Collection data() { + byte[] payload = new byte[1000]; + Arrays.fill(payload, (byte) 1); List values = new ArrayList(); - for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes())) - for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes())) + for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload)) + for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload)) for (CompressionType compression : CompressionType.values()) - values.add(new Object[] { key, value, compression }); + values.add(new Object[] {key, value, compression}); return values; } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java new file mode 100644 index 0000000000000..8b2aca85fa738 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -0,0 +1,187 @@ +/** + * 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 org.apache.kafka.common.requests; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.protocol.Errors; +import org.junit.Test; + +import java.lang.reflect.Method; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class RequestResponseTest { + + @Test + public void testSerialization() throws Exception { + List requestResponseList = Arrays.asList( + createRequestHeader(), + createResponseHeader(), + createConsumerMetadataRequest(), + createConsumerMetadataRequest().getErrorResponse(0, new UnknownServerException()), + createConsumerMetadataResponse(), + createFetchRequest(), + createFetchRequest().getErrorResponse(0, new UnknownServerException()), + createFetchResponse(), + createHeartBeatRequest(), + createHeartBeatRequest().getErrorResponse(0, new UnknownServerException()), + createHeartBeatResponse(), + createJoinGroupRequest(), + createJoinGroupRequest().getErrorResponse(0, new UnknownServerException()), + createJoinGroupResponse(), + createListOffsetRequest(), + createListOffsetRequest().getErrorResponse(0, new UnknownServerException()), + createListOffsetResponse(), + createMetadataRequest(), + createMetadataRequest().getErrorResponse(0, new UnknownServerException()), + createMetadataResponse(), + createOffsetCommitRequest(), + createOffsetCommitRequest().getErrorResponse(0, new UnknownServerException()), + createOffsetCommitResponse(), + createOffsetFetchRequest(), + createOffsetFetchRequest().getErrorResponse(0, new UnknownServerException()), + createOffsetFetchResponse(), + createProduceRequest(), + createProduceRequest().getErrorResponse(0, new UnknownServerException()), + createProduceResponse()); + + for (AbstractRequestResponse req: requestResponseList) { + ByteBuffer buffer = ByteBuffer.allocate(req.sizeOf()); + req.writeTo(buffer); + buffer.rewind(); + Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class); + AbstractRequestResponse deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer); + assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should be the same.", req, deserialized); + assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should have the same hashcode.", + req.hashCode(), deserialized.hashCode()); + } + } + + private AbstractRequestResponse createRequestHeader() { + return new RequestHeader((short) 10, (short) 1, "", 10); + } + + private AbstractRequestResponse createResponseHeader() { + return new ResponseHeader(10); + } + + private AbstractRequest createConsumerMetadataRequest() { + return new ConsumerMetadataRequest("test-group"); + } + + private AbstractRequestResponse createConsumerMetadataResponse() { + return new ConsumerMetadataResponse((short) 1, new Node(10, "host1", 2014)); + } + + private AbstractRequest createFetchRequest() { + Map fetchData = new HashMap(); + fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 1000000)); + fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 1000000)); + return new FetchRequest(-1, 100, 100000, fetchData); + } + + private AbstractRequestResponse createFetchResponse() { + Map responseData = new HashMap(); + responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE.code(), 1000000, ByteBuffer.allocate(10))); + return new FetchResponse(responseData); + } + + private AbstractRequest createHeartBeatRequest() { + return new HeartbeatRequest("group1", 1, "consumer1"); + } + + private AbstractRequestResponse createHeartBeatResponse() { + return new HeartbeatResponse(Errors.NONE.code()); + } + + private AbstractRequest createJoinGroupRequest() { + return new JoinGroupRequest("group1", 30000, Arrays.asList("topic1"), "consumer1", "strategy1"); + } + + private AbstractRequestResponse createJoinGroupResponse() { + return new JoinGroupResponse(Errors.NONE.code(), 1, "consumer1", Arrays.asList(new TopicPartition("test11", 1), new TopicPartition("test2", 1))); + } + + private AbstractRequest createListOffsetRequest() { + Map offsetData = new HashMap(); + offsetData.put(new TopicPartition("test", 0), new ListOffsetRequest.PartitionData(1000000L, 10)); + return new ListOffsetRequest(-1, offsetData); + } + + private AbstractRequestResponse createListOffsetResponse() { + Map responseData = new HashMap(); + responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData(Errors.NONE.code(), Arrays.asList(100L))); + return new ListOffsetResponse(responseData); + } + + private AbstractRequest createMetadataRequest() { + return new MetadataRequest(Arrays.asList("topic1")); + } + + private AbstractRequestResponse createMetadataResponse() { + Node node = new Node(1, "host1", 1001); + Node[] replicas = new Node[1]; + replicas[0] = node; + Node[] isr = new Node[1]; + isr[0] = node; + Cluster cluster = new Cluster(Arrays.asList(node), Arrays.asList(new PartitionInfo("topic1", 1, node, replicas, isr))); + + Map errors = new HashMap(); + errors.put("topic2", Errors.LEADER_NOT_AVAILABLE); + return new MetadataResponse(cluster, errors); + } + + private AbstractRequest createOffsetCommitRequest() { + Map commitData = new HashMap(); + commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, "")); + return new OffsetCommitRequest("group1", 100, "consumer1", 1000000, commitData); + } + + private AbstractRequestResponse createOffsetCommitResponse() { + Map responseData = new HashMap(); + responseData.put(new TopicPartition("test", 0), Errors.NONE.code()); + return new OffsetCommitResponse(responseData); + } + + private AbstractRequest createOffsetFetchRequest() { + return new OffsetFetchRequest("group1", Arrays.asList(new TopicPartition("test11", 1))); + } + + private AbstractRequestResponse createOffsetFetchResponse() { + Map responseData = new HashMap(); + responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", Errors.NONE.code())); + return new OffsetFetchResponse(responseData); + } + + private AbstractRequest createProduceRequest() { + Map produceData = new HashMap(); + produceData.put(new TopicPartition("test", 0), ByteBuffer.allocate(10)); + return new ProduceRequest((short) 1, 5000, produceData); + } + + private AbstractRequestResponse createProduceResponse() { + Map responseData = new HashMap(); + responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000)); + return new ProduceResponse(responseData); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java new file mode 100644 index 0000000000000..383bf48989d1c --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -0,0 +1,90 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class SerializationTest { + + private static class SerDeser { + final Serializer serializer; + final Deserializer deserializer; + + public SerDeser(Serializer serializer, Deserializer deserializer) { + this.serializer = serializer; + this.deserializer = deserializer; + } + } + + @Test + public void testStringSerializer() { + String str = "my string"; + String mytopic = "testTopic"; + List encodings = new ArrayList(); + encodings.add("UTF8"); + encodings.add("UTF-16"); + + for (String encoding : encodings) { + SerDeser serDeser = getStringSerDeser(encoding); + Serializer serializer = serDeser.serializer; + Deserializer deserializer = serDeser.deserializer; + + assertEquals("Should get the original string after serialization and deserialization with encoding " + encoding, + str, deserializer.deserialize(mytopic, serializer.serialize(mytopic, str))); + + assertEquals("Should support null in serialization and deserialization with encoding " + encoding, + null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null))); + } + } + + @Test + public void testIntegerSerializer() { + Integer[] integers = new Integer[]{ + 423412424, + -41243432 + }; + String mytopic = "testTopic"; + + Serializer serializer = new IntegerSerializer(); + Deserializer deserializer = new IntegerDeserializer(); + + for (Integer integer : integers) { + assertEquals("Should get the original integer after serialization and deserialization", + integer, deserializer.deserialize(mytopic, serializer.serialize(mytopic, integer))); + } + + assertEquals("Should support null in serialization and deserialization", + null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null))); + } + + private SerDeser getStringSerDeser(String encoder) { + Map serializerConfigs = new HashMap(); + serializerConfigs.put("key.serializer.encoding", encoder); + Serializer serializer = new StringSerializer(); + serializer.configure(serializerConfigs, true); + + Map deserializerConfigs = new HashMap(); + deserializerConfigs.put("key.deserializer.encoding", encoder); + Deserializer deserializer = new StringDeserializer(); + deserializer.configure(deserializerConfigs, true); + + return new SerDeser(serializer, deserializer); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java b/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java index 1df226606fad2..c788e66035b24 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java @@ -20,13 +20,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; +import java.util.*; -import org.apache.kafka.common.utils.AbstractIterator; import org.junit.Test; public class AbstractIteratorTest { @@ -49,7 +44,7 @@ public void testIterator() { @Test(expected = NoSuchElementException.class) public void testEmptyIterator() { - Iterator iter = new ListIterator(Arrays.asList()); + Iterator iter = new ListIterator(Collections.emptyList()); iter.next(); } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java b/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java new file mode 100644 index 0000000000000..c39c3cff64d4e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java @@ -0,0 +1,59 @@ +/** + * 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 org.apache.kafka.common.utils; + +import static org.junit.Assert.assertEquals; +import org.junit.Test; + +import java.nio.ByteBuffer; + +public class CrcTest { + + @Test + public void testUpdate() { + final byte[] bytes = "Any String you want".getBytes(); + final int len = bytes.length; + + Crc32 crc1 = new Crc32(); + Crc32 crc2 = new Crc32(); + Crc32 crc3 = new Crc32(); + + crc1.update(bytes, 0, len); + for (int i = 0; i < len; i++) + crc2.update(bytes[i]); + crc3.update(bytes, 0, len / 2); + crc3.update(bytes, len / 2, len - len / 2); + + assertEquals("Crc values should be the same", crc1.getValue(), crc2.getValue()); + assertEquals("Crc values should be the same", crc1.getValue(), crc3.getValue()); + } + + @Test + public void testUpdateInt() { + final int value = 1000; + final ByteBuffer buffer = ByteBuffer.allocate(4); + buffer.putInt(value); + + Crc32 crc1 = new Crc32(); + Crc32 crc2 = new Crc32(); + + crc1.updateInt(value); + crc2.update(buffer.array(), buffer.arrayOffset(), 4); + + assertEquals("Crc values should be the same", crc1.getValue(), crc2.getValue()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java index cda8e644587aa..eb7fcf07e529e 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java @@ -1,25 +1,22 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * 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 org.apache.kafka.common.utils; import java.util.concurrent.TimeUnit; -import org.apache.kafka.common.utils.Time; - +/** + * A clock that you can manually advance by calling sleep + */ public class MockTime implements Time { private long nanos = 0; diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java new file mode 100755 index 0000000000000..e7951d835472e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -0,0 +1,111 @@ +/** + * 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 org.apache.kafka.common.utils; + +import java.util.Arrays; +import java.util.Collections; +import java.nio.ByteBuffer; + +import org.junit.Test; + +import static org.apache.kafka.common.utils.Utils.getHost; +import static org.apache.kafka.common.utils.Utils.getPort; +import static org.apache.kafka.common.utils.Utils.formatAddress; +import static org.junit.Assert.*; + +public class UtilsTest { + + @Test + public void testGetHost() { + assertEquals("127.0.0.1", getHost("127.0.0.1:8000")); + assertEquals("mydomain.com", getHost("PLAINTEXT://mydomain.com:8080")); + assertEquals("::1", getHost("[::1]:1234")); + assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); + } + + @Test + public void testGetPort() { + assertEquals(8000, getPort("127.0.0.1:8000").intValue()); + assertEquals(8080, getPort("mydomain.com:8080").intValue()); + assertEquals(1234, getPort("[::1]:1234").intValue()); + assertEquals(5678, getPort("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678").intValue()); + } + + @Test + public void testFormatAddress() { + assertEquals("127.0.0.1:8000", formatAddress("127.0.0.1", 8000)); + assertEquals("mydomain.com:8080", formatAddress("mydomain.com", 8080)); + assertEquals("[::1]:1234", formatAddress("::1", 1234)); + assertEquals("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678", formatAddress("2001:db8:85a3:8d3:1319:8a2e:370:7348", 5678)); + } + + @Test + public void testJoin() { + assertEquals("", Utils.join(Collections.emptyList(), ",")); + assertEquals("1", Utils.join(Arrays.asList("1"), ",")); + assertEquals("1,2,3", Utils.join(Arrays.asList(1, 2, 3), ",")); + } + + @Test + public void testAbs() { + assertEquals(0, Utils.abs(Integer.MIN_VALUE)); + assertEquals(10, Utils.abs(-10)); + assertEquals(10, Utils.abs(10)); + assertEquals(0, Utils.abs(0)); + assertEquals(1, Utils.abs(-1)); + } + + private void subTest(ByteBuffer buffer) { + // The first byte should be 'A' + assertEquals('A', (Utils.readBytes(buffer, 0, 1))[0]); + + // The offset is 2, so the first 2 bytes should be skipped. + byte[] results = Utils.readBytes(buffer, 2, 3); + assertEquals('y', results[0]); + assertEquals(' ', results[1]); + assertEquals('S', results[2]); + assertEquals(3, results.length); + + // test readBytes without offset and length specified. + results = Utils.readBytes(buffer); + assertEquals('A', results[0]); + assertEquals('t', results[buffer.limit() - 1]); + assertEquals(buffer.limit(), results.length); + } + + @Test + public void testReadBytes() { + byte[] myvar = "Any String you want".getBytes(); + ByteBuffer buffer = ByteBuffer.allocate(myvar.length); + buffer.put(myvar); + buffer.rewind(); + + this.subTest(buffer); + + // test readonly buffer, different path + buffer = ByteBuffer.wrap(myvar).asReadOnlyBuffer(); + this.subTest(buffer); + } + + @Test + public void testMin() { + assertEquals(1, Utils.min(1)); + assertEquals(1, Utils.min(1, 2, 3)); + assertEquals(1, Utils.min(2, 1, 3)); + assertEquals(1, Utils.min(2, 3, 1)); + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/test/MetricsBench.java b/clients/src/test/java/org/apache/kafka/test/MetricsBench.java index 7239b4a56e93f..633d4bbf214a5 100644 --- a/clients/src/test/java/org/apache/kafka/test/MetricsBench.java +++ b/clients/src/test/java/org/apache/kafka/test/MetricsBench.java @@ -1,23 +1,20 @@ /** - * 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. + * 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 org.apache.kafka.test; import java.util.Arrays; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; @@ -27,7 +24,6 @@ import org.apache.kafka.common.metrics.stats.Percentiles; import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing; - public class MetricsBench { public static void main(String[] args) { @@ -36,19 +32,19 @@ public static void main(String[] args) { Sensor parent = metrics.sensor("parent"); Sensor child = metrics.sensor("child", parent); for (Sensor sensor : Arrays.asList(parent, child)) { - sensor.add(sensor.name() + ".avg", new Avg()); - sensor.add(sensor.name() + ".count", new Count()); - sensor.add(sensor.name() + ".max", new Max()); + sensor.add(new MetricName(sensor.name() + ".avg", "grp1"), new Avg()); + sensor.add(new MetricName(sensor.name() + ".count", "grp1"), new Count()); + sensor.add(new MetricName(sensor.name() + ".max", "grp1"), new Max()); sensor.add(new Percentiles(1024, 0.0, iters, BucketSizing.CONSTANT, - new Percentile(sensor.name() + ".median", 50.0), - new Percentile(sensor.name() + ".p_99", 99.0))); + new Percentile(new MetricName(sensor.name() + ".median", "grp1"), 50.0), + new Percentile(new MetricName(sensor.name() + ".p_99", "grp1"), 99.0))); } long start = System.nanoTime(); for (int i = 0; i < iters; i++) - child.record(i); + parent.record(i); double ellapsed = (System.nanoTime() - start) / (double) iters; System.out.println(String.format("%.2f ns per metric recording.", ellapsed)); } diff --git a/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java b/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java index 46cf86e67e5ee..8cd19b2edfa75 100644 --- a/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java +++ b/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.test; @@ -24,11 +20,11 @@ import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantLock; import org.apache.kafka.common.utils.CopyOnWriteMap; import org.apache.kafka.common.utils.SystemTime; - public class Microbenchmarks { public static void main(String[] args) throws Exception { @@ -88,7 +84,7 @@ public void run() { counter++; } } - System.out.println("synchronized: " + ((System.nanoTime() - start) / iters)); + System.out.println("synchronized: " + ((time.nanoseconds() - start) / iters)); System.out.println(counter); done.set(true); } @@ -112,6 +108,43 @@ public void run() { t1.join(); t2.join(); + System.out.println("Testing locks"); + done.set(false); + final ReentrantLock lock2 = new ReentrantLock(); + Thread t3 = new Thread() { + public void run() { + time.sleep(1); + int counter = 0; + long start = time.nanoseconds(); + for (int i = 0; i < iters; i++) { + lock2.lock(); + counter++; + lock2.unlock(); + } + System.out.println("lock: " + ((time.nanoseconds() - start) / iters)); + System.out.println(counter); + done.set(true); + } + }; + + Thread t4 = new Thread() { + public void run() { + int counter = 0; + while (!done.get()) { + time.sleep(1); + lock2.lock(); + counter++; + lock2.unlock(); + } + System.out.println("Counter: " + counter); + } + }; + + t3.start(); + t4.start(); + t3.join(); + t4.join(); + Map values = new HashMap(); for (int i = 0; i < 100; i++) values.put(Integer.toString(i), i); @@ -129,7 +162,6 @@ private static void benchMap(int numThreads, final int iters, final Map metrics) { + INIT_COUNT.incrementAndGet(); + } + + @Override + public void metricChange(KafkaMetric metric) { + + } + + @Override + public void close() { + CLOSE_COUNT.incrementAndGet(); + } + + @Override + public void configure(Map configs) { + + } +} diff --git a/clients/src/test/java/org/apache/kafka/test/MockSelector.java b/clients/src/test/java/org/apache/kafka/test/MockSelector.java index d61de52917331..51eb9d142f566 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockSelector.java +++ b/clients/src/test/java/org/apache/kafka/test/MockSelector.java @@ -1,18 +1,14 @@ /** - * 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. + * 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 org.apache.kafka.test; @@ -24,31 +20,32 @@ import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Selectable; +import org.apache.kafka.common.network.Send; import org.apache.kafka.common.utils.Time; - /** * A fake selector to use for testing */ public class MockSelector implements Selectable { private final Time time; - private final List completedSends = new ArrayList(); + private final List initiatedSends = new ArrayList(); + private final List completedSends = new ArrayList(); private final List completedReceives = new ArrayList(); - private final List disconnected = new ArrayList(); - private final List connected = new ArrayList(); + private final List disconnected = new ArrayList(); + private final List connected = new ArrayList(); public MockSelector(Time time) { this.time = time; } @Override - public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { + public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { this.connected.add(id); } @Override - public void disconnect(int id) { + public void disconnect(String id) { this.disconnected.add(id); } @@ -68,13 +65,19 @@ public void clear() { } @Override - public void poll(long timeout, List sends) throws IOException { - this.completedSends.addAll(sends); + public void send(Send send) { + this.initiatedSends.add(send); + } + + @Override + public void poll(long timeout) throws IOException { + this.completedSends.addAll(this.initiatedSends); + this.initiatedSends.clear(); time.sleep(timeout); } @Override - public List completedSends() { + public List completedSends() { return completedSends; } @@ -92,13 +95,29 @@ public void completeReceive(NetworkReceive receive) { } @Override - public List disconnected() { + public List disconnected() { return disconnected; } @Override - public List connected() { + public List connected() { return connected; } + @Override + public void mute(String id) { + } + + @Override + public void unmute(String id) { + } + + @Override + public void muteAll() { + } + + @Override + public void unmuteAll() { + } + } diff --git a/clients/src/test/java/org/apache/kafka/test/MockSerializer.java b/clients/src/test/java/org/apache/kafka/test/MockSerializer.java new file mode 100644 index 0000000000000..03482581228fc --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/test/MockSerializer.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.test; + +import org.apache.kafka.common.serialization.Serializer; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +public class MockSerializer implements Serializer { + public static final AtomicInteger INIT_COUNT = new AtomicInteger(0); + public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0); + + public MockSerializer() { + INIT_COUNT.incrementAndGet(); + } + + @Override + public void configure(Map configs, boolean isKey) { + } + + @Override + public byte[] serialize(String topic, byte[] data) { + return data; + } + + @Override + public void close() { + CLOSE_COUNT.incrementAndGet(); + } +} diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 36cfc0fda742e..ccf3a5f5f72db 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -19,8 +19,6 @@ import static java.util.Arrays.asList; import java.io.File; -import java.io.IOException; -import java.net.ServerSocket; import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -35,15 +33,15 @@ */ public class TestUtils { - public static File IO_TMP_DIR = new File(System.getProperty("java.io.tmpdir")); + public static final File IO_TMP_DIR = new File(System.getProperty("java.io.tmpdir")); - public static String LETTERS = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"; - public static String DIGITS = "0123456789"; - public static String LETTERS_AND_DIGITS = LETTERS + DIGITS; + public static final String LETTERS = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"; + public static final String DIGITS = "0123456789"; + public static final String LETTERS_AND_DIGITS = LETTERS + DIGITS; /* A consistent random number generator to make tests repeatable */ - public static final Random seededRandom = new Random(192348092834L); - public static final Random random = new Random(); + public static final Random SEEDED_RANDOM = new Random(192348092834L); + public static final Random RANDOM = new Random(); public static Cluster singletonCluster(String topic, int partitions) { return clusterWith(1, topic, partitions); @@ -59,40 +57,14 @@ public static Cluster clusterWith(int nodes, String topic, int partitions) { return new Cluster(asList(ns), parts); } - /** - * Choose a number of random available ports - */ - public static int[] choosePorts(int count) { - try { - ServerSocket[] sockets = new ServerSocket[count]; - int[] ports = new int[count]; - for (int i = 0; i < count; i++) { - sockets[i] = new ServerSocket(0); - ports[i] = sockets[i].getLocalPort(); - } - for (int i = 0; i < count; i++) - sockets[i].close(); - return ports; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - /** - * Choose an available port - */ - public static int choosePort() { - return choosePorts(1)[0]; - } - /** * Generate an array of random bytes * - * @param numBytes The size of the array + * @param size The size of the array */ public static byte[] randomBytes(int size) { byte[] bytes = new byte[size]; - seededRandom.nextBytes(bytes); + SEEDED_RANDOM.nextBytes(bytes); return bytes; } @@ -105,7 +77,7 @@ public static byte[] randomBytes(int size) { public static String randomString(int len) { StringBuilder b = new StringBuilder(); for (int i = 0; i < len; i++) - b.append(LETTERS_AND_DIGITS.charAt(seededRandom.nextInt(LETTERS_AND_DIGITS.length()))); + b.append(LETTERS_AND_DIGITS.charAt(SEEDED_RANDOM.nextInt(LETTERS_AND_DIGITS.length()))); return b.toString(); } diff --git a/system_test/migration_tool_testsuite/0.7/bin/zookeeper-server-start.sh b/clients/src/test/resources/log4j.properties old mode 100755 new mode 100644 similarity index 75% rename from system_test/migration_tool_testsuite/0.7/bin/zookeeper-server-start.sh rename to clients/src/test/resources/log4j.properties index 184a10be05f58..b1d5b7f2b4091 --- a/system_test/migration_tool_testsuite/0.7/bin/zookeeper-server-start.sh +++ b/clients/src/test/resources/log4j.properties @@ -1,23 +1,21 @@ -#!/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. +log4j.rootLogger=OFF, stdout -if [ $# -ne 1 ]; -then - echo "USAGE: $0 zookeeper.properties" - exit 1 -fi +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -$(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@ +log4j.logger.org.apache.kafka=ERROR diff --git a/config/consumer.properties b/config/consumer.properties index 7343cbc28cf8b..83847de30d10b 100644 --- a/config/consumer.properties +++ b/config/consumer.properties @@ -20,7 +20,7 @@ zookeeper.connect=127.0.0.1:2181 # timeout in ms for connecting to zookeeper -zookeeper.connection.timeout.ms=1000000 +zookeeper.connection.timeout.ms=6000 #consumer group id group.id=test-consumer-group diff --git a/config/log4j.properties b/config/log4j.properties index 1ab850772a965..c51ab8b6b20d2 100644 --- a/config/log4j.properties +++ b/config/log4j.properties @@ -13,8 +13,6 @@ # 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 @@ -41,7 +39,7 @@ 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.File=${kafka.logs.dir}/log-cleaner.log log4j.appender.cleanerAppender.layout=org.apache.log4j.PatternLayout log4j.appender.cleanerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n @@ -73,8 +71,6 @@ 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/producer.properties b/config/producer.properties index 52a76114f5d09..47ae3e241301e 100644 --- a/config/producer.properties +++ b/config/producer.properties @@ -26,8 +26,8 @@ metadata.broker.list=localhost:9092 # specifies whether the messages are sent asynchronously (async) or synchronously (sync) producer.type=sync -# specify the compression codec for all data generated: none , gzip, snappy. -# the old config values work as well: 0, 1, 2 for none, gzip, snappy, respectivally +# specify the compression codec for all data generated: none, gzip, snappy, lz4. +# the old config values work as well: 0, 1, 2, 3 for none, gzip, snappy, lz4, respectively compression.codec=none # message encoder diff --git a/config/server.properties b/config/server.properties index 2ffe0ebccf109..80ee2fc6e94a1 100644 --- a/config/server.properties +++ b/config/server.properties @@ -21,8 +21,10 @@ broker.id=0 ############################# Socket Server Settings ############################# +listeners=PLAINTEXT://:9092 + # The port the socket server listens on -port=9092 +#port=9092 # Hostname the broker will bind to. If not set, the server will bind to all interfaces #host.name=localhost @@ -37,16 +39,16 @@ port=9092 #advertised.port= # The number of threads handling network requests -num.network.threads=2 +num.network.threads=3 # The number of threads doing disk I/O -num.io.threads=2 +num.io.threads=8 # The send buffer (SO_SNDBUF) used by the socket server -socket.send.buffer.bytes=1048576 +socket.send.buffer.bytes=102400 # The receive buffer (SO_RCVBUF) used by the socket server -socket.receive.buffer.bytes=1048576 +socket.receive.buffer.bytes=102400 # The maximum size of a request that the socket server will accept (protection against OOM) socket.request.max.bytes=104857600 @@ -60,7 +62,11 @@ log.dirs=/tmp/kafka-logs # 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 +num.partitions=1 + +# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. +# This value is recommended to be increased for installations with data dirs located in RAID array. +num.recovery.threads.per.data.dir=1 ############################# Log Flush Policy ############################# @@ -94,11 +100,15 @@ log.retention.hours=168 #log.retention.bytes=1073741824 # The maximum size of a log segment file. When this size is reached a new log segment will be created. -log.segment.bytes=536870912 +log.segment.bytes=1073741824 # The interval at which log segments are checked to see if they can be deleted according # to the retention policies -log.retention.check.interval.ms=60000 +log.retention.check.interval.ms=300000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false ############################# Zookeeper ############################# @@ -110,7 +120,4 @@ log.retention.check.interval.ms=60000 zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper -zookeeper.connection.timeout.ms=1000000 - - -log.cleanup.policy=delete +zookeeper.connection.timeout.ms=6000 diff --git a/config/test-log4j.properties b/config/test-log4j.properties index a3ae33f20e4b7..e0bbc134233c9 100644 --- a/config/test-log4j.properties +++ b/config/test-log4j.properties @@ -12,7 +12,7 @@ # 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.rootLogger=INFO, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout @@ -45,8 +45,8 @@ 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.kafka.tools=DEBUG, kafkaAppender +log4j.logger.kafka.tools.ProducerPerformance$ProducerThread=DEBUG, kafkaAppender #log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG log4j.logger.kafka=INFO, kafkaAppender @@ -64,5 +64,3 @@ log4j.additivity.kafka.controller=false log4j.logger.state.change.logger=TRACE, stateChangeAppender log4j.additivity.state.change.logger=false - - diff --git a/config/tools-log4j.properties b/config/tools-log4j.properties index 7924049014983..52f07c96019b4 100644 --- a/config/tools-log4j.properties +++ b/config/tools-log4j.properties @@ -18,5 +18,3 @@ log4j.rootLogger=WARN, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n - - diff --git a/contrib/LICENSE b/contrib/LICENSE new file mode 120000 index 0000000000000..ea5b60640b01f --- /dev/null +++ b/contrib/LICENSE @@ -0,0 +1 @@ +../LICENSE \ No newline at end of file diff --git a/contrib/NOTICE b/contrib/NOTICE new file mode 120000 index 0000000000000..7e1b82f6e6a12 --- /dev/null +++ b/contrib/NOTICE @@ -0,0 +1 @@ +../NOTICE \ No newline at end of file diff --git a/contrib/hadoop-consumer/LICENSE b/contrib/hadoop-consumer/LICENSE deleted file mode 100644 index 6b0b1270ff0ca..0000000000000 --- a/contrib/hadoop-consumer/LICENSE +++ /dev/null @@ -1,203 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - diff --git a/contrib/hadoop-consumer/build.sbt b/contrib/hadoop-consumer/build.sbt deleted file mode 100644 index 02e95eb8ca2c7..0000000000000 --- a/contrib/hadoop-consumer/build.sbt +++ /dev/null @@ -1 +0,0 @@ -crossPaths := false diff --git a/contrib/hadoop-consumer/lib/piggybank.jar b/contrib/hadoop-consumer/lib/piggybank.jar deleted file mode 100644 index cbd46e065aab0..0000000000000 Binary files a/contrib/hadoop-consumer/lib/piggybank.jar and /dev/null differ diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java index 1d0e0a9179857..c9b90189884bd 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java @@ -159,7 +159,7 @@ public boolean fetchMore () throws IOException { _response = _consumer.fetch(fetchRequest); if(_response != null) { _respIterator = new ArrayList(){{ - add((ByteBufferMessageSet) _response.messageSet(_request.getTopic(), _request.getPartition())); + add(_response.messageSet(_request.getTopic(), _request.getPartition())); }}.iterator(); } _requestTime += (System.currentTimeMillis() - tempTime); diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java index 3514ec77bfd1f..71eb80f313fa2 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java @@ -115,11 +115,11 @@ public Props(Properties... properties) { } /** - * build props from a list of strings and interprate them as + * build props from a list of strings and interpret them as * key, value, key, value,.... * * @param args - * @return + * @return props */ @SuppressWarnings("unchecked") public static Props of(String... args) { @@ -148,7 +148,7 @@ public void put(Properties properties) { /** * get property of "key" and split the value by " ," * @param key - * @return + * @return list of values */ public List getStringList(String key) { return getStringList(key, "\\s*,\\s*"); @@ -158,7 +158,7 @@ public List getStringList(String key) { * get property of "key" and split the value by "sep" * @param key * @param sep - * @return + * @return string list of values */ public List getStringList(String key, String sep) { String val = super.getProperty(key); @@ -176,7 +176,7 @@ public List getStringList(String key, String sep) { * get string list with default value. default delimiter is "," * @param key * @param defaultValue - * @return + * @return string list of values */ public List getStringList(String key, List defaultValue) { if (containsKey(key)) @@ -189,7 +189,7 @@ public List getStringList(String key, List defaultValue) { * get string list with default value * @param key * @param defaultValue - * @return + * @return string list of values */ public List getStringList(String key, List defaultValue, String sep) { @@ -251,10 +251,10 @@ else throw new UndefinedPropertyException ("Property " + key + } /** - * get boolean value + * get boolean value with default value * @param key * @param defaultValue - * @return + * @return boolean value * @throws Exception if value is not of type boolean or string */ public Boolean getBoolean(String key, Boolean defaultValue) @@ -265,8 +265,7 @@ public Boolean getBoolean(String key, Boolean defaultValue) /** * get boolean value * @param key - * @param defaultValue - * @return + * @return boolean value * @throws Exception if value is not of type boolean or string or * if value doesn't exist */ @@ -275,10 +274,10 @@ public Boolean getBoolean(String key) throws Exception { } /** - * get long value - * @param key + * get long value with default value + * @param name * @param defaultValue - * @return + * @return long value * @throws Exception if value is not of type long or string */ public Long getLong(String name, Long defaultValue) @@ -288,9 +287,8 @@ public Long getLong(String name, Long defaultValue) /** * get long value - * @param key - * @param defaultValue - * @return + * @param name + * @return long value * @throws Exception if value is not of type long or string or * if value doesn't exist */ @@ -299,10 +297,10 @@ public Long getLong(String name) throws Exception { } /** - * get integer value - * @param key + * get integer value with default value + * @param name * @param defaultValue - * @return + * @return integer value * @throws Exception if value is not of type integer or string */ public Integer getInt(String name, Integer defaultValue) @@ -312,9 +310,8 @@ public Integer getInt(String name, Integer defaultValue) /** * get integer value - * @param key - * @param defaultValue - * @return + * @param name + * @return integer value * @throws Exception if value is not of type integer or string or * if value doesn't exist */ @@ -323,10 +320,10 @@ public Integer getInt(String name) throws Exception { } /** - * get double value - * @param key + * get double value with default value + * @param name * @param defaultValue - * @return + * @return double value * @throws Exception if value is not of type double or string */ public Double getDouble(String name, double defaultValue) @@ -336,9 +333,8 @@ public Double getDouble(String name, double defaultValue) /** * get double value - * @param key - * @param defaultValue - * @return + * @param name + * @return double value * @throws Exception if value is not of type double or string or * if value doesn't exist */ @@ -347,10 +343,10 @@ public double getDouble(String name) throws Exception { } /** - * get URI value - * @param key + * get URI value with default value + * @param name * @param defaultValue - * @return + * @return URI value * @throws Exception if value is not of type URI or string */ public URI getUri(String name, URI defaultValue) throws Exception { @@ -359,9 +355,9 @@ public URI getUri(String name, URI defaultValue) throws Exception { /** * get URI value - * @param key + * @param name * @param defaultValue - * @return + * @return URI value * @throws Exception if value is not of type URI or string */ public URI getUri(String name, String defaultValue) @@ -372,9 +368,8 @@ public URI getUri(String name, String defaultValue) /** * get URI value - * @param key - * @param defaultValue - * @return + * @param name + * @return URI value * @throws Exception if value is not of type URI or string or * if value doesn't exist */ @@ -385,7 +380,7 @@ public URI getUri(String name) throws Exception { /** * compare two props * @param p - * @return + * @return true or false */ public boolean equalsProps(Props p) { if (p == null) { @@ -432,7 +427,7 @@ public void store(OutputStream out) throws IOException { /** * get all property names - * @return + * @return set of property names */ public Set getKeySet() { return super.stringPropertyNames(); @@ -453,7 +448,7 @@ public void logProperties(String comment) { /** * clone a Props * @param p - * @return + * @return props */ public static Props clone(Props p) { return new Props(p); 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 f3fb3fd99869e..d27a511fcdd73 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 @@ -27,7 +27,6 @@ import kafka.etl.KafkaETLRequest; import kafka.etl.Props; import kafka.javaapi.producer.Producer; -import kafka.message.Message; import kafka.producer.ProducerConfig; import kafka.producer.KeyedMessage; import org.apache.hadoop.fs.FileSystem; @@ -36,6 +35,8 @@ import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapred.JobConf; +import static org.apache.kafka.common.utils.Utils.formatAddress; + /** * Use this class to produce test events to Kafka server. Each event contains a * random timestamp in text format. @@ -70,7 +71,7 @@ public DataGenerator(String id, Props props) throws Exception { System.out.println("server uri:" + _uri.toString()); Properties producerProps = new Properties(); - producerProps.put("metadata.broker.list", String.format("%s:%d", _uri.getHost(), _uri.getPort())); + producerProps.put("metadata.broker.list", formatAddress(_uri.getHost(), _uri.getPort())); producerProps.put("send.buffer.bytes", String.valueOf(TCP_BUFFER_SIZE)); producerProps.put("connect.timeout.ms", String.valueOf(CONNECT_TIMEOUT)); producerProps.put("reconnect.interval", String.valueOf(RECONNECT_INTERVAL)); @@ -108,7 +109,7 @@ protected void generateOffsets() throws Exception { if (fs.exists(outPath)) fs.delete(outPath); KafkaETLRequest request = - new KafkaETLRequest(_topic, "tcp://" + _uri.getHost() + ":" + _uri.getPort(), 0); + new KafkaETLRequest(_topic, "tcp://" + formatAddress(_uri.getHost(), _uri.getPort()), 0); System.out.println("Dump " + request.toString() + " to " + outPath.toUri().toString()); byte[] bytes = request.toString().getBytes("UTF-8"); diff --git a/contrib/hadoop-producer/LICENSE b/contrib/hadoop-producer/LICENSE deleted file mode 100644 index 6b0b1270ff0ca..0000000000000 --- a/contrib/hadoop-producer/LICENSE +++ /dev/null @@ -1,203 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - diff --git a/contrib/hadoop-producer/build.sbt b/contrib/hadoop-producer/build.sbt deleted file mode 100644 index 02e95eb8ca2c7..0000000000000 --- a/contrib/hadoop-producer/build.sbt +++ /dev/null @@ -1 +0,0 @@ -crossPaths := false diff --git a/contrib/hadoop-producer/lib/piggybank.jar b/contrib/hadoop-producer/lib/piggybank.jar deleted file mode 100644 index cbd46e065aab0..0000000000000 Binary files a/contrib/hadoop-producer/lib/piggybank.jar and /dev/null differ diff --git a/core/build.sbt b/core/build.sbt deleted file mode 100644 index 3eff64ef3f15a..0000000000000 --- a/core/build.sbt +++ /dev/null @@ -1,32 +0,0 @@ -import sbt._ -import Keys._ -import AssemblyKeys._ - -name := "kafka" - -resolvers ++= Seq( - "SonaType ScalaTest repo" at "https://oss.sonatype.org/content/groups/public/org/scalatest/" -) - -libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _ ) - -libraryDependencies ++= Seq( - "org.apache.zookeeper" % "zookeeper" % "3.3.4", - "com.101tec" % "zkclient" % "0.3", - "org.xerial.snappy" % "snappy-java" % "1.0.5", - "com.yammer.metrics" % "metrics-core" % "2.2.0", - "com.yammer.metrics" % "metrics-annotation" % "2.2.0", - "org.easymock" % "easymock" % "3.0" % "test", - "junit" % "junit" % "4.1" % "test" -) - -libraryDependencies <<= (scalaVersion, libraryDependencies) { (sv, deps) => - deps :+ (sv match { - case "2.8.0" => "org.scalatest" % "scalatest" % "1.2" % "test" - case v if v.startsWith("2.10") => "org.scalatest" %% "scalatest" % "1.9.1" % "test" - case _ => "org.scalatest" %% "scalatest" % "1.8" % "test" - }) -} - -assemblySettings - diff --git a/core/src/main/scala/kafka/Kafka.scala b/core/src/main/scala/kafka/Kafka.scala old mode 100644 new mode 100755 index 988014a51c332..6af7b8068073c --- a/core/src/main/scala/kafka/Kafka.scala +++ b/core/src/main/scala/kafka/Kafka.scala @@ -17,37 +17,62 @@ package kafka +import java.util.Properties +import scala.collection.JavaConversions._ +import joptsimple.OptionParser import metrics.KafkaMetricsReporter import server.{KafkaConfig, KafkaServerStartable, KafkaServer} -import utils.{Utils, Logging} +import kafka.utils.{VerifiableProperties, CommandLineUtils, Logging} +import org.apache.kafka.common.utils.Utils object Kafka extends Logging { - def main(args: Array[String]): Unit = { - if (args.length != 1) { - println("USAGE: java [options] %s server.properties".format(classOf[KafkaServer].getSimpleName())) - System.exit(1) + def getPropsFromArgs(args: Array[String]): Properties = { + val optionParser = new OptionParser + val overrideOpt = optionParser.accepts("override", "Optional property that should override values set in server.properties file") + .withRequiredArg() + .ofType(classOf[String]) + + if (args.length == 0) { + CommandLineUtils.printUsageAndDie(optionParser, "USAGE: java [options] %s server.properties [--override property=value]*".format(classOf[KafkaServer].getSimpleName())) + } + + val props = Utils.loadProps(args(0)) + + if(args.length > 1) { + val options = optionParser.parse(args.slice(1, args.length): _*) + + if(options.nonOptionArguments().size() > 0) { + CommandLineUtils.printUsageAndDie(optionParser, "Found non argument parameters: " + options.nonOptionArguments().toArray.mkString(",")) + } + + props.putAll(CommandLineUtils.parseKeyValueArgs(options.valuesOf(overrideOpt))) } - + props + } + + def main(args: Array[String]): Unit = { try { - val props = Utils.loadProps(args(0)) - val serverConfig = new KafkaConfig(props) - KafkaMetricsReporter.startReporters(serverConfig.props) - val kafkaServerStartble = new KafkaServerStartable(serverConfig) + val serverProps = getPropsFromArgs(args) + val serverConfig = KafkaConfig.fromProps(serverProps) + KafkaMetricsReporter.startReporters(new VerifiableProperties(serverProps)) + val kafkaServerStartable = new KafkaServerStartable(serverConfig) // attach shutdown handler to catch control-c Runtime.getRuntime().addShutdownHook(new Thread() { override def run() = { - kafkaServerStartble.shutdown + kafkaServerStartable.shutdown } }) - kafkaServerStartble.startup - kafkaServerStartble.awaitShutdown + kafkaServerStartable.startup + kafkaServerStartable.awaitShutdown } catch { - case e: Throwable => fatal(e) + case e: Throwable => + fatal(e) + System.exit(1) } System.exit(0) } diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 36ddeb44490e8..2b4e028f8a60f 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -17,25 +17,32 @@ package kafka.admin +import kafka.common._ +import kafka.cluster.{BrokerEndPoint, Broker} + +import kafka.log.LogConfig +import kafka.utils._ +import kafka.api.{TopicMetadata, PartitionMetadata} + import java.util.Random import java.util.Properties -import kafka.api.{TopicMetadata, PartitionMetadata} -import kafka.cluster.Broker -import kafka.log.LogConfig -import kafka.utils.{Logging, ZkUtils, Json} -import org.I0Itec.zkclient.ZkClient -import org.I0Itec.zkclient.exception.ZkNodeExistsException +import org.apache.kafka.common.protocol.SecurityProtocol + +import scala.Predef._ 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 +import org.I0Itec.zkclient.ZkClient +import org.I0Itec.zkclient.exception.ZkNodeExistsException + object AdminUtils extends Logging { val rand = new Random + + val AdminClientId = "__admin_client" + val TopicConfigChangeZnodePrefix = "config_change_" /** @@ -87,7 +94,23 @@ object AdminUtils extends Logging { ret.toMap } - def addPartitions(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "") { + + /** + * Add partitions to existing topic with optional replica assignment + * + * @param zkClient Zookeeper client + * @param topic Topic for adding partitions to + * @param numPartitions Number of partitions to be set + * @param replicaAssignmentStr Manual replica assignment + * @param checkBrokerAvailable Ignore checking if assigned replica broker is available. Only used for testing + * @param config Pre-existing properties that should be preserved + */ + def addPartitions(zkClient: ZkClient, + topic: String, + numPartitions: Int = 1, + replicaAssignmentStr: String = "", + checkBrokerAvailable: Boolean = true, + config: Properties = new Properties) { val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) if (existingPartitionsReplicaList.size == 0) throw new AdminOperationException("The topic %s does not exist".format(topic)) @@ -102,7 +125,7 @@ object AdminUtils extends Logging { val newPartitionReplicaList = if (replicaAssignmentStr == null || replicaAssignmentStr == "") AdminUtils.assignReplicasToBrokers(brokerList, partitionsToAdd, existingReplicaList.size, existingReplicaList.head, existingPartitionsReplicaList.size) else - getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet, existingPartitionsReplicaList.size) + getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet, existingPartitionsReplicaList.size, checkBrokerAvailable) // check if manual assignment has the right replication factor val unmatchedRepFactorList = newPartitionReplicaList.values.filter(p => (p.size != existingReplicaList.size)) @@ -114,10 +137,10 @@ object AdminUtils extends Logging { val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2) // add the new list partitionReplicaList ++= newPartitionReplicaList - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true) } - def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int): Map[Int, List[Int]] = { + def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = { var partitionList = replicaAssignmentList.split(",") val ret = new mutable.HashMap[Int, List[Int]]() var partitionId = startPartitionId @@ -128,7 +151,7 @@ object AdminUtils extends Logging { 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)) + if (checkBrokerAvailable && !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) @@ -140,9 +163,69 @@ object AdminUtils extends Logging { } def deleteTopic(zkClient: ZkClient, topic: String) { - ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic)) + try { + ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic)) + } catch { + case e1: ZkNodeExistsException => throw new TopicAlreadyMarkedForDeletionException( + "topic %s is already marked for deletion".format(topic)) + case e2: Throwable => throw new AdminOperationException(e2.toString) + } } + def isConsumerGroupActive(zkClient: ZkClient, group: String) = { + ZkUtils.getConsumersInGroup(zkClient, group).nonEmpty + } + + /** + * Delete the whole directory of the given consumer group if the group is inactive. + * + * @param zkClient Zookeeper client + * @param group Consumer group + * @return whether or not we deleted the consumer group information + */ + def deleteConsumerGroupInZK(zkClient: ZkClient, group: String) = { + if (!isConsumerGroupActive(zkClient, group)) { + val dir = new ZKGroupDirs(group) + ZkUtils.deletePathRecursive(zkClient, dir.consumerGroupDir) + true + } + else false + } + + /** + * Delete the given consumer group's information for the given topic in Zookeeper if the group is inactive. + * If the consumer group consumes no other topics, delete the whole consumer group directory. + * + * @param zkClient Zookeeper client + * @param group Consumer group + * @param topic Topic of the consumer group information we wish to delete + * @return whether or not we deleted the consumer group information for the given topic + */ + def deleteConsumerGroupInfoForTopicInZK(zkClient: ZkClient, group: String, topic: String) = { + val topics = ZkUtils.getTopicsByConsumerGroup(zkClient, group) + if (topics == Seq(topic)) { + deleteConsumerGroupInZK(zkClient, group) + } + else if (!isConsumerGroupActive(zkClient, group)) { + val dir = new ZKGroupTopicDirs(group, topic) + ZkUtils.deletePathRecursive(zkClient, dir.consumerOwnerDir) + ZkUtils.deletePathRecursive(zkClient, dir.consumerOffsetDir) + true + } + else false + } + + /** + * Delete every inactive consumer group's information about the given topic in Zookeeper. + * + * @param zkClient Zookeeper client + * @param topic Topic of the consumer group information we wish to delete + */ + def deleteAllConsumerGroupInfoForTopicInZK(zkClient: ZkClient, topic: String) { + val groups = ZkUtils.getAllConsumerGroupsForTopic(zkClient, topic) + groups.foreach(group => deleteConsumerGroupInfoForTopicInZK(zkClient, group, topic)) + } + def topicExists(zkClient: ZkClient, topic: String): Boolean = zkClient.exists(ZkUtils.getTopicPath(topic)) @@ -240,12 +323,17 @@ object AdminUtils extends Logging { if(str != null) { Json.parseFull(str) match { case None => // there are no config overrides - case Some(map: Map[String, _]) => + case Some(mapAnon: Map[_, _]) => + val map = mapAnon collect { case (k: String, v: Any) => k -> v } require(map("version") == 1) map.get("config") match { - case Some(config: Map[String, String]) => - for((k,v) <- config) - props.setProperty(k, v) + case Some(config: Map[_, _]) => + for(configTup <- config) + configTup match { + case (k: String, v: String) => + props.setProperty(k, v) + case _ => throw new IllegalArgumentException("Invalid topic config: " + str) + } case _ => throw new IllegalArgumentException("Invalid topic config: " + str) } @@ -266,7 +354,9 @@ object AdminUtils extends Logging { topics.map(topic => fetchTopicMetadataFromZk(topic, zkClient, cachedBrokerInfo)) } - private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker]): TopicMetadata = { + + + private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): 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) @@ -277,22 +367,22 @@ object AdminUtils extends Logging { 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 + var leaderInfo: Option[BrokerEndPoint] = None + var replicaInfo: Seq[BrokerEndPoint] = Nil + var isrInfo: Seq[BrokerEndPoint] = Nil try { leaderInfo = leader match { case Some(l) => try { - Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head) + Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head.getBrokerEndPoint(protocol)) } 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) + replicaInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, replicas).map(_.getBrokerEndPoint(protocol)) + isrInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, inSyncReplicas).map(_.getBrokerEndPoint(protocol)) } catch { case e: Throwable => throw new ReplicaNotAvailableException(e) } diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala new file mode 100755 index 0000000000000..f23120ede5f9b --- /dev/null +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -0,0 +1,311 @@ +/** + * 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 kafka.utils._ +import org.I0Itec.zkclient.ZkClient +import kafka.common._ +import java.util.Properties +import kafka.client.ClientUtils +import kafka.api.{OffsetRequest, PartitionOffsetRequestInfo, OffsetFetchResponse, OffsetFetchRequest} +import org.I0Itec.zkclient.exception.ZkNoNodeException +import kafka.common.TopicAndPartition +import joptsimple.{OptionSpec, OptionParser} +import scala.collection.{Set, mutable} +import kafka.consumer.SimpleConsumer +import collection.JavaConversions._ +import org.apache.kafka.common.utils.Utils + + +object ConsumerGroupCommand { + + def main(args: Array[String]) { + val opts = new ConsumerGroupCommandOptions(args) + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(opts.parser, "List all consumer groups, describe a consumer group, or delete consumer group info.") + + // should have exactly one action + val actions = Seq(opts.listOpt, opts.describeOpt, opts.deleteOpt).count(opts.options.has _) + if(actions != 1) + CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: --list, --describe, --delete") + + opts.checkArgs() + + val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000) + + try { + if (opts.options.has(opts.listOpt)) + list(zkClient) + else if (opts.options.has(opts.describeOpt)) + describe(zkClient, opts) + else if (opts.options.has(opts.deleteOpt)) + delete(zkClient, opts) + } catch { + case e: Throwable => + println("Error while executing consumer group command " + e.getMessage) + println(Utils.stackTrace(e)) + } finally { + zkClient.close() + } + } + + def list(zkClient: ZkClient) { + ZkUtils.getConsumerGroups(zkClient).foreach(println) + } + + def describe(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + val configs = parseConfigs(opts) + val channelSocketTimeoutMs = configs.getProperty("channelSocketTimeoutMs", "600").toInt + val channelRetryBackoffMs = configs.getProperty("channelRetryBackoffMsOpt", "300").toInt + val group = opts.options.valueOf(opts.groupOpt) + val topics = ZkUtils.getTopicsByConsumerGroup(zkClient, group) + if (topics.isEmpty) { + println("No topic available for consumer group provided") + } + topics.foreach(topic => describeTopic(zkClient, group, topic, channelSocketTimeoutMs, channelRetryBackoffMs)) + } + + def delete(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + if (opts.options.has(opts.groupOpt) && opts.options.has(opts.topicOpt)) { + deleteForTopic(zkClient, opts) + } + else if (opts.options.has(opts.groupOpt)) { + deleteForGroup(zkClient, opts) + } + else if (opts.options.has(opts.topicOpt)) { + deleteAllForTopic(zkClient, opts) + } + } + + private def deleteForGroup(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + val groups = opts.options.valuesOf(opts.groupOpt) + groups.foreach { group => + try { + if (AdminUtils.deleteConsumerGroupInZK(zkClient, group)) + println("Deleted all consumer group information for group %s in zookeeper.".format(group)) + else + println("Delete for group %s failed because its consumers are still active.".format(group)) + } + catch { + case e: ZkNoNodeException => + println("Delete for group %s failed because group does not exist.".format(group)) + } + } + } + + private def deleteForTopic(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + val groups = opts.options.valuesOf(opts.groupOpt) + val topic = opts.options.valueOf(opts.topicOpt) + Topic.validate(topic) + groups.foreach { group => + try { + if (AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, group, topic)) + println("Deleted consumer group information for group %s topic %s in zookeeper.".format(group, topic)) + else + println("Delete for group %s topic %s failed because its consumers are still active.".format(group, topic)) + } + catch { + case e: ZkNoNodeException => + println("Delete for group %s topic %s failed because group does not exist.".format(group, topic)) + } + } + } + + private def deleteAllForTopic(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + val topic = opts.options.valueOf(opts.topicOpt) + Topic.validate(topic) + AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkClient, topic) + println("Deleted consumer group information for all inactive consumer groups for topic %s in zookeeper.".format(topic)) + } + + private def parseConfigs(opts: ConsumerGroupCommandOptions): Properties = { + val configsToBeAdded = opts.options.valuesOf(opts.configOpt).map(_.split("""\s*=\s*""")) + require(configsToBeAdded.forall(config => config.length == 2), + "Invalid 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)) + props + } + + private def describeTopic(zkClient: ZkClient, + group: String, + topic: String, + channelSocketTimeoutMs: Int, + channelRetryBackoffMs: Int) { + val topicPartitions = getTopicPartitions(zkClient, topic) + val partitionOffsets = getPartitionOffsets(zkClient, group, topicPartitions, channelSocketTimeoutMs, channelRetryBackoffMs) + println("%s, %s, %s, %s, %s, %s, %s" + .format("GROUP", "TOPIC", "PARTITION", "CURRENT OFFSET", "LOG END OFFSET", "LAG", "OWNER")) + topicPartitions + .sortBy { case topicPartition => topicPartition.partition } + .foreach { topicPartition => + describePartition(zkClient, group, topicPartition.topic, topicPartition.partition, partitionOffsets.get(topicPartition)) + } + } + + private def getTopicPartitions(zkClient: ZkClient, topic: String) = { + val topicPartitionMap = ZkUtils.getPartitionsForTopics(zkClient, Seq(topic)) + val partitions = topicPartitionMap.getOrElse(topic, Seq.empty) + partitions.map(TopicAndPartition(topic, _)) + } + + private def getPartitionOffsets(zkClient: ZkClient, + group: String, + topicPartitions: Seq[TopicAndPartition], + channelSocketTimeoutMs: Int, + channelRetryBackoffMs: Int): Map[TopicAndPartition, Long] = { + val offsetMap = mutable.Map[TopicAndPartition, Long]() + val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs) + channel.send(OffsetFetchRequest(group, topicPartitions)) + val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload()) + + offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => + if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) { + val topicDirs = new ZKGroupTopicDirs(group, topicAndPartition.topic) + // this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool + // (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage) + try { + val offset = ZkUtils.readData(zkClient, topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1.toLong + offsetMap.put(topicAndPartition, offset) + } catch { + case z: ZkNoNodeException => + println("Could not fetch offset from zookeeper for group %s partition %s due to missing offset data in zookeeper." + .format(group, topicAndPartition)) + } + } + else if (offsetAndMetadata.error == ErrorMapping.NoError) + offsetMap.put(topicAndPartition, offsetAndMetadata.offset) + else + println("Could not fetch offset from kafka for group %s partition %s due to %s." + .format(group, topicAndPartition, ErrorMapping.exceptionFor(offsetAndMetadata.error))) + } + channel.disconnect() + offsetMap.toMap + } + + private def describePartition(zkClient: ZkClient, + group: String, + topic: String, + partition: Int, + offsetOpt: Option[Long]) { + val topicAndPartition = TopicAndPartition(topic, partition) + val groupDirs = new ZKGroupTopicDirs(group, topic) + val owner = ZkUtils.readDataMaybeNull(zkClient, groupDirs.consumerOwnerDir + "/" + partition)._1 + ZkUtils.getLeaderForPartition(zkClient, topic, partition) match { + case Some(-1) => + println("%s, %s, %s, %s, %s, %s, %s" + .format(group, topic, partition, offsetOpt.getOrElse("unknown"), "unknown", "unknown", owner.getOrElse("none"))) + case Some(brokerId) => + val consumerOpt = getConsumer(zkClient, brokerId) + consumerOpt match { + case Some(consumer) => + val request = + OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1))) + val logEndOffset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head + consumer.close() + + val lag = offsetOpt.filter(_ != -1).map(logEndOffset - _) + println("%s, %s, %s, %s, %s, %s, %s" + .format(group, topic, partition, offsetOpt.getOrElse("unknown"), logEndOffset, lag.getOrElse("unknown"), owner.getOrElse("none"))) + case None => // ignore + } + case None => + println("No broker for partition %s".format(topicAndPartition)) + } + } + + private def getConsumer(zkClient: ZkClient, brokerId: Int): Option[SimpleConsumer] = { + try { + ZkUtils.readDataMaybeNull(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 match { + case Some(brokerInfoString) => + Json.parseFull(brokerInfoString) match { + case Some(m) => + val brokerInfo = m.asInstanceOf[Map[String, Any]] + val host = brokerInfo.get("host").get.asInstanceOf[String] + val port = brokerInfo.get("port").get.asInstanceOf[Int] + Some(new SimpleConsumer(host, port, 10000, 100000, "ConsumerGroupCommand")) + case None => + throw new BrokerNotAvailableException("Broker id %d does not exist".format(brokerId)) + } + case None => + throw new BrokerNotAvailableException("Broker id %d does not exist".format(brokerId)) + } + } catch { + case t: Throwable => + println("Could not parse broker info due to " + t.getMessage) + None + } + } + + class ConsumerGroupCommandOptions(args: Array[String]) { + val ZkConnectDoc = "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + + "Multiple URLS can be given to allow fail-over." + val GroupDoc = "The consumer group we wish to act on." + val TopicDoc = "The topic whose consumer group information should be deleted." + val ConfigDoc = "Configuration for timeouts. For instance --config channelSocketTimeoutMs=600" + val ListDoc = "List all consumer groups." + val DescribeDoc = "Describe consumer group and list offset lag related to given group." + val nl = System.getProperty("line.separator") + val DeleteDoc = "Pass in groups to delete topic partition offsets and ownership information " + + "over the entire consumer group. For instance --group g1 --group g2" + nl + + "Pass in groups with a single topic to just delete the given topic's partition offsets and ownership " + + "information for the given consumer groups. For instance --group g1 --group g2 --topic t1" + nl + + "Pass in just a topic to delete the given topic's partition offsets and ownership information " + + "for every consumer group. For instance --topic t1" + nl + + "WARNING: Only does deletions on consumer groups that are not active." + val parser = new OptionParser + val zkConnectOpt = parser.accepts("zookeeper", ZkConnectDoc) + .withRequiredArg + .describedAs("urls") + .ofType(classOf[String]) + val groupOpt = parser.accepts("group", GroupDoc) + .withRequiredArg + .describedAs("consumer group") + .ofType(classOf[String]) + val topicOpt = parser.accepts("topic", TopicDoc) + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) + val configOpt = parser.accepts("config", ConfigDoc) + .withRequiredArg + .describedAs("name=value") + .ofType(classOf[String]) + val listOpt = parser.accepts("list", ListDoc) + val describeOpt = parser.accepts("describe", DescribeDoc) + val deleteOpt = parser.accepts("delete", DeleteDoc) + val options = parser.parse(args : _*) + + val allConsumerGroupLevelOpts: Set[OptionSpec[_]] = Set(listOpt, describeOpt, deleteOpt) + + def checkArgs() { + // check required args + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt) + if (options.has(describeOpt)) + CommandLineUtils.checkRequiredArgs(parser, options, groupOpt) + if (options.has(deleteOpt) && !options.has(groupOpt) && !options.has(topicOpt)) + CommandLineUtils.printUsageAndDie(parser, "Option %s either takes %s, %s, or both".format(deleteOpt, groupOpt, topicOpt)) + + // check invalid args + CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, allConsumerGroupLevelOpts - describeOpt - deleteOpt) + CommandLineUtils.checkInvalidArgs(parser, options, topicOpt, allConsumerGroupLevelOpts - deleteOpt) + CommandLineUtils.checkInvalidArgs(parser, options, configOpt, allConsumerGroupLevelOpts - describeOpt) + } + } +} diff --git a/core/src/main/scala/kafka/admin/DeleteTopicCommand.scala b/core/src/main/scala/kafka/admin/DeleteTopicCommand.scala deleted file mode 100644 index 804b331a7e441..0000000000000 --- a/core/src/main/scala/kafka/admin/DeleteTopicCommand.scala +++ /dev/null @@ -1,66 +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 DeleteTopicCommand { - - def main(args: Array[String]): Unit = { - val parser = new OptionParser - val topicOpt = parser.accepts("topic", "REQUIRED: The topic to be deleted.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + - "Multiple URLS can be given to allow fail-over.") - .withRequiredArg - .describedAs("urls") - .ofType(classOf[String]) - - val options = parser.parse(args : _*) - - for(arg <- List(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) - var zkClient: ZkClient = null - try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) - zkClient.deleteRecursive(ZkUtils.getTopicPath(topic)) - println("deletion succeeded!") - } - catch { - case e: Throwable => - println("delection failed because of " + e.getMessage) - println(Utils.stackTrace(e)) - } - finally { - if (zkClient != null) - zkClient.close() - } - } -} \ 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 old mode 100644 new mode 100755 index 9b3c6aeaf77db..2aa6e62134fbd --- a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala +++ b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala @@ -22,7 +22,7 @@ import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException import kafka.common.{TopicAndPartition, AdminCommandFailedException} import collection._ -import mutable.ListBuffer +import org.apache.kafka.common.utils.Utils object PreferredReplicaLeaderElectionCommand extends Logging { @@ -40,6 +40,10 @@ object PreferredReplicaLeaderElectionCommand extends Logging { .withRequiredArg .describedAs("urls") .ofType(classOf[String]) + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "This tool causes leadership for each partition to be transferred back to the 'preferred replica'," + + " it can be used to balance leadership among the servers.") val options = parser.parse(args : _*) @@ -49,7 +53,7 @@ object PreferredReplicaLeaderElectionCommand extends Logging { var zkClient: ZkClient = null try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) val partitionsForPreferredReplicaElection = if (!options.has(jsonFileOpt)) ZkUtils.getAllPartitions(zkClient) @@ -74,12 +78,17 @@ object PreferredReplicaLeaderElectionCommand extends Logging { case Some(m) => m.asInstanceOf[Map[String, Any]].get("partitions") match { case Some(partitionsList) => - val partitions = partitionsList.asInstanceOf[List[Map[String, Any]]] - partitions.map { p => + val partitionsRaw = partitionsList.asInstanceOf[List[Map[String, Any]]] + val partitions = partitionsRaw.map { p => val topic = p.get("topic").get.asInstanceOf[String] val partition = p.get("partition").get.asInstanceOf[Int] TopicAndPartition(topic, partition) - }.toSet + } + val duplicatePartitions = CoreUtils.duplicates(partitions) + val partitionsSet = partitions.toSet + if (duplicatePartitions.nonEmpty) + throw new AdminOperationException("Preferred replica election data contains duplicate partitions: %s".format(duplicatePartitions.mkString(","))) + partitionsSet case None => throw new AdminOperationException("Preferred replica election data is empty") } case None => throw new AdminOperationException("Preferred replica election data is empty") diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala old mode 100644 new mode 100755 index 2637586af99cf..ea345895a5297 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -22,6 +22,7 @@ import collection._ import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException import kafka.common.{TopicAndPartition, AdminCommandFailedException} +import org.apache.kafka.common.utils.Utils object ReassignPartitionsCommand extends Logging { @@ -31,15 +32,13 @@ object ReassignPartitionsCommand extends Logging { // 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(actions != 1) + CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: --generate, --execute or --verify") CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt) val zkConnect = opts.options.valueOf(opts.zkConnectOpt) - var zkClient: ZkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + var zkClient: ZkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) try { if(opts.options.has(opts.verifyOpt)) verifyAssignment(zkClient, opts) @@ -58,10 +57,8 @@ 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") - } + if(!opts.options.has(opts.reassignmentJsonFileOpt)) + CommandLineUtils.printUsageAndDie(opts.parser, "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) @@ -81,14 +78,18 @@ object ReassignPartitionsCommand extends Logging { } 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") - } + if(!(opts.options.has(opts.topicsToMoveJsonFileOpt) && opts.options.has(opts.brokerListOpt))) + CommandLineUtils.printUsageAndDie(opts.parser, "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 duplicateReassignments = CoreUtils.duplicates(brokerListToReassign) + if (duplicateReassignments.nonEmpty) + throw new AdminCommandFailedException("Broker list contains duplicate entries: %s".format(duplicateReassignments.mkString(","))) val topicsToMoveJsonString = Utils.readFileAsString(topicsToMoveJsonFile) val topicsToReassign = ZkUtils.parseTopicsData(topicsToMoveJsonString) + val duplicateTopicsToReassign = CoreUtils.duplicates(topicsToReassign) + if (duplicateTopicsToReassign.nonEmpty) + throw new AdminCommandFailedException("List of topics to reassign contains duplicate entries: %s".format(duplicateTopicsToReassign.mkString(","))) val topicPartitionsToReassign = ZkUtils.getReplicaAssignmentForTopics(zkClient, topicsToReassign) var partitionsToBeReassigned : Map[TopicAndPartition, Seq[Int]] = new mutable.HashMap[TopicAndPartition, List[Int]]() @@ -105,24 +106,33 @@ object ReassignPartitionsCommand extends Logging { } 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") - } + if(!opts.options.has(opts.reassignmentJsonFileOpt)) + CommandLineUtils.printUsageAndDie(opts.parser, "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) + val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentDataWithoutDedup(reassignmentJsonString) if (partitionsToBeReassigned.isEmpty) throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(reassignmentJsonFile)) - val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned) + val duplicateReassignedPartitions = CoreUtils.duplicates(partitionsToBeReassigned.map{ case(tp,replicas) => tp}) + if (duplicateReassignedPartitions.nonEmpty) + throw new AdminCommandFailedException("Partition reassignment contains duplicate topic partitions: %s".format(duplicateReassignedPartitions.mkString(","))) + val duplicateEntries= partitionsToBeReassigned + .map{ case(tp,replicas) => (tp, CoreUtils.duplicates(replicas))} + .filter{ case (tp,duplicatedReplicas) => duplicatedReplicas.nonEmpty } + if (duplicateEntries.nonEmpty) { + val duplicatesMsg = duplicateEntries + .map{ case (tp,duplicateReplicas) => "%s contains multiple entries for %s".format(tp, duplicateReplicas.mkString(",")) } + .mkString(". ") + throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: %s".format(duplicatesMsg)) + } + val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned.toMap) // before starting assignment, output the current replica assignment to facilitate rollback - val currentPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, partitionsToBeReassigned.map(_._1.topic).toSeq) + val currentPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, partitionsToBeReassigned.map(_._1.topic)) 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))) + println("Successfully started reassignment of partitions %s".format(ZkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned.toMap))) else println("Failed to reassign partitions %s".format(partitionsToBeReassigned)) } @@ -185,6 +195,9 @@ object ReassignPartitionsCommand extends Logging { .withRequiredArg .describedAs("brokerlist") .ofType(classOf[String]) + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "This command moves topic partitions between replicas.") val options = parser.parse(args : _*) } @@ -195,9 +208,14 @@ class ReassignPartitionsCommand(zkClient: ZkClient, partitions: collection.Map[T def reassignPartitions(): Boolean = { try { val validPartitions = partitions.filter(p => validatePartition(zkClient, p._1.topic, p._1.partition)) - val jsonReassignmentData = ZkUtils.getPartitionReassignmentZkData(validPartitions) - ZkUtils.createPersistentPath(zkClient, ZkUtils.ReassignPartitionsPath, jsonReassignmentData) - true + if(validPartitions.isEmpty) { + false + } + else { + val jsonReassignmentData = ZkUtils.getPartitionReassignmentZkData(validPartitions) + ZkUtils.createPersistentPath(zkClient, ZkUtils.ReassignPartitionsPath, jsonReassignmentData) + true + } } catch { case ze: ZkNodeExistsException => val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient) diff --git a/core/src/main/scala/kafka/admin/ShutdownBroker.scala b/core/src/main/scala/kafka/admin/ShutdownBroker.scala deleted file mode 100644 index 2dd47e7362f42..0000000000000 --- a/core/src/main/scala/kafka/admin/ShutdownBroker.scala +++ /dev/null @@ -1,124 +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 javax.management.remote.{JMXServiceURL, JMXConnectorFactory} -import javax.management.ObjectName -import kafka.controller.KafkaController -import scala.Some -import kafka.common.{TopicAndPartition, BrokerNotAvailableException} - - -object ShutdownBroker extends Logging { - - private case class ShutdownParams(zkConnect: String, brokerId: java.lang.Integer) - - private def invokeShutdown(params: ShutdownParams): Boolean = { - var zkClient: ZkClient = null - try { - zkClient = new ZkClient(params.zkConnect, 30000, 30000, ZKStringSerializer) - val controllerBrokerId = ZkUtils.getController(zkClient) - ZkUtils.readDataMaybeNull(zkClient, ZkUtils.BrokerIdsPath + "/" + controllerBrokerId)._1 match { - case Some(controllerInfo) => - var controllerHost: String = null - var controllerJmxPort: Int = -1 - try { - Json.parseFull(controllerInfo) match { - case Some(m) => - val brokerInfo = m.asInstanceOf[Map[String, Any]] - controllerHost = brokerInfo.get("host").get.toString - controllerJmxPort = brokerInfo.get("jmx_port").get.asInstanceOf[Int] - case None => - throw new BrokerNotAvailableException("Broker id %d does not exist".format(controllerBrokerId)) - } - } - val jmxUrl = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi".format(controllerHost, controllerJmxPort)) - info("Connecting to jmx url " + jmxUrl) - val jmxc = JMXConnectorFactory.connect(jmxUrl, null) - val mbsc = jmxc.getMBeanServerConnection - val leaderPartitionsRemaining = mbsc.invoke(new ObjectName(KafkaController.MBeanName), - "shutdownBroker", - Array(params.brokerId), - Array(classOf[Int].getName)).asInstanceOf[Set[TopicAndPartition]] - val shutdownComplete = (leaderPartitionsRemaining.size == 0) - info("Shutdown status: " + - (if (shutdownComplete) "complete" else "incomplete (broker still leads %d partitions)".format(leaderPartitionsRemaining))) - shutdownComplete - case None => - throw new BrokerNotAvailableException("Broker id %d does not exist".format(controllerBrokerId)) - } - } catch { - case t: Throwable => - error("Operation failed due to controller failure", t) - false - } finally { - if (zkClient != null) - zkClient.close() - } - } - - def main(args: Array[String]) { - val parser = new OptionParser - val brokerOpt = parser.accepts("broker", "REQUIRED: The broker to shutdown.") - .withRequiredArg - .describedAs("Broker Id") - .ofType(classOf[java.lang.Integer]) - 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 numRetriesOpt = parser.accepts("num.retries", "Number of attempts to retry if shutdown does not complete.") - .withRequiredArg - .describedAs("number of retries") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) - val retryIntervalOpt = parser.accepts("retry.interval.ms", "Retry interval if retries requested.") - .withRequiredArg - .describedAs("retry interval in ms (> 1000)") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1000) - - val options = parser.parse(args : _*) - CommandLineUtils.checkRequiredArgs(parser, options, brokerOpt, zkConnectOpt) - - val retryIntervalMs = options.valueOf(retryIntervalOpt).intValue.max(1000) - val numRetries = options.valueOf(numRetriesOpt).intValue - - val shutdownParams = ShutdownParams(options.valueOf(zkConnectOpt), options.valueOf(brokerOpt)) - - if (!invokeShutdown(shutdownParams)) { - (1 to numRetries).takeWhile(attempt => { - info("Retry " + attempt) - try { - Thread.sleep(retryIntervalMs) - } - catch { - case ie: InterruptedException => // ignore - } - !invokeShutdown(shutdownParams) - }) - } - } - -} - diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala old mode 100644 new mode 100755 index fc8d6861ccab5..a90aa8787ff21 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -19,50 +19,57 @@ package kafka.admin import joptsimple._ import java.util.Properties +import kafka.common.{Topic, AdminCommandFailedException} import kafka.utils._ import org.I0Itec.zkclient.ZkClient +import org.I0Itec.zkclient.exception.ZkNodeExistsException import scala.collection._ import scala.collection.JavaConversions._ -import kafka.cluster.Broker import kafka.log.LogConfig import kafka.consumer.Whitelist +import org.apache.kafka.common.utils.Utils +import kafka.coordinator.ConsumerCoordinator -object TopicCommand { + +object TopicCommand extends Logging { def main(args: Array[String]): Unit = { val opts = new TopicCommandOptions(args) + if(args.length == 0) + CommandLineUtils.printUsageAndDie(opts.parser, "Create, delete, describe, or change a topic.") + // 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) - } + val actions = Seq(opts.createOpt, opts.listOpt, opts.alterOpt, opts.describeOpt, opts.deleteOpt).count(opts.options.has _) + if(actions != 1) + CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: --list, --describe, --create, --alter or --delete") opts.checkArgs() - val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) - + val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000) + var exitCode = 0 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) + else if(opts.options.has(opts.deleteOpt)) + deleteTopic(zkClient, opts) } catch { - case e => - println("Error while executing topic command " + e.getMessage) - println(Utils.stackTrace(e)) + case e: Throwable => + println("Error while executing topic command : " + e.getMessage) + error(Utils.stackTrace(e)) + exitCode = 1 } finally { zkClient.close() + System.exit(exitCode) } + } private def getTopics(zkClient: ZkClient, opts: TopicCommandOptions): Seq[String] = { @@ -70,7 +77,7 @@ object TopicCommand { if (opts.options.has(opts.topicOpt)) { val topicsSpec = opts.options.valueOf(opts.topicOpt) val topicsFilter = new Whitelist(topicsSpec) - allTopics.filter(topicsFilter.isTopicAllowed) + allTopics.filter(topicsFilter.isTopicAllowed(_, excludeInternalTopics = false)) } else allTopics } @@ -92,42 +99,72 @@ object TopicCommand { def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topics = getTopics(zkClient, opts) + if (topics.length == 0) { + throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt), + opts.options.valueOf(opts.zkConnectOpt))) + } topics.foreach { topic => + val configs = AdminUtils.fetchTopicConfig(zkClient, 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.partitionsOpt)) { + if (topic == ConsumerCoordinator.OffsetsTopicName) { + throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.") + } 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) + AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs) println("Adding partitions succeeded!") } } } - def deleteTopic(zkClient: ZkClient, opts: TopicCommandOptions) { + def listTopics(zkClient: ZkClient, opts: TopicCommandOptions) { val topics = getTopics(zkClient, opts) - topics.foreach { topic => - AdminUtils.deleteTopic(zkClient, topic) - println("Topic \"%s\" queued for deletion.".format(topic)) + for(topic <- topics) { + if (ZkUtils.pathExists(zkClient,ZkUtils.getDeleteTopicPath(topic))) { + println("%s - marked for deletion".format(topic)) + } else { + println(topic) + } } } - - def listTopics(zkClient: ZkClient, opts: TopicCommandOptions) { + + def deleteTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topics = getTopics(zkClient, opts) - for(topic <- topics) - println(topic) + if (topics.length == 0) { + throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt), + opts.options.valueOf(opts.zkConnectOpt))) + } + topics.foreach { topic => + try { + if (Topic.InternalTopics.contains(topic)) { + throw new AdminOperationException("Topic %s is a kafka internal topic and is not allowed to be marked for deletion.".format(topic)) + } else { + ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic)) + println("Topic %s is marked for deletion.".format(topic)) + println("Note: This will have no impact if delete.topic.enable is not set to true.") + } + } catch { + case e: ZkNodeExistsException => + println("Topic %s is already marked for deletion.".format(topic)) + case e: AdminOperationException => + throw e + case e: Throwable => + throw new AdminOperationException("Error while deleting topic %s".format(topic)) + } + } } - + def describeTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topics = getTopics(zkClient, opts) val reportUnderReplicatedPartitions = if (opts.options.has(opts.reportUnderReplicatedPartitionsOpt)) true else false @@ -169,9 +206,7 @@ object TopicCommand { } } } - - 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), @@ -199,6 +234,9 @@ object TopicCommand { val ret = new mutable.HashMap[Int, List[Int]]() for (i <- 0 until partitionList.size) { val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) + val duplicateBrokers = CoreUtils.duplicates(brokerList) + if (duplicateBrokers.nonEmpty) + throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: %s".format(duplicateBrokers.mkString(","))) ret.put(i, brokerList.toList) if (ret(i).size != ret(0).size) throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList) @@ -215,20 +253,23 @@ object TopicCommand { .ofType(classOf[String]) val listOpt = parser.accepts("list", "List all available topics.") val createOpt = parser.accepts("create", "Create a new topic.") + val deleteOpt = parser.accepts("delete", "Delete a 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 " + + val topicOpt = parser.accepts("topic", "The topic to be create, alter 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.") + val nl = System.getProperty("line.separator") + val configOpt = parser.accepts("config", "A topic configuration override for the topic being created or altered." + + "The following is a list of valid configurations: " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl + + "See the Kafka documentation for full details on the topic configs.") .withRequiredArg .describedAs("name=value") .ofType(classOf[String]) - val deleteConfigOpt = parser.accepts("deleteConfig", "A topic configuration override to be removed for an existing topic") + val deleteConfigOpt = parser.accepts("delete-config", "A topic configuration override to be removed for an existing topic (see the list of configurations under the --config option).") .withRequiredArg .describedAs("name") .ofType(classOf[String]) @@ -255,7 +296,7 @@ object TopicCommand { val options = parser.parse(args : _*) - val allTopicLevelOpts: Set[OptionSpec[_]] = Set(alterOpt, createOpt, deleteOpt, describeOpt, listOpt) + val allTopicLevelOpts: Set[OptionSpec[_]] = Set(alterOpt, createOpt, describeOpt, listOpt) def checkArgs() { // check required args @@ -268,8 +309,9 @@ object TopicCommand { CommandLineUtils.checkInvalidArgs(parser, options, deleteConfigOpt, allTopicLevelOpts -- Set(alterOpt)) CommandLineUtils.checkInvalidArgs(parser, options, partitionsOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicationFactorOpt, allTopicLevelOpts -- Set(createOpt)) - CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, - allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) + CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, allTopicLevelOpts -- Set(createOpt,alterOpt)) + if(options.has(createOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, Set(partitionsOpt, replicationFactorOpt)) CommandLineUtils.checkInvalidArgs(parser, options, reportUnderReplicatedPartitionsOpt, allTopicLevelOpts -- Set(describeOpt) + reportUnavailablePartitionsOpt + topicsWithOverridesOpt) CommandLineUtils.checkInvalidArgs(parser, options, reportUnavailablePartitionsOpt, diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala new file mode 100644 index 0000000000000..a7c15f3781afa --- /dev/null +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -0,0 +1,78 @@ +/* + * 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 + +/** + * This class contains the different Kafka versions. + * Right now, we use them for upgrades - users can configure the version of the API brokers will use to communicate between themselves. + * This is only for inter-broker communications - when communicating with clients, the client decides on the API version. + * + * Note that the ID we initialize for each version is important. + * We consider a version newer than another, if it has a higher ID (to avoid depending on lexicographic order) + */ +object ApiVersion { + // This implicit is necessary due to: https://issues.scala-lang.org/browse/SI-8541 + implicit def orderingByVersion[A <: ApiVersion]: Ordering[A] = Ordering.by(_.id) + + private val versionNameMap = Map( + "0.8.0" -> KAFKA_080, + "0.8.1" -> KAFKA_081, + "0.8.2" -> KAFKA_082, + "0.8.3" -> KAFKA_083 + ) + + def apply(version: String): ApiVersion = versionNameMap(version.split("\\.").slice(0,3).mkString(".")) + + def latestVersion = versionNameMap.values.max +} + +sealed trait ApiVersion extends Ordered[ApiVersion] { + val version: String + val id: Int + + override def compare(that: ApiVersion): Int = { + ApiVersion.orderingByVersion.compare(this, that) + } + + def onOrAfter(that: ApiVersion): Boolean = { + this.compare(that) >= 0 + } + + override def toString(): String = version +} + +// Keep the IDs in order of versions +case object KAFKA_080 extends ApiVersion { + val version: String = "0.8.0.X" + val id: Int = 0 +} + +case object KAFKA_081 extends ApiVersion { + val version: String = "0.8.1.X" + val id: Int = 1 +} + +case object KAFKA_082 extends ApiVersion { + val version: String = "0.8.2.X" + val id: Int = 2 +} + +case object KAFKA_083 extends ApiVersion { + val version: String = "0.8.3.X" + val id: Int = 3 +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala new file mode 100644 index 0000000000000..258d5fe289fbb --- /dev/null +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -0,0 +1,80 @@ +/** + * 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.common.ErrorMapping +import kafka.network.{RequestOrResponseSend, RequestChannel} +import kafka.network.RequestChannel.Response + +object ConsumerMetadataRequest { + val CurrentVersion = 0.shortValue + val DefaultClientId = "" + + def readFrom(buffer: ByteBuffer) = { + // envelope + val versionId = buffer.getShort + val correlationId = buffer.getInt + val clientId = ApiUtils.readShortString(buffer) + + // request + val group = ApiUtils.readShortString(buffer) + ConsumerMetadataRequest(group, versionId, correlationId, clientId) + } + +} + +case class ConsumerMetadataRequest(group: String, + versionId: Short = ConsumerMetadataRequest.CurrentVersion, + correlationId: Int = 0, + clientId: String = ConsumerMetadataRequest.DefaultClientId) + extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey)) { + + def sizeInBytes = + 2 + /* versionId */ + 4 + /* correlationId */ + ApiUtils.shortStringLength(clientId) + + ApiUtils.shortStringLength(group) + + def writeTo(buffer: ByteBuffer) { + // envelope + buffer.putShort(versionId) + buffer.putInt(correlationId) + ApiUtils.writeShortString(buffer, clientId) + + // consumer metadata request + ApiUtils.writeShortString(buffer, group) + } + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + // return ConsumerCoordinatorNotAvailable for all uncaught errors + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, correlationId) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) + } + + def describe(details: Boolean) = { + val consumerMetadataRequest = new StringBuilder + consumerMetadataRequest.append("Name: " + this.getClass.getSimpleName) + consumerMetadataRequest.append("; Version: " + versionId) + consumerMetadataRequest.append("; CorrelationId: " + correlationId) + consumerMetadataRequest.append("; ClientId: " + clientId) + consumerMetadataRequest.append("; Group: " + group) + consumerMetadataRequest.toString() + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala new file mode 100644 index 0000000000000..ea1c0d04b1036 --- /dev/null +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.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.api + +import java.nio.ByteBuffer +import kafka.cluster.BrokerEndPoint +import kafka.common.ErrorMapping + +object ConsumerMetadataResponse { + val CurrentVersion = 0 + + private val NoBrokerEndpointOpt = Some(BrokerEndPoint(id = -1, host = "", port = -1)) + + def readFrom(buffer: ByteBuffer) = { + val correlationId = buffer.getInt + val errorCode = buffer.getShort + val broker = BrokerEndPoint.readFrom(buffer) + val coordinatorOpt = if (errorCode == ErrorMapping.NoError) + Some(broker) + else + None + + ConsumerMetadataResponse(coordinatorOpt, errorCode, correlationId) + } + +} + +case class ConsumerMetadataResponse (coordinatorOpt: Option[BrokerEndPoint], errorCode: Short, correlationId: Int) + extends RequestOrResponse() { + + def sizeInBytes = + 4 + /* correlationId */ + 2 + /* error code */ + coordinatorOpt.orElse(ConsumerMetadataResponse.NoBrokerEndpointOpt).get.sizeInBytes + + def writeTo(buffer: ByteBuffer) { + buffer.putInt(correlationId) + buffer.putShort(errorCode) + coordinatorOpt.orElse(ConsumerMetadataResponse.NoBrokerEndpointOpt).foreach(_.writeTo(buffer)) + } + + def describe(details: Boolean) = toString +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala index 7dacb20237880..809200706ef58 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala @@ -18,10 +18,9 @@ package kafka.api import java.nio.ByteBuffer -import kafka.api.ApiUtils._ -import collection.mutable.ListBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.common.{TopicAndPartition, ErrorMapping} + +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.utils.Logging @@ -37,10 +36,10 @@ object ControlledShutdownRequest extends Logging { } } -case class ControlledShutdownRequest(val versionId: Short, - override val correlationId: Int, - val brokerId: Int) - extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey), correlationId){ +case class ControlledShutdownRequest(versionId: Short, + correlationId: Int, + brokerId: Int) + extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey)){ def this(correlationId: Int, brokerId: Int) = this(ControlledShutdownRequest.CurrentVersion, correlationId, brokerId) @@ -63,7 +62,7 @@ case class ControlledShutdownRequest(val versionId: Short, 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))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean = false): String = { @@ -74,4 +73,4 @@ case class ControlledShutdownRequest(val versionId: Short, controlledShutdownRequest.append("; BrokerId: " + brokerId) controlledShutdownRequest.toString() } -} \ 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 46ec3db28f88b..9ecdee73c171b 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala @@ -39,10 +39,10 @@ object ControlledShutdownResponse { } -case class ControlledShutdownResponse(override val correlationId: Int, - val errorCode: Short = ErrorMapping.NoError, - val partitionsRemaining: Set[TopicAndPartition]) - extends RequestOrResponse(correlationId = correlationId) { +case class ControlledShutdownResponse(correlationId: Int, + errorCode: Short = ErrorMapping.NoError, + partitionsRemaining: Set[TopicAndPartition]) + extends RequestOrResponse() { def sizeInBytes(): Int ={ var size = 4 /* correlation id */ + @@ -68,4 +68,4 @@ case class ControlledShutdownResponse(override val correlationId: Int, 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 dea118a2e3792..5b38f8554898e 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -17,20 +17,19 @@ package kafka.api -import java.nio.ByteBuffer import kafka.utils.nonthreadsafe import kafka.api.ApiUtils._ -import scala.collection.immutable.Map import kafka.common.{ErrorMapping, TopicAndPartition} import kafka.consumer.ConsumerConfig -import java.util.concurrent.atomic.AtomicInteger import kafka.network.RequestChannel import kafka.message.MessageSet +import java.util.concurrent.atomic.AtomicInteger +import java.nio.ByteBuffer +import scala.collection.immutable.Map case class PartitionFetchInfo(offset: Long, fetchSize: Int) - object FetchRequest { val CurrentVersion = 0.shortValue val DefaultMaxWait = 0 @@ -59,14 +58,14 @@ object FetchRequest { } } -case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentVersion, - override val correlationId: Int = FetchRequest.DefaultCorrelationId, - clientId: String = ConsumerConfig.DefaultClientId, - replicaId: Int = Request.OrdinaryConsumerId, - maxWait: Int = FetchRequest.DefaultMaxWait, - minBytes: Int = FetchRequest.DefaultMinBytes, - requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) - extends RequestOrResponse(Some(RequestKeys.FetchKey), correlationId) { +case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, + correlationId: Int = FetchRequest.DefaultCorrelationId, + clientId: String = ConsumerConfig.DefaultClientId, + replicaId: Int = Request.OrdinaryConsumerId, + maxWait: Int = FetchRequest.DefaultMaxWait, + minBytes: Int = FetchRequest.DefaultMinBytes, + requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) + extends RequestOrResponse(Some(RequestKeys.FetchKey)) { /** * Partitions the request info into a map of maps (one for each topic). @@ -132,7 +131,7 @@ case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentV }) } - def isFromFollower = Request.isReplicaIdFromFollower(replicaId) + def isFromFollower = Request.isValidBrokerId(replicaId) def isFromOrdinaryConsumer = replicaId == Request.OrdinaryConsumerId @@ -150,7 +149,7 @@ case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentV (topicAndPartition, FetchResponsePartitionData(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1, MessageSet.Empty)) } val errorResponse = FetchResponse(correlationId, fetchResponsePartitionData) - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(errorResponse))) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index d117f10f724b0..0b6b33ab6f7a7 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -19,10 +19,15 @@ package kafka.api import java.nio.ByteBuffer import java.nio.channels.GatheringByteChannel + import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.message.{MessageSet, ByteBufferMessageSet} -import kafka.network.{MultiSend, Send} import kafka.api.ApiUtils._ +import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.network.Send +import org.apache.kafka.common.network.MultiSend + +import scala.collection._ object FetchResponsePartitionData { def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = { @@ -59,10 +64,12 @@ class PartitionDataSend(val partitionId: Int, buffer.putInt(partitionData.messages.sizeInBytes) buffer.rewind() - override def complete = !buffer.hasRemaining && messagesSentSize >= messageSize + override def completed = !buffer.hasRemaining && messagesSentSize >= messageSize - override def writeTo(channel: GatheringByteChannel): Int = { - var written = 0 + override def destination: String = "" + + override def writeTo(channel: GatheringByteChannel): Long = { + var written = 0L if(buffer.hasRemaining) written += channel.write(buffer) if(!buffer.hasRemaining && messagesSentSize < messageSize) { @@ -72,6 +79,8 @@ class PartitionDataSend(val partitionId: Int, } written } + + override def size = buffer.capacity() + messageSize } object TopicData { @@ -98,29 +107,32 @@ case class TopicData(topic: String, partitionData: Map[Int, FetchResponsePartiti val headerSize = TopicData.headerSize(topic) } -class TopicDataSend(val topicData: TopicData) extends Send { - private val size = topicData.sizeInBytes +class TopicDataSend(val dest: String, val topicData: TopicData) extends Send { - private var sent = 0 + private var sent = 0L - override def complete = sent >= size + override def completed: Boolean = sent >= size + + override def destination: String = dest + + override def size = topicData.headerSize + sends.size() private val buffer = ByteBuffer.allocate(topicData.headerSize) writeShortString(buffer, topicData.topic) buffer.putInt(topicData.partitionData.size) buffer.rewind() - val sends = new MultiSend(topicData.partitionData.toList - .map(d => new PartitionDataSend(d._1, d._2))) { - val expectedBytesToWrite = topicData.sizeInBytes - topicData.headerSize - } + private val sends = new MultiSend(dest, + JavaConversions.seqAsJavaList(topicData.partitionData.toList.map(d => new PartitionDataSend(d._1, d._2)))) + + override def writeTo(channel: GatheringByteChannel): Long = { + if (completed) + throw new KafkaException("This operation cannot be completed on a complete request.") - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - var written = 0 + var written = 0L if(buffer.hasRemaining) written += channel.write(buffer) - if(!buffer.hasRemaining && !sends.complete) { + if(!buffer.hasRemaining && !sends.completed) { written += sends.writeTo(channel) } sent += written @@ -149,9 +161,8 @@ object FetchResponse { } } - -case class FetchResponse(correlationId: Int, - data: Map[TopicAndPartition, FetchResponsePartitionData]) { +case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchResponsePartitionData]) + extends RequestOrResponse() { /** * Partitions the data into a map of maps (one for each topic). @@ -167,6 +178,16 @@ case class FetchResponse(correlationId: Int, folded + topicData.sizeInBytes }) + /* + * FetchResponse uses [sendfile](http://man7.org/linux/man-pages/man2/sendfile.2.html) + * api for data transfer through the FetchResponseSend, so `writeTo` aren't actually being used. + * It is implemented as an empty function to conform to `RequestOrResponse.writeTo` + * abstract method signature. + */ + def writeTo(buffer: ByteBuffer): Unit = throw new UnsupportedOperationException + + override def describe(details: Boolean): String = toString + private def partitionDataFor(topic: String, partition: Int): FetchResponsePartitionData = { val topicAndPartition = TopicAndPartition(topic, partition) data.get(topicAndPartition) match { @@ -188,34 +209,36 @@ case class FetchResponse(correlationId: Int, } -class FetchResponseSend(val fetchResponse: FetchResponse) extends Send { - private val size = fetchResponse.sizeInBytes +class FetchResponseSend(val dest: String, val fetchResponse: FetchResponse) extends Send { + private val payloadSize = fetchResponse.sizeInBytes - private var sent = 0 + private var sent = 0L - private val sendSize = 4 /* for size */ + size + override def size = 4 /* for size byte */ + payloadSize - override def complete = sent >= sendSize + override def completed = sent >= size + + override def destination = dest private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize) - buffer.putInt(size) + buffer.putInt(payloadSize) buffer.putInt(fetchResponse.correlationId) buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count buffer.rewind() - val sends = new MultiSend(fetchResponse.dataGroupedByTopic.toList.map { - case(topic, data) => new TopicDataSend(TopicData(topic, + private val sends = new MultiSend(dest, JavaConversions.seqAsJavaList(fetchResponse.dataGroupedByTopic.toList.map { + case(topic, data) => new TopicDataSend(dest, TopicData(topic, data.map{case(topicAndPartition, message) => (topicAndPartition.partition, message)})) - }) { - val expectedBytesToWrite = fetchResponse.sizeInBytes - FetchResponse.headerSize - } + })) + + override def writeTo(channel: GatheringByteChannel): Long = { + if (completed) + throw new KafkaException("This operation cannot be completed on a complete request.") - def writeTo(channel: GatheringByteChannel):Int = { - expectIncomplete() - var written = 0 + var written = 0L if(buffer.hasRemaining) written += channel.write(buffer) - if(!buffer.hasRemaining && !sends.complete) { + if(!buffer.hasRemaining && !sends.completed) { written += sends.writeTo(channel) } sent += written diff --git a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala new file mode 100644 index 0000000000000..b0c6d7a339850 --- /dev/null +++ b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer +import org.apache.kafka.common.requests.AbstractRequestResponse +import kafka.api.ApiUtils._ + +private[kafka] abstract class GenericRequestAndHeader(val versionId: Short, + val correlationId: Int, + val clientId: String, + val body: AbstractRequestResponse, + val name: String, + override val requestId: Option[Short] = None) + extends RequestOrResponse(requestId) { + + def writeTo(buffer: ByteBuffer) { + buffer.putShort(versionId) + buffer.putInt(correlationId) + writeShortString(buffer, clientId) + body.writeTo(buffer) + } + + def sizeInBytes(): Int = { + 2 /* version id */ + + 4 /* correlation id */ + + (2 + clientId.length) /* client id */ + + body.sizeOf() + } + + override def toString(): String = { + describe(true) + } + + override def describe(details: Boolean): String = { + val strBuffer = new StringBuilder + strBuffer.append("Name: " + name) + strBuffer.append("; Version: " + versionId) + strBuffer.append("; CorrelationId: " + correlationId) + strBuffer.append("; ClientId: " + clientId) + strBuffer.append("; Body: " + body.toString) + strBuffer.toString() + } +} diff --git a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala new file mode 100644 index 0000000000000..748b5e935235f --- /dev/null +++ b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package kafka.api + +import java.nio.ByteBuffer +import org.apache.kafka.common.requests.AbstractRequestResponse + +private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int, + val body: AbstractRequestResponse, + val name: String, + override val requestId: Option[Short] = None) + extends RequestOrResponse(requestId) { + + def writeTo(buffer: ByteBuffer) { + buffer.putInt(correlationId) + body.writeTo(buffer) + } + + def sizeInBytes(): Int = { + 4 /* correlation id */ + + body.sizeOf() + } + + override def toString(): String = { + describe(true) + } + + override def describe(details: Boolean): String = { + val strBuffer = new StringBuilder + strBuffer.append("Name: " + name) + strBuffer.append("; CorrelationId: " + correlationId) + strBuffer.append("; Body: " + body.toString) + strBuffer.toString() + } +} diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index 03117377e1cb2..c2584e0c94301 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -19,19 +19,23 @@ package kafka.api import java.nio._ -import kafka.utils._ + import kafka.api.ApiUtils._ -import kafka.cluster.Broker -import kafka.controller.LeaderIsrAndControllerEpoch -import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.cluster.BrokerEndPoint import kafka.common.ErrorMapping +import kafka.controller.LeaderIsrAndControllerEpoch +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response -import collection.Set +import kafka.utils._ + +import scala.collection.Set object LeaderAndIsr { val initialLeaderEpoch: Int = 0 val initialZKVersion: Int = 0 + val NoLeader = -1 + val LeaderDuringDelete = -2 } case class LeaderAndIsr(var leader: Int, var leaderEpoch: Int, var isr: List[Int], var zkVersion: Int) { @@ -57,8 +61,8 @@ object PartitionStateInfo { } } -case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, - val allReplicas: Set[Int]) { +case class PartitionStateInfo(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, + allReplicas: Set[Int]) { def replicationFactor = allReplicas.size def writeTo(buffer: ByteBuffer) { @@ -118,24 +122,24 @@ object LeaderAndIsrRequest { } val leadersCount = buffer.getInt - var leaders = Set[Broker]() + var leaders = Set[BrokerEndPoint]() for (i <- 0 until leadersCount) - leaders += Broker.readFrom(buffer) + leaders += BrokerEndPoint.readFrom(buffer) new LeaderAndIsrRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos.toMap, leaders) } } case class LeaderAndIsrRequest (versionId: Short, - override val correlationId: Int, + correlationId: Int, clientId: String, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[(String, Int), PartitionStateInfo], - leaders: Set[Broker]) - extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey), correlationId) { + leaders: Set[BrokerEndPoint]) + extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) { - def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[Broker], controllerId: Int, + def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[BrokerEndPoint], controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String) = { this(LeaderAndIsrRequest.CurrentVersion, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos, leaders) @@ -182,7 +186,7 @@ case class LeaderAndIsrRequest (versionId: Short, case (topicAndPartition, partitionAndState) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } val errorResponse = LeaderAndIsrResponse(correlationId, responseMap) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { @@ -198,4 +202,4 @@ case class LeaderAndIsrRequest (versionId: Short, leaderAndIsrRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) leaderAndIsrRequest.toString() } -} \ 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 f63644448bb5a..22ce48a0cc0ab 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala @@ -41,10 +41,10 @@ object LeaderAndIsrResponse { } -case class LeaderAndIsrResponse(override val correlationId: Int, +case class LeaderAndIsrResponse(correlationId: Int, responseMap: Map[(String, Int), Short], errorCode: Short = ErrorMapping.NoError) - extends RequestOrResponse(correlationId = correlationId) { + extends RequestOrResponse() { def sizeInBytes(): Int ={ var size = 4 /* correlation id */ + diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 4d1fa5cbfde92..5b362ef7a7652 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -20,22 +20,49 @@ 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.common.{ErrorMapping, OffsetAndMetadata, TopicAndPartition} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response +import kafka.utils.Logging + +import scala.collection._ + object OffsetCommitRequest extends Logging { - val CurrentVersion: Short = 0 + val CurrentVersion: Short = 2 val DefaultClientId = "" def readFrom(buffer: ByteBuffer): OffsetCommitRequest = { // Read values from the envelope val versionId = buffer.getShort + assert(versionId == 0 || versionId == 1 || versionId == 2, + "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0, 1 or 2.") + val correlationId = buffer.getInt val clientId = readShortString(buffer) // Read the OffsetRequest val consumerGroupId = readShortString(buffer) + + // version 1 and 2 specific fields + val groupGenerationId: Int = + if (versionId >= 1) + buffer.getInt + else + org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID + + val consumerId: String = + if (versionId >= 1) + readShortString(buffer) + else + org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID + + // version 2 specific fields + val retentionMs: Long = + if (versionId >= 2) + buffer.getLong + else + org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME + val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { val topic = readShortString(buffer) @@ -43,23 +70,38 @@ object OffsetCommitRequest extends Logging { (1 to partitionCount).map(_ => { val partitionId = buffer.getInt val offset = buffer.getLong + val timestamp = { + // version 1 specific field + if (versionId == 1) + buffer.getLong + else + org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP + } val metadata = readShortString(buffer) - (TopicAndPartition(topic, partitionId), OffsetMetadataAndError(offset, metadata)) + + (TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp)) }) }) - OffsetCommitRequest(consumerGroupId, Map(pairs:_*), versionId, correlationId, clientId) + + OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId, retentionMs) } } case class OffsetCommitRequest(groupId: String, - requestInfo: Map[TopicAndPartition, OffsetMetadataAndError], + requestInfo: immutable.Map[TopicAndPartition, OffsetAndMetadata], versionId: Short = OffsetCommitRequest.CurrentVersion, - override val correlationId: Int = 0, - clientId: String = OffsetCommitRequest.DefaultClientId) - extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey), correlationId) { + correlationId: Int = 0, + clientId: String = OffsetCommitRequest.DefaultClientId, + groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID, + consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID, + retentionMs: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) + extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey)) { + + assert(versionId == 0 || versionId == 1 || versionId == 2, + "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0, 1 or 2.") lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) - + def writeTo(buffer: ByteBuffer) { // Write envelope buffer.putShort(versionId) @@ -68,14 +110,29 @@ case class OffsetCommitRequest(groupId: String, // Write OffsetCommitRequest writeShortString(buffer, groupId) // consumer group + + // version 1 and 2 specific data + if (versionId >= 1) { + buffer.putInt(groupGenerationId) + writeShortString(buffer, consumerId) + } + + // version 2 or above specific data + if (versionId >= 2) { + buffer.putLong(retentionMs) + } + 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 + buffer.putInt(t2._1.partition) + buffer.putLong(t2._2.offset) + // version 1 specific data + if (versionId == 1) + buffer.putLong(t2._2.commitTimestamp) + writeShortString(buffer, t2._2.metadata) }) }) } @@ -84,7 +141,9 @@ case class OffsetCommitRequest(groupId: String, 2 + /* versionId */ 4 + /* correlationId */ shortStringLength(clientId) + - shortStringLength(groupId) + + shortStringLength(groupId) + + (if (versionId >= 1) 4 /* group generation id */ + shortStringLength(consumerId) else 0) + + (if (versionId >= 2) 8 /* retention time */ else 0) + 4 + /* topic count */ requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { val (topic, offsets) = topicAndOffsets @@ -95,16 +154,17 @@ case class OffsetCommitRequest(groupId: String, innerCount + 4 /* partition */ + 8 /* offset */ + + (if (versionId == 1) 8 else 0) /* timestamp */ + 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))) + val errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + val commitStatus = requestInfo.mapValues(_ => errorCode) + val commitResponse = OffsetCommitResponse(commitStatus, correlationId) + + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, commitResponse))) } override def describe(details: Boolean): String = { @@ -114,12 +174,15 @@ case class OffsetCommitRequest(groupId: String, offsetCommitRequest.append("; CorrelationId: " + correlationId) offsetCommitRequest.append("; ClientId: " + clientId) offsetCommitRequest.append("; GroupId: " + groupId) + offsetCommitRequest.append("; GroupGenerationId: " + groupGenerationId) + offsetCommitRequest.append("; ConsumerId: " + consumerId) + offsetCommitRequest.append("; RetentionMs: " + retentionMs) if(details) offsetCommitRequest.append("; RequestInfo: " + requestInfo.mkString(",")) offsetCommitRequest.toString() } - override def toString(): String = { - describe(true) + override def toString = { + describe(details = true) } } diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala index 9e1795f9db15e..116547ae1139e 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala @@ -19,9 +19,8 @@ package kafka.api import java.nio.ByteBuffer -import kafka.api.ApiUtils._ -import kafka.common.TopicAndPartition import kafka.utils.Logging +import kafka.common.{ErrorMapping, TopicAndPartition} object OffsetCommitResponse extends Logging { val CurrentVersion: Short = 0 @@ -30,7 +29,7 @@ object OffsetCommitResponse extends Logging { val correlationId = buffer.getInt val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { - val topic = readShortString(buffer) + val topic = ApiUtils.readShortString(buffer) val partitionCount = buffer.getInt (1 to partitionCount).map(_ => { val partitionId = buffer.getInt @@ -42,37 +41,36 @@ object OffsetCommitResponse extends Logging { } } -case class OffsetCommitResponse(requestInfo: Map[TopicAndPartition, Short], - override val correlationId: Int = 0) - extends RequestOrResponse(correlationId=correlationId) { +case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Short], + correlationId: Int = 0) + extends RequestOrResponse() { - lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) + lazy val commitStatusGroupedByTopic = commitStatus.groupBy(_._1.topic) + + def hasError = commitStatus.exists{ case (topicAndPartition, errorCode) => errorCode != ErrorMapping.NoError } 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 - }) - }) + buffer.putInt(commitStatusGroupedByTopic.size) + commitStatusGroupedByTopic.foreach { case(topic, statusMap) => + ApiUtils.writeShortString(buffer, topic) + buffer.putInt(statusMap.size) // partition count + statusMap.foreach { case(topicAndPartition, errorCode) => + buffer.putInt(topicAndPartition.partition) + buffer.putShort(errorCode) + } + } } override def sizeInBytes = 4 + /* correlationId */ 4 + /* topic count */ - requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { - val (topic, offsets) = topicAndOffsets + commitStatusGroupedByTopic.foldLeft(0)((count, partitionStatusMap) => { + val (topic, partitionStatus) = partitionStatusMap count + - shortStringLength(topic) + /* topic */ - 4 + /* number of partitions */ - offsets.size * ( - 4 + /* partition */ - 2 /* error */ - ) + ApiUtils.shortStringLength(topic) + + 4 + /* partition count */ + partitionStatus.size * ( 4 /* partition */ + 2 /* error code */) }) 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 index 7036532db1405..a83e147b2c9c1 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -20,12 +20,13 @@ package kafka.api import java.nio.ByteBuffer import kafka.api.ApiUtils._ -import kafka.utils.Logging -import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.common.{TopicAndPartition, _} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response -import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition} +import kafka.utils.Logging + object OffsetFetchRequest extends Logging { - val CurrentVersion: Short = 0 + val CurrentVersion: Short = 1 val DefaultClientId = "" def readFrom(buffer: ByteBuffer): OffsetFetchRequest = { @@ -50,11 +51,11 @@ object OffsetFetchRequest extends Logging { } 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) { + requestInfo: Seq[TopicAndPartition], + versionId: Short = OffsetFetchRequest.CurrentVersion, + correlationId: Int = 0, + clientId: String = OffsetFetchRequest.DefaultClientId) + extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey)) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic) @@ -91,12 +92,11 @@ case class OffsetFetchRequest(groupId: String, 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]]) + ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) )) }.toMap val errorResponse = OffsetFetchResponse(requestInfo=responseMap, correlationId=correlationId) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { @@ -111,7 +111,7 @@ case class OffsetFetchRequest(groupId: String, offsetFetchRequest.toString() } - override def toString(): String = { - describe(true) + override def toString: String = { + describe(details = 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 index c1222f422ddb6..e3523f8dcc028 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala @@ -45,8 +45,8 @@ object OffsetFetchResponse extends Logging { } case class OffsetFetchResponse(requestInfo: Map[TopicAndPartition, OffsetMetadataAndError], - override val correlationId: Int = 0) - extends RequestOrResponse(correlationId = correlationId) { + correlationId: Int = 0) + extends RequestOrResponse() { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala index 7cbc26c6e3842..f418868046f7c 100644 --- a/core/src/main/scala/kafka/api/OffsetRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetRequest.scala @@ -18,9 +18,10 @@ package kafka.api import java.nio.ByteBuffer -import kafka.common.{ErrorMapping, TopicAndPartition} + import kafka.api.ApiUtils._ -import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response @@ -57,10 +58,10 @@ case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int) case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], versionId: Short = OffsetRequest.CurrentVersion, - override val correlationId: Int = 0, + correlationId: Int = 0, clientId: String = OffsetRequest.DefaultClientId, replicaId: Int = Request.OrdinaryConsumerId) - extends RequestOrResponse(Some(RequestKeys.OffsetsKey), correlationId) { + extends RequestOrResponse(Some(RequestKeys.OffsetsKey)) { def this(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], correlationId: Int, replicaId: Int) = this(requestInfo, OffsetRequest.CurrentVersion, correlationId, OffsetRequest.DefaultClientId, replicaId) @@ -117,7 +118,7 @@ case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequ (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), null)) } val errorResponse = OffsetResponse(correlationId, partitionOffsetResponseMap) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/OffsetResponse.scala b/core/src/main/scala/kafka/api/OffsetResponse.scala index 0e1d6e362a1ce..63c0899ec46c4 100644 --- a/core/src/main/scala/kafka/api/OffsetResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetResponse.scala @@ -51,9 +51,9 @@ case class PartitionOffsetsResponse(error: Short, offsets: Seq[Long]) { } -case class OffsetResponse(override val correlationId: Int, +case class OffsetResponse(correlationId: Int, partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse]) - extends RequestOrResponse(correlationId = correlationId) { + extends RequestOrResponse() { lazy val offsetsGroupedByTopic = partitionErrorAndOffsets.groupBy(_._1.topic) diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index 0c295a2fe6712..c866180d3680d 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -18,11 +18,12 @@ package kafka.api import java.nio._ -import kafka.message._ + import kafka.api.ApiUtils._ import kafka.common._ +import kafka.message._ +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response -import kafka.network.{RequestChannel, BoundedByteBufferSend} object ProducerRequest { val CurrentVersion = 0.shortValue @@ -53,12 +54,12 @@ object ProducerRequest { } case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, - override val correlationId: Int, + correlationId: Int, clientId: String, requiredAcks: Short, ackTimeoutMs: Int, data: collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]) - extends RequestOrResponse(Some(RequestKeys.ProduceKey), correlationId) { + extends RequestOrResponse(Some(RequestKeys.ProduceKey)) { /** * Partitions the data into a map of maps (one for each topic). @@ -136,7 +137,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, (topicAndPartition, ProducerResponseStatus(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1l)) } val errorResponse = ProducerResponse(correlationId, producerResponseStatus) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } } @@ -153,7 +154,6 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, 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 06261b9136399..5d1fac4cb8943 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -41,11 +41,10 @@ object ProducerResponse { } } -case class ProducerResponseStatus(error: Short, offset: Long) +case class ProducerResponseStatus(var error: Short, offset: Long) -case class ProducerResponse(override val correlationId: Int, - status: Map[TopicAndPartition, ProducerResponseStatus]) - extends RequestOrResponse(correlationId = correlationId) { +case class ProducerResponse(correlationId: Int, status: Map[TopicAndPartition, ProducerResponseStatus]) + extends RequestOrResponse() { /** * Partitions the status map into a map of maps (one for each topic). diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala index c81214fec0bf7..155cb650e9cff 100644 --- a/core/src/main/scala/kafka/api/RequestKeys.scala +++ b/core/src/main/scala/kafka/api/RequestKeys.scala @@ -20,6 +20,8 @@ package kafka.api import kafka.common.KafkaException import java.nio.ByteBuffer +import kafka.network.InvalidRequestException + object RequestKeys { val ProduceKey: Short = 0 val FetchKey: Short = 1 @@ -31,6 +33,9 @@ object RequestKeys { val ControlledShutdownKey: Short = 7 val OffsetCommitKey: Short = 8 val OffsetFetchKey: Short = 9 + val ConsumerMetadataKey: Short = 10 + val JoinGroupKey: Short = 11 + val HeartbeatKey: Short = 12 val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]= Map(ProduceKey -> ("Produce", ProducerRequest.readFrom), @@ -42,7 +47,9 @@ object RequestKeys { UpdateMetadataKey -> ("UpdateMetadata", UpdateMetadataRequest.readFrom), ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom), OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom), - OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom)) + OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom), + ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom) + ) def nameForKey(key: Short): String = { keyToNameAndDeserializerMap.get(key) match { @@ -54,7 +61,7 @@ object RequestKeys { def deserializerForKey(key: Short): (ByteBuffer) => RequestOrResponse = { keyToNameAndDeserializerMap.get(key) match { case Some(nameAndSerializer) => nameAndSerializer._2 - case None => throw new KafkaException("Wrong request type %d".format(key)) + case None => throw new InvalidRequestException("Wrong request type %d".format(key)) } } } diff --git a/core/src/main/scala/kafka/api/RequestOrResponse.scala b/core/src/main/scala/kafka/api/RequestOrResponse.scala index 708e547a358c9..73ec1d9fb811c 100644 --- a/core/src/main/scala/kafka/api/RequestOrResponse.scala +++ b/core/src/main/scala/kafka/api/RequestOrResponse.scala @@ -25,12 +25,12 @@ 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) + // Broker ids are non-negative int. + def isValidBrokerId(brokerId: Int): Boolean = (brokerId >= 0) } -private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None, val correlationId: Int) extends Logging { +abstract class RequestOrResponse(val requestId: Option[Short] = None) extends Logging { def sizeInBytes: Int diff --git a/core/src/main/scala/kafka/api/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala index 68fc1389ee711..4441fc677ca48 100644 --- a/core/src/main/scala/kafka/api/StopReplicaRequest.scala +++ b/core/src/main/scala/kafka/api/StopReplicaRequest.scala @@ -19,7 +19,7 @@ package kafka.api import java.nio._ import kafka.api.ApiUtils._ -import kafka.network.{BoundedByteBufferSend, RequestChannel, InvalidRequestException} +import kafka.network.{RequestOrResponseSend, RequestChannel, InvalidRequestException} import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.network.RequestChannel.Response import kafka.utils.Logging @@ -54,13 +54,13 @@ object StopReplicaRequest extends Logging { } case class StopReplicaRequest(versionId: Short, - override val correlationId: Int, + correlationId: Int, clientId: String, controllerId: Int, controllerEpoch: Int, deletePartitions: Boolean, partitions: Set[TopicAndPartition]) - extends RequestOrResponse(Some(RequestKeys.StopReplicaKey), correlationId) { + extends RequestOrResponse(Some(RequestKeys.StopReplicaKey)) { def this(deletePartitions: Boolean, partitions: Set[TopicAndPartition], controllerId: Int, controllerEpoch: Int, correlationId: Int) = { this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId, @@ -106,7 +106,7 @@ case class StopReplicaRequest(versionId: Short, case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) }.toMap val errorResponse = StopReplicaResponse(correlationId, responseMap) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/StopReplicaResponse.scala b/core/src/main/scala/kafka/api/StopReplicaResponse.scala index c90ddee3d8204..2fc3c9585fbc6 100644 --- a/core/src/main/scala/kafka/api/StopReplicaResponse.scala +++ b/core/src/main/scala/kafka/api/StopReplicaResponse.scala @@ -42,10 +42,10 @@ object StopReplicaResponse { } -case class StopReplicaResponse(override val correlationId: Int, - val responseMap: Map[TopicAndPartition, Short], - val errorCode: Short = ErrorMapping.NoError) - extends RequestOrResponse(correlationId = correlationId) { +case class StopReplicaResponse(correlationId: Int, + responseMap: Map[TopicAndPartition, Short], + errorCode: Short = ErrorMapping.NoError) + extends RequestOrResponse() { def sizeInBytes(): Int ={ var size = 4 /* correlation id */ + @@ -72,4 +72,4 @@ case class StopReplicaResponse(override val correlationId: Int, } override def describe(details: Boolean):String = { toString } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index 0513a59ed94e5..7b56b31d697a2 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -17,24 +17,25 @@ package kafka.api -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.utils.Logging -import collection.mutable.ArrayBuffer import kafka.common._ object TopicMetadata { val NoLeaderNodeId = -1 - def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): TopicMetadata = { + def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndPoint]): TopicMetadata = { val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue)) val topic = readShortString(buffer) val numPartitions = readIntInRange(buffer, "number of partitions", (0, Int.MaxValue)) - val partitionsMetadata = new ArrayBuffer[PartitionMetadata]() - for(i <- 0 until numPartitions) - partitionsMetadata += PartitionMetadata.readFrom(buffer, brokers) + val partitionsMetadata: Array[PartitionMetadata] = new Array[PartitionMetadata](numPartitions) + for(i <- 0 until numPartitions) { + val partitionMetadata = PartitionMetadata.readFrom(buffer, brokers) + partitionsMetadata(i) = partitionMetadata + } new TopicMetadata(topic, partitionsMetadata, errorCode) } } @@ -87,7 +88,7 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat object PartitionMetadata { - def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): PartitionMetadata = { + def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndPoint]): PartitionMetadata = { val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue)) val partitionId = readIntInRange(buffer, "partition id", (0, Int.MaxValue)) /* partition id */ val leaderId = buffer.getInt @@ -108,9 +109,9 @@ object PartitionMetadata { } case class PartitionMetadata(partitionId: Int, - val leader: Option[Broker], - replicas: Seq[Broker], - isr: Seq[Broker] = Seq.empty, + leader: Option[BrokerEndPoint], + replicas: Seq[BrokerEndPoint], + isr: Seq[BrokerEndPoint] = Seq.empty, errorCode: Short = ErrorMapping.NoError) extends Logging { def sizeInBytes: Int = { 2 /* error code */ + @@ -140,14 +141,13 @@ case class PartitionMetadata(partitionId: Int, override def toString(): String = { val partitionMetadataString = new StringBuilder partitionMetadataString.append("\tpartition " + partitionId) - partitionMetadataString.append("\tleader: " + (if(leader.isDefined) formatBroker(leader.get) else "none")) - partitionMetadataString.append("\treplicas: " + replicas.map(formatBroker).mkString(",")) - partitionMetadataString.append("\tisr: " + isr.map(formatBroker).mkString(",")) + partitionMetadataString.append("\tleader: " + (if(leader.isDefined) leader.get.toString else "none")) + partitionMetadataString.append("\treplicas: " + replicas.mkString(",")) + partitionMetadataString.append("\tisr: " + isr.mkString(",")) partitionMetadataString.append("\tisUnderReplicated: %s".format(if(isr.size < replicas.size) "true" else "false")) partitionMetadataString.toString() } - private def formatBroker(broker: Broker) = broker.id + " (" + broker.host + ":" + broker.port + ")" } diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index a319f2f438bfd..401c583671499 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -18,13 +18,15 @@ package kafka.api import java.nio.ByteBuffer + import kafka.api.ApiUtils._ -import collection.mutable.ListBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.utils.Logging +import scala.collection.mutable.ListBuffer + object TopicMetadataRequest extends Logging { val CurrentVersion = 0.shortValue val DefaultClientId = "" @@ -46,11 +48,11 @@ object TopicMetadataRequest extends Logging { } } -case class TopicMetadataRequest(val versionId: Short, - override val correlationId: Int, - val clientId: String, - val topics: Seq[String]) - extends RequestOrResponse(Some(RequestKeys.MetadataKey), correlationId){ +case class TopicMetadataRequest(versionId: Short, + correlationId: Int, + clientId: String, + topics: Seq[String]) + extends RequestOrResponse(Some(RequestKeys.MetadataKey)){ def this(topics: Seq[String], correlationId: Int) = this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics) @@ -79,8 +81,8 @@ case class TopicMetadataRequest(val versionId: Short, val topicMetadata = topics.map { topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } - val errorResponse = TopicMetadataResponse(topicMetadata, correlationId) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + val errorResponse = TopicMetadataResponse(Seq(), topicMetadata, correlationId) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { @@ -93,4 +95,4 @@ case class TopicMetadataRequest(val versionId: Short, 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 f6b7429faeab3..f2f89e0b3f1e3 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala @@ -17,7 +17,7 @@ package kafka.api -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import java.nio.ByteBuffer object TopicMetadataResponse { @@ -25,38 +25,31 @@ object TopicMetadataResponse { def readFrom(buffer: ByteBuffer): TopicMetadataResponse = { val correlationId = buffer.getInt val brokerCount = buffer.getInt - val brokers = (0 until brokerCount).map(_ => Broker.readFrom(buffer)) + val brokers = (0 until brokerCount).map(_ => BrokerEndPoint.readFrom(buffer)) val brokerMap = brokers.map(b => (b.id, b)).toMap val topicCount = buffer.getInt val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap)) - new TopicMetadataResponse(topicsMetadata, correlationId) + new TopicMetadataResponse(brokers, topicsMetadata, correlationId) } } -case class TopicMetadataResponse(topicsMetadata: Seq[TopicMetadata], - override val correlationId: Int) - extends RequestOrResponse(correlationId = correlationId) { +case class TopicMetadataResponse(brokers: Seq[BrokerEndPoint], + topicsMetadata: Seq[TopicMetadata], + correlationId: Int) + extends RequestOrResponse() { val sizeInBytes: Int = { - val brokers = extractBrokers(topicsMetadata).values 4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum } def writeTo(buffer: ByteBuffer) { buffer.putInt(correlationId) /* brokers */ - val brokers = extractBrokers(topicsMetadata).values buffer.putInt(brokers.size) brokers.foreach(_.writeTo(buffer)) /* topic metadata */ buffer.putInt(topicsMetadata.length) topicsMetadata.foreach(_.writeTo(buffer)) } - - def extractBrokers(topicMetadatas: Seq[TopicMetadata]): Map[Int, Broker] = { - val parts = topicsMetadata.flatMap(_.partitionsMetadata) - 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 543e262b25a94..d59de82178a0a 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -14,18 +14,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.api +package kafka.api import java.nio.ByteBuffer + import kafka.api.ApiUtils._ -import kafka.cluster.Broker -import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.cluster.{Broker, BrokerEndPoint} +import kafka.common.{ErrorMapping, KafkaException, TopicAndPartition} +import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response -import collection.Set +import org.apache.kafka.common.protocol.SecurityProtocol + +import scala.collection.Set object UpdateMetadataRequest { - val CurrentVersion = 0.shortValue + val CurrentVersion = 1.shortValue val IsInit: Boolean = true val NotInit: Boolean = false val DefaultAckTimeout: Int = 1000 @@ -48,20 +51,26 @@ object UpdateMetadataRequest { } val numAliveBrokers = buffer.getInt - val aliveBrokers = for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer) + + val aliveBrokers = versionId match { + case 0 => for(i <- 0 until numAliveBrokers) yield new Broker(BrokerEndPoint.readFrom(buffer),SecurityProtocol.PLAINTEXT) + case 1 => for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer) + case v => throw new KafkaException( "Version " + v.toString + " is invalid for UpdateMetadataRequest. Valid versions are 0 or 1.") + } + new UpdateMetadataRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos.toMap, aliveBrokers.toSet) } } case class UpdateMetadataRequest (versionId: Short, - override val correlationId: Int, + correlationId: Int, clientId: String, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker]) - extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey), correlationId) { + extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey)) { def this(controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String, partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker]) = { @@ -82,7 +91,12 @@ case class UpdateMetadataRequest (versionId: Short, value.writeTo(buffer) } buffer.putInt(aliveBrokers.size) - aliveBrokers.foreach(_.writeTo(buffer)) + + versionId match { + case 0 => aliveBrokers.foreach(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).writeTo(buffer)) + case 1 => aliveBrokers.foreach(_.writeTo(buffer)) + case v => throw new KafkaException( "Version " + v.toString + " is invalid for UpdateMetadataRequest. Valid versions are 0 or 1.") + } } def sizeInBytes(): Int = { @@ -96,8 +110,15 @@ case class UpdateMetadataRequest (versionId: Short, for((key, value) <- partitionStateInfos) size += (2 + key.topic.length) /* topic */ + 4 /* partition */ + value.sizeInBytes /* partition state info */ size += 4 /* number of alive brokers in the cluster */ - for(broker <- aliveBrokers) - size += broker.sizeInBytes /* broker info */ + + versionId match { + case 0 => for(broker <- aliveBrokers) + size += broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).sizeInBytes /* broker info */ + case 1 => for(broker <- aliveBrokers) + size += broker.sizeInBytes + case v => throw new KafkaException( "Version " + v.toString + " is invalid for UpdateMetadataRequest. Valid versions are 0 or 1.") + } + size } @@ -107,7 +128,7 @@ case class UpdateMetadataRequest (versionId: Short, 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))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala index c583c1f00c89a..53f606752055a 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala @@ -32,9 +32,9 @@ object UpdateMetadataResponse { } } -case class UpdateMetadataResponse(override val correlationId: Int, +case class UpdateMetadataResponse(correlationId: Int, errorCode: Short = ErrorMapping.NoError) - extends RequestOrResponse(correlationId = correlationId) { + extends RequestOrResponse() { def sizeInBytes(): Int = 4 /* correlation id */ + 2 /* error code */ def writeTo(buffer: ByteBuffer) { diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala old mode 100644 new mode 100755 index 1d2f81be4f980..68c7e7f594f2f --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -14,18 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.client +package kafka.client + +import org.apache.kafka.common.protocol.SecurityProtocol import scala.collection._ import kafka.cluster._ import kafka.api._ import kafka.producer._ -import kafka.common.KafkaException -import kafka.utils.{Utils, Logging} +import kafka.common.{ErrorMapping, KafkaException} +import kafka.utils.{CoreUtils, Logging} import java.util.Properties import util.Random +import kafka.network.BlockingChannel +import kafka.utils.ZkUtils._ +import org.I0Itec.zkclient.ZkClient +import java.io.IOException -/** + /** * Helper functions common to clients (producer, consumer, or admin) */ object ClientUtils extends Logging{ @@ -37,7 +43,7 @@ object ClientUtils extends Logging{ * @param producerConfig The producer's config * @return topic metadata response */ - def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], producerConfig: ProducerConfig, correlationId: Int): TopicMetadataResponse = { + def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndPoint], producerConfig: ProducerConfig, correlationId: Int): TopicMetadataResponse = { var fetchMetaDataSucceeded: Boolean = false var i: Int = 0 val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, producerConfig.clientId, topics.toSeq) @@ -68,7 +74,7 @@ object ClientUtils extends Logging{ } else { debug("Successfully fetched metadata for %d topic(s) %s".format(topics.size, topics)) } - return topicMetadataResponse + topicMetadataResponse } /** @@ -78,10 +84,10 @@ object ClientUtils extends Logging{ * @param clientId The client's identifier * @return topic metadata response */ - def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String, timeoutMs: Int, + def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndPoint], clientId: String, timeoutMs: Int, correlationId: Int = 0): TopicMetadataResponse = { val props = new Properties() - props.put("metadata.broker.list", brokers.map(_.getConnectionString()).mkString(",")) + props.put("metadata.broker.list", brokers.map(_.connectionString).mkString(",")) props.put("client.id", clientId) props.put("request.timeout.ms", timeoutMs.toString) val producerConfig = new ProducerConfig(props) @@ -91,17 +97,105 @@ object ClientUtils extends Logging{ /** * Parse a list of broker urls in the form host1:port1, host2:port2, ... */ - def parseBrokerList(brokerListStr: String): Seq[Broker] = { - val brokersStr = Utils.parseCsvList(brokerListStr) - - brokersStr.zipWithIndex.map(b =>{ - val brokerStr = b._1 - val brokerId = b._2 - val brokerInfos = brokerStr.split(":") - val hostName = brokerInfos(0) - val port = brokerInfos(1).toInt - new Broker(brokerId, hostName, port) - }) + def parseBrokerList(brokerListStr: String): Seq[BrokerEndPoint] = { + val brokersStr = CoreUtils.parseCsvList(brokerListStr) + + brokersStr.zipWithIndex.map { case (address, brokerId) => + BrokerEndPoint.createBrokerEndPoint(brokerId, address) + } } - -} \ No newline at end of file + + /** + * Creates a blocking channel to a random broker + */ + def channelToAnyBroker(zkClient: ZkClient, socketTimeoutMs: Int = 3000) : BlockingChannel = { + var channel: BlockingChannel = null + var connected = false + while (!connected) { + val allBrokers = getAllBrokerEndPointsForChannel(zkClient, SecurityProtocol.PLAINTEXT) + Random.shuffle(allBrokers).find { broker => + trace("Connecting to broker %s:%d.".format(broker.host, broker.port)) + try { + channel = new BlockingChannel(broker.host, broker.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, socketTimeoutMs) + channel.connect() + debug("Created channel to broker %s:%d.".format(channel.host, channel.port)) + true + } catch { + case e: Exception => + if (channel != null) channel.disconnect() + channel = null + info("Error while creating channel to %s:%d.".format(broker.host, broker.port)) + false + } + } + connected = if (channel == null) false else true + } + + channel + } + + /** + * Creates a blocking channel to the offset manager of the given group + */ + def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = { + var queryChannel = channelToAnyBroker(zkClient) + + var offsetManagerChannelOpt: Option[BlockingChannel] = None + + while (!offsetManagerChannelOpt.isDefined) { + + var coordinatorOpt: Option[BrokerEndPoint] = None + + while (!coordinatorOpt.isDefined) { + try { + if (!queryChannel.isConnected) + queryChannel = channelToAnyBroker(zkClient) + debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group)) + queryChannel.send(ConsumerMetadataRequest(group)) + val response = queryChannel.receive() + val consumerMetadataResponse = ConsumerMetadataResponse.readFrom(response.payload()) + debug("Consumer metadata response: " + consumerMetadataResponse.toString) + if (consumerMetadataResponse.errorCode == ErrorMapping.NoError) + coordinatorOpt = consumerMetadataResponse.coordinatorOpt + else { + debug("Query to %s:%d to locate offset manager for %s failed - will retry in %d milliseconds." + .format(queryChannel.host, queryChannel.port, group, retryBackOffMs)) + Thread.sleep(retryBackOffMs) + } + } + catch { + case ioe: IOException => + info("Failed to fetch consumer metadata from %s:%d.".format(queryChannel.host, queryChannel.port)) + queryChannel.disconnect() + } + } + + val coordinator = coordinatorOpt.get + if (coordinator.host == queryChannel.host && coordinator.port == queryChannel.port) { + offsetManagerChannelOpt = Some(queryChannel) + } else { + val connectString = "%s:%d".format(coordinator.host, coordinator.port) + var offsetManagerChannel: BlockingChannel = null + try { + debug("Connecting to offset manager %s.".format(connectString)) + offsetManagerChannel = new BlockingChannel(coordinator.host, coordinator.port, + BlockingChannel.UseDefaultBufferSize, + BlockingChannel.UseDefaultBufferSize, + socketTimeoutMs) + offsetManagerChannel.connect() + offsetManagerChannelOpt = Some(offsetManagerChannel) + queryChannel.disconnect() + } + catch { + case ioe: IOException => // offsets manager may have moved + info("Error while connecting to %s.".format(connectString)) + if (offsetManagerChannel != null) offsetManagerChannel.disconnect() + Thread.sleep(retryBackOffMs) + offsetManagerChannelOpt = None // just in case someone decides to change shutdownChannel to not swallow exceptions + } + } + } + + offsetManagerChannelOpt.get + } + } diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala old mode 100644 new mode 100755 index 9407ed21fbbd5..79e16c167f67c --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -17,17 +17,44 @@ package kafka.cluster -import kafka.utils.Utils._ +import kafka.utils.CoreUtils._ import kafka.utils.Json import kafka.api.ApiUtils._ import java.nio.ByteBuffer -import kafka.common.{KafkaException, BrokerNotAvailableException} + +import kafka.common.{BrokerEndPointNotAvailableException, BrokerNotAvailableException, KafkaException} +import kafka.utils.Json +import org.apache.kafka.common.protocol.SecurityProtocol /** - * A Kafka broker + * A Kafka broker. + * A broker has an id and a collection of end-points. + * Each end-point is (host, port,protocolType). + * Currently the only protocol type is PlainText but we will add SSL and Kerberos in the future. */ -private[kafka] object Broker { +object Broker { + /** + * Create a broker object from id and JSON string. + * @param id + * @param brokerInfoString + * + * Version 1 JSON schema for a broker is: + * {"version":1, + * "host":"localhost", + * "port":9092 + * "jmx_port":9999, + * "timestamp":"2233345666" } + * + * The current JSON schema for a broker is: + * {"version":2, + * "host","localhost", + * "port",9092 + * "jmx_port":9999, + * "timestamp":"2233345666", + * "endpoints": ["PLAINTEXT://host1:9092", + * "SSL://host1:9093"] + */ def createBroker(id: Int, brokerInfoString: String): Broker = { if(brokerInfoString == null) throw new BrokerNotAvailableException("Broker id %s does not exist".format(id)) @@ -35,9 +62,21 @@ private[kafka] object Broker { Json.parseFull(brokerInfoString) match { case Some(m) => 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 version = brokerInfo("version").asInstanceOf[Int] + val endpoints = version match { + case 1 => + val host = brokerInfo("host").asInstanceOf[String] + val port = brokerInfo("port").asInstanceOf[Int] + Map(SecurityProtocol.PLAINTEXT -> new EndPoint(host, port, SecurityProtocol.PLAINTEXT)) + case 2 => + val listeners = brokerInfo("endpoints").asInstanceOf[List[String]] + listeners.map(listener => { + val ep = EndPoint.createEndPoint(listener) + (ep.protocolType, ep) + }).toMap + case _ => throw new KafkaException("Unknown version of broker registration. Only versions 1 and 2 are supported." + brokerInfoString) + } + new Broker(id, endpoints) case None => throw new BrokerNotAvailableException("Broker id %d does not exist".format(id)) } @@ -46,36 +85,60 @@ private[kafka] object Broker { } } + /** + * + * @param buffer Containing serialized broker. + * Current serialization is: + * id (int), number of endpoints (int), serialized endpoints + * @return broker object + */ def readFrom(buffer: ByteBuffer): Broker = { val id = buffer.getInt - val host = readShortString(buffer) - val port = buffer.getInt - new Broker(id, host, port) + val numEndpoints = buffer.getInt + + val endpoints = List.range(0, numEndpoints).map(i => EndPoint.readFrom(buffer)) + .map(ep => ep.protocolType -> ep).toMap + new Broker(id, endpoints) } } -private[kafka] case class Broker(val id: Int, val host: String, val port: Int) { - - override def toString(): String = new String("id:" + id + ",host:" + host + ",port:" + port) +case class Broker(id: Int, endPoints: Map[SecurityProtocol, EndPoint]) { + + override def toString: String = id + " : " + endPoints.values.mkString("(",",",")") + + def this(id: Int, host: String, port: Int, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { + this(id, Map(protocol -> EndPoint(host, port, protocol))) + } + + def this(bep: BrokerEndPoint, protocol: SecurityProtocol) = { + this(bep.id, bep.host, bep.port, protocol) + } - def getConnectionString(): String = host + ":" + port def writeTo(buffer: ByteBuffer) { buffer.putInt(id) - writeShortString(buffer, host) - buffer.putInt(port) + buffer.putInt(endPoints.size) + for(endpoint <- endPoints.values) { + endpoint.writeTo(buffer) + } } - def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ + def sizeInBytes: Int = + 4 + /* broker id*/ + 4 + /* number of endPoints */ + endPoints.values.map(_.sizeInBytes).sum /* end points */ - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case n: Broker => id == n.id && host == n.host && port == n.port - case _ => false + def supportsChannel(protocolType: SecurityProtocol): Unit = { + endPoints.contains(protocolType) + } + + def getBrokerEndPoint(protocolType: SecurityProtocol): BrokerEndPoint = { + val endpoint = endPoints.get(protocolType) + endpoint match { + case Some(endpoint) => new BrokerEndPoint(id, endpoint.host, endpoint.port) + case None => + throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id)) } } - - override def hashCode(): Int = hashcode(id, host, port) - + } diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala new file mode 100644 index 0000000000000..3395108b74fe9 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -0,0 +1,67 @@ +/** + * 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.cluster + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.KafkaException +import org.apache.kafka.common.utils.Utils._ + +object BrokerEndPoint { + def createBrokerEndPoint(brokerId: Int, connectionString: String): BrokerEndPoint = { + + // BrokerEndPoint URI is host:port or [ipv6_host]:port + // Note that unlike EndPoint (or listener) this URI has no security information. + val uriParseExp = """\[?([0-9a-z\-.:]*)\]?:([0-9]+)""".r + + connectionString match { + case uriParseExp(host, port) => new BrokerEndPoint(brokerId, host, port.toInt) + case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") + } + } + + def readFrom(buffer: ByteBuffer): BrokerEndPoint = { + val brokerId = buffer.getInt() + val host = readShortString(buffer) + val port = buffer.getInt() + BrokerEndPoint(brokerId, host, port) + } +} + +/** + * BrokerEndpoint is used to connect to specific host:port pair. + * It is typically used by clients (or brokers when connecting to other brokers) + * and contains no information about the security protocol used on the connection. + * Clients should know which security protocol to use from configuration. + * This allows us to keep the wire protocol with the clients unchanged where the protocol is not needed. + */ +case class BrokerEndPoint(id: Int, host: String, port: Int) { + + def connectionString(): String = formatAddress(host, port) + + def writeTo(buffer: ByteBuffer): Unit = { + buffer.putInt(id) + writeShortString(buffer, host) + buffer.putInt(port) + } + + def sizeInBytes: Int = + 4 + /* broker Id */ + 4 + /* port */ + shortStringLength(host) +} diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala new file mode 100644 index 0000000000000..76997b5a272c8 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -0,0 +1,78 @@ +/* + * 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.cluster + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.KafkaException +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.utils.Utils + +object EndPoint { + + def readFrom(buffer: ByteBuffer): EndPoint = { + val port = buffer.getInt() + val host = readShortString(buffer) + val protocol = buffer.getShort() + EndPoint(host, port, SecurityProtocol.forId(protocol)) + } + + /** + * Create EndPoint object from connectionString + * @param connectionString the format is protocol://host:port or protocol://[ipv6 host]:port + * for example: PLAINTEXT://myhost:9092 or PLAINTEXT://[::1]:9092 + * Host can be empty (PLAINTEXT://:9092) in which case we'll bind to default interface + * Negative ports are also accepted, since they are used in some unit tests + * @return + */ + def createEndPoint(connectionString: String): EndPoint = { + val uriParseExp = """^(.*)://\[?([0-9a-zA-Z\-.:]*)\]?:(-?[0-9]+)""".r + connectionString match { + case uriParseExp(protocol, "", port) => new EndPoint(null, port.toInt, SecurityProtocol.valueOf(protocol)) + case uriParseExp(protocol, host, port) => new EndPoint(host, port.toInt, SecurityProtocol.valueOf(protocol)) + case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") + } + } +} + +/** + * Part of the broker definition - matching host/port pair to a protocol + */ +case class EndPoint(host: String, port: Int, protocolType: SecurityProtocol) { + + def connectionString(): String = { + val hostport = + if (host == null) + ":"+port + else + Utils.formatAddress(host, port) + protocolType + "://" + hostport + } + + def writeTo(buffer: ByteBuffer): Unit = { + buffer.putInt(port) + writeShortString(buffer, host) + buffer.putShort(protocolType.id) + } + + def sizeInBytes: Int = + 4 + /* port */ + shortStringLength(host) + + 2 /* protocol id */ +} diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala old mode 100644 new mode 100755 index 1087a2e91c86e..2649090b6cbf8 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -16,39 +16,41 @@ */ package kafka.cluster -import scala.collection._ -import kafka.admin.AdminUtils +import kafka.common._ import kafka.utils._ -import java.lang.Object +import kafka.utils.CoreUtils.{inReadLock,inWriteLock} +import kafka.admin.AdminUtils import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig -import kafka.server.ReplicaManager -import com.yammer.metrics.core.Gauge +import kafka.server.{TopicPartitionOperationKey, LogOffsetMetadata, LogReadResult, ReplicaManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController -import org.apache.log4j.Logger import kafka.message.ByteBufferMessageSet -import kafka.common.{NotAssignedReplicaException, TopicAndPartition, NotLeaderForPartitionException, ErrorMapping} + import java.io.IOException +import java.util.concurrent.locks.ReentrantReadWriteLock +import scala.collection.immutable.Set +import com.yammer.metrics.core.Gauge /** * Data structure that represents a topic partition. The leader maintains the AR, ISR, CUR, RAR */ class Partition(val topic: String, val partitionId: Int, - var replicationFactor: Int, time: Time, - val replicaManager: ReplicaManager) extends Logging with KafkaMetricsGroup { + replicaManager: ReplicaManager) extends Logging with KafkaMetricsGroup { private val localBrokerId = replicaManager.config.brokerId private val logManager = replicaManager.logManager private val zkClient = replicaManager.zkClient - var leaderReplicaIdOpt: Option[Int] = None - var inSyncReplicas: Set[Replica] = Set.empty[Replica] - private val assignedReplicaMap = new Pool[Int,Replica] - private val leaderIsrUpdateLock = new Object + private val assignedReplicaMap = new Pool[Int, Replica] + // The read lock is only required when multiple reads are executed and needs to be in a consistent manner + private val leaderIsrUpdateLock = new ReentrantReadWriteLock() private var zkVersion: Int = LeaderAndIsr.initialZKVersion - private var leaderEpoch: Int = LeaderAndIsr.initialLeaderEpoch - 1 + @volatile private var leaderEpoch: Int = LeaderAndIsr.initialLeaderEpoch - 1 + @volatile var leaderReplicaIdOpt: Option[Int] = None + @volatile var inSyncReplicas: Set[Replica] = Set.empty[Replica] + /* Epoch of the controller that last changed the leader. This needs to be initialized correctly upon broker startup. * One way of doing that is through the controller's start replica state change command. When a new broker starts up * the controller sends it a start replica command containing the leader for each partition that the broker hosts. @@ -56,27 +58,25 @@ class Partition(val topic: String, * each partition. */ private var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1 this.logIdent = "Partition [%s,%d] on broker %d: ".format(topic, partitionId, localBrokerId) - private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) private def isReplicaLocal(replicaId: Int) : Boolean = (replicaId == localBrokerId) + val tags = Map("topic" -> topic, "partition" -> partitionId.toString) - newGauge( - topic + "-" + partitionId + "-UnderReplicated", + newGauge("UnderReplicated", new Gauge[Int] { def value = { if (isUnderReplicated) 1 else 0 } - } + }, + tags ) def isUnderReplicated(): Boolean = { - leaderIsrUpdateLock synchronized { - leaderReplicaIfLocal() match { - case Some(_) => - inSyncReplicas.size < assignedReplicas.size - case None => - false - } + leaderReplicaIfLocal() match { + case Some(_) => + inSyncReplicas.size < assignedReplicas.size + case None => + false } } @@ -86,12 +86,12 @@ class Partition(val topic: String, case Some(replica) => replica case None => if (isReplicaLocal(replicaId)) { - val config = LogConfig.fromProps(logManager.defaultConfig.toProps, AdminUtils.fetchTopicConfig(zkClient, topic)) + val config = LogConfig.fromProps(logManager.defaultConfig.originals, AdminUtils.fetchTopicConfig(zkClient, topic)) val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) - val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParent) + val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) val offsetMap = checkpoint.read if (!offsetMap.contains(TopicAndPartition(topic, partitionId))) - warn("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId)) + info("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) @@ -112,15 +112,13 @@ class Partition(val topic: String, } def leaderReplicaIfLocal(): Option[Replica] = { - leaderIsrUpdateLock synchronized { - leaderReplicaIdOpt match { - case Some(leaderReplicaId) => - if (leaderReplicaId == localBrokerId) - getReplica(localBrokerId) - else - None - case None => None - } + leaderReplicaIdOpt match { + case Some(leaderReplicaId) => + if (leaderReplicaId == localBrokerId) + getReplica(localBrokerId) + else + None + case None => None } } @@ -138,12 +136,13 @@ class Partition(val topic: String, def delete() { // need to hold the lock to prevent appendMessagesToLeader() from hitting I/O exceptions due to log being deleted - leaderIsrUpdateLock synchronized { + inWriteLock(leaderIsrUpdateLock) { assignedReplicaMap.clear() inSyncReplicas = Set.empty[Replica] leaderReplicaIdOpt = None try { logManager.deleteLog(TopicAndPartition(topic, partitionId)) + removePartitionMetrics() } catch { case e: IOException => fatal("Error deleting the log for partition [%s,%d]".format(topic, partitionId), e) @@ -153,9 +152,7 @@ class Partition(val topic: String, } def getLeaderEpoch(): Int = { - leaderIsrUpdateLock synchronized { - return this.leaderEpoch - } + return this.leaderEpoch } /** @@ -164,7 +161,7 @@ class Partition(val topic: String, */ def makeLeader(controllerId: Int, partitionStateInfo: PartitionStateInfo, correlationId: Int): Boolean = { - leaderIsrUpdateLock synchronized { + inWriteLock(leaderIsrUpdateLock) { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr @@ -176,25 +173,30 @@ class Partition(val topic: String, 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 leaderEpoch = leaderAndIsr.leaderEpoch zkVersion = leaderAndIsr.zkVersion leaderReplicaIdOpt = Some(localBrokerId) + // construct the high watermark metadata for the new leader replica + val newLeaderReplica = getReplica().get + newLeaderReplica.convertHWToLocalOffsetMetadata() + // reset log end offset for remote replicas + assignedReplicas.foreach(r => + if (r.brokerId != localBrokerId) r.updateLogReadResult(LogReadResult.UnknownLogReadResult)) // we may need to increment high watermark since ISR could be down to 1 - maybeIncrementLeaderHW(getReplica().get) + maybeIncrementLeaderHW(newLeaderReplica) true } } /** * Make the local replica the follower by setting the new leader and ISR to empty + * If the leader replica id does not change, return false to indicate the replica manager */ def makeFollower(controllerId: Int, partitionStateInfo: PartitionStateInfo, - leaders: Set[Broker], correlationId: Int): Boolean = { - leaderIsrUpdateLock synchronized { + correlationId: Int): Boolean = { + inWriteLock(leaderIsrUpdateLock) { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr @@ -202,112 +204,159 @@ class Partition(val topic: String, // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch - // TODO: Delete leaders from LeaderAndIsrRequest in 0.8.1 - leaders.find(_.id == newLeaderBrokerId) match { - case Some(leaderBroker) => - // 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) - case None => // we should not come here - stateChangeLogger.error(("Broker %d aborted the become-follower state change with correlation id %d from " + - "controller %d epoch %d for partition [%s,%d] new leader %d") - .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, - topic, partitionId, newLeaderBrokerId)) + // 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 + + if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) { + false + } + else { + leaderReplicaIdOpt = Some(newLeaderBrokerId) + true } - true } } - def updateLeaderHWAndMaybeExpandIsr(replicaId: Int, offset: Long) { - leaderIsrUpdateLock synchronized { - debug("Recording follower %d position %d for partition [%s,%d].".format(replicaId, offset, topic, partitionId)) - 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 + /** + * Update the log end offset of a certain replica of this partition + */ + def updateReplicaLogReadResult(replicaId: Int, logReadResult: LogReadResult) { + getReplica(replicaId) match { + case Some(replica) => + replica.updateLogReadResult(logReadResult) + // check if we need to expand ISR to include this replica + // if it is not in the ISR yet + maybeExpandIsr(replicaId) + + debug("Recorded replica %d log end offset (LEO) position %d for partition %s." + .format(replicaId, + logReadResult.info.fetchOffsetMetadata.messageOffset, + TopicAndPartition(topic, partitionId))) + case None => + throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" + + " is not recognized to be one of the assigned replicas %s for partition %s.") + .format(localBrokerId, + replicaId, + logReadResult.info.fetchOffsetMetadata.messageOffset, + assignedReplicas().map(_.brokerId).mkString(","), + TopicAndPartition(topic, partitionId))) + } + } + /** + * Check and maybe expand the ISR of the partition. + * + * This function can be triggered when a replica's LEO has incremented + */ + def maybeExpandIsr(replicaId: Int) { + inWriteLock(leaderIsrUpdateLock) { // check if this replica needs to be added to the ISR leaderReplicaIfLocal() match { case Some(leaderReplica) => val replica = getReplica(replicaId).get val leaderHW = leaderReplica.highWatermark - // 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 + if(!inSyncReplicas.contains(replica) && + assignedReplicas.map(_.brokerId).contains(replicaId) && + replica.logEndOffset.offsetDiff(leaderHW) >= 0) { val newInSyncReplicas = inSyncReplicas + replica info("Expanding ISR for partition [%s,%d] from %s to %s" - .format(topic, partitionId, inSyncReplicas.map(_.brokerId).mkString(","), newInSyncReplicas.map(_.brokerId).mkString(","))) + .format(topic, partitionId, inSyncReplicas.map(_.brokerId).mkString(","), + newInSyncReplicas.map(_.brokerId).mkString(","))) // update ISR in ZK and cache updateIsr(newInSyncReplicas) replicaManager.isrExpandRate.mark() } + + // check if the HW of the partition can now be incremented + // since the replica maybe now be in the ISR and its LEO has just incremented maybeIncrementLeaderHW(leaderReplica) + case None => // nothing to do if no longer leader } } } - def checkEnoughReplicasReachOffset(requiredOffset: Long, requiredAcks: Int): (Boolean, Short) = { - leaderIsrUpdateLock synchronized { - leaderReplicaIfLocal() match { - case Some(_) => - val numAcks = inSyncReplicas.count(r => { - if (!r.isLocal) - r.logEndOffset >= requiredOffset + /* + * Note that this method will only be called if requiredAcks = -1 + * and we are waiting for all replicas in ISR to be fully caught up to + * the (local) leader's offset corresponding to this produce request + * before we acknowledge the produce request. + */ + def checkEnoughReplicasReachOffset(requiredOffset: Long): (Boolean, Short) = { + leaderReplicaIfLocal() match { + case Some(leaderReplica) => + // keep the current immutable replica list reference + val curInSyncReplicas = inSyncReplicas + val numAcks = curInSyncReplicas.count(r => { + if (!r.isLocal) + if (r.logEndOffset.messageOffset >= requiredOffset) { + trace("Replica %d of %s-%d received offset %d".format(r.brokerId, topic, partitionId, requiredOffset)) + true + } else - true /* also count the local (leader) replica */ - }) - trace("%d/%d acks satisfied for %s-%d".format(numAcks, requiredAcks, topic, partitionId)) - if ((requiredAcks < 0 && numAcks >= inSyncReplicas.size) || - (requiredAcks > 0 && numAcks >= requiredAcks)) { - /* - * requiredAcks < 0 means acknowledge after all replicas in ISR - * are fully caught up to the (local) leader's offset - * corresponding to this produce request. - */ + false + else + true /* also count the local (leader) replica */ + }) + + trace("%d acks satisfied for %s-%d with acks = -1".format(numAcks, topic, partitionId)) + + val minIsr = leaderReplica.log.get.config.minInSyncReplicas + + if (leaderReplica.highWatermark.messageOffset >= requiredOffset ) { + /* + * The topic may be configured not to accept messages if there are not enough replicas in ISR + * in this scenario the request was already appended locally and then added to the purgatory before the ISR was shrunk + */ + if (minIsr <= curInSyncReplicas.size) { (true, ErrorMapping.NoError) - } else - (false, ErrorMapping.NoError) - case None => - (false, ErrorMapping.NotLeaderForPartitionCode) - } + } else { + (true, ErrorMapping.NotEnoughReplicasAfterAppendCode) + } + } else + (false, ErrorMapping.NoError) + case None => + (false, ErrorMapping.NotLeaderForPartitionCode) } } /** - * There is no need to acquire the leaderIsrUpdate lock here since all callers of this private API acquire that lock - * @param leaderReplica + * Check and maybe increment the high watermark of the partition; + * this function can be triggered when + * + * 1. Partition ISR changed + * 2. Any replica's LEO changed + * + * Note There is no need to acquire the leaderIsrUpdate lock here + * since all callers of this private API acquire that lock */ private def maybeIncrementLeaderHW(leaderReplica: Replica) { val allLogEndOffsets = inSyncReplicas.map(_.logEndOffset) - val newHighWatermark = allLogEndOffsets.min + val newHighWatermark = allLogEndOffsets.min(new LogOffsetMetadata.OffsetOrdering) val oldHighWatermark = leaderReplica.highWatermark - if(newHighWatermark > oldHighWatermark) { + if(oldHighWatermark.precedes(newHighWatermark)) { leaderReplica.highWatermark = newHighWatermark - debug("Highwatermark for partition [%s,%d] updated to %d".format(topic, partitionId, newHighWatermark)) + debug("High watermark for partition [%s,%d] updated to %s".format(topic, partitionId, newHighWatermark)) + // some delayed operations may be unblocked after HW changed + val requestKey = new TopicPartitionOperationKey(this.topic, this.partitionId) + replicaManager.tryCompleteDelayedFetch(requestKey) + replicaManager.tryCompleteDelayedProduce(requestKey) + } else { + debug("Skipping update high watermark since Old hw %s is larger than new hw %s for partition [%s,%d]. All leo's are %s" + .format(oldHighWatermark, newHighWatermark, topic, partitionId, allLogEndOffsets.mkString(","))) } - else - debug("Old hw for partition [%s,%d] is %d. New hw is %d. All leo's are %s" - .format(topic, partitionId, oldHighWatermark, newHighWatermark, allLogEndOffsets.mkString(","))) } - def maybeShrinkIsr(replicaMaxLagTimeMs: Long, replicaMaxLagMessages: Long) { - leaderIsrUpdateLock synchronized { + def maybeShrinkIsr(replicaMaxLagTimeMs: Long) { + inWriteLock(leaderIsrUpdateLock) { leaderReplicaIfLocal() match { case Some(leaderReplica) => - val outOfSyncReplicas = getOutOfSyncReplicas(leaderReplica, replicaMaxLagTimeMs, replicaMaxLagMessages) + val outOfSyncReplicas = getOutOfSyncReplicas(leaderReplica, replicaMaxLagTimeMs) if(outOfSyncReplicas.size > 0) { val newInSyncReplicas = inSyncReplicas -- outOfSyncReplicas assert(newInSyncReplicas.size > 0) @@ -324,34 +373,46 @@ class Partition(val topic: String, } } - def getOutOfSyncReplicas(leaderReplica: Replica, keepInSyncTimeMs: Long, keepInSyncMessages: Long): Set[Replica] = { + def getOutOfSyncReplicas(leaderReplica: Replica, maxLagMs: Long): Set[Replica] = { /** - * there are two cases that need to be handled here - - * 1. Stuck followers: If the leo of the replica hasn't been updated for keepInSyncTimeMs ms, + * there are two cases that will be handled here - + * 1. Stuck followers: If the leo of the replica hasn't been updated for maxLagMs 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 + * 2. Slow followers: If the replica has not read up to the leo within the last maxLagMs ms, + * then the follower is lagging and should be removed from the ISR + * Both these cases are handled by checking the lastCaughtUpTimeMs which represents + * the last time when the replica was fully caught up. If either of the above conditions + * is violated, that replica is considered to be out of sync + * **/ val leaderLogEndOffset = leaderReplica.logEndOffset val candidateReplicas = inSyncReplicas - leaderReplica - // Case 1 above - 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 - val slowReplicas = candidateReplicas.filter(r => r.logEndOffset >= 0 && (leaderLogEndOffset - r.logEndOffset) > keepInSyncMessages) - if(slowReplicas.size > 0) - debug("Slow replicas for partition [%s,%d] are %s".format(topic, partitionId, slowReplicas.map(_.brokerId).mkString(","))) - stuckReplicas ++ slowReplicas + + val laggingReplicas = candidateReplicas.filter(r => (time.milliseconds - r.lastCaughtUpTimeMs) > maxLagMs) + if(laggingReplicas.size > 0) + debug("Lagging replicas for partition %s are %s".format(TopicAndPartition(topic, partitionId), laggingReplicas.map(_.brokerId).mkString(","))) + + laggingReplicas } - def appendMessagesToLeader(messages: ByteBufferMessageSet) = { - leaderIsrUpdateLock synchronized { + def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int = 0) = { + inReadLock(leaderIsrUpdateLock) { val leaderReplicaOpt = leaderReplicaIfLocal() leaderReplicaOpt match { case Some(leaderReplica) => val log = leaderReplica.log.get + val minIsr = log.config.minInSyncReplicas + val inSyncSize = inSyncReplicas.size + + // Avoid writing to leader if there are not enough insync replicas to make it safe + if (inSyncSize < minIsr && requiredAcks == -1) { + throw new NotEnoughReplicasException("Number of insync replicas for partition [%s,%d] is [%d], below required minimum [%d]" + .format(topic, partitionId, inSyncSize, minIsr)) + } + val info = log.append(messages, assignOffsets = true) + // probably unblock some follower fetch requests since log end offset has been updated + replicaManager.tryCompleteDelayedFetch(new TopicPartitionOperationKey(this.topic, this.partitionId)) // we may need to increment high watermark since ISR could be down to 1 maybeIncrementLeaderHW(leaderReplica) info @@ -363,13 +424,10 @@ class Partition(val topic: String, } private def updateIsr(newIsr: Set[Replica]) { - debug("Updated ISR for partition [%s,%d] to %s".format(topic, partitionId, newIsr.mkString(","))) val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion) - // use the epoch of the controller that made the leadership decision, instead of the current controller epoch - val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath(zkClient, - ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partitionId), - ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch), zkVersion) - if (updateSucceeded){ + val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partitionId, + newLeaderAndIsr, controllerEpoch, zkVersion) + if(updateSucceeded) { inSyncReplicas = newIsr zkVersion = newVersion trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion)) @@ -378,6 +436,13 @@ class Partition(val topic: String, } } + /** + * remove deleted log metrics + */ + private def removePartitionMetrics() { + removeMetric("UnderReplicated", tags) + } + override def equals(that: Any): Boolean = { if(!(that.isInstanceOf[Partition])) return false @@ -392,14 +457,12 @@ class Partition(val topic: String, } override def toString(): String = { - leaderIsrUpdateLock synchronized { - val partitionString = new StringBuilder - partitionString.append("Topic: " + topic) - partitionString.append("; Partition: " + partitionId) - partitionString.append("; Leader: " + leaderReplicaIdOpt) - partitionString.append("; AssignedReplicas: " + assignedReplicaMap.keys.mkString(",")) - partitionString.append("; InSyncReplicas: " + inSyncReplicas.map(_.brokerId).mkString(",")) - partitionString.toString() - } + val partitionString = new StringBuilder + partitionString.append("Topic: " + topic) + partitionString.append("; Partition: " + partitionId) + partitionString.append("; Leader: " + leaderReplicaIdOpt) + partitionString.append("; AssignedReplicas: " + assignedReplicaMap.keys.mkString(",")) + partitionString.append("; InSyncReplicas: " + inSyncReplicas.map(_.brokerId).mkString(",")) + partitionString.toString() } } diff --git a/core/src/main/scala/kafka/cluster/Replica.scala b/core/src/main/scala/kafka/cluster/Replica.scala index 5e659b4a5c025..740e83567c1a1 100644 --- a/core/src/main/scala/kafka/cluster/Replica.scala +++ b/core/src/main/scala/kafka/cluster/Replica.scala @@ -19,8 +19,9 @@ package kafka.cluster import kafka.log.Log import kafka.utils.{SystemTime, Time, Logging} +import kafka.server.{LogReadResult, LogOffsetMetadata} import kafka.common.KafkaException -import kafka.server.ReplicaManager + import java.util.concurrent.atomic.AtomicLong class Replica(val brokerId: Int, @@ -28,33 +29,15 @@ class Replica(val brokerId: Int, time: Time = SystemTime, initialHighWatermarkValue: Long = 0L, val log: Option[Log] = None) extends Logging { - //only defined in local replica - private[this] var highWatermarkValue: AtomicLong = new AtomicLong(initialHighWatermarkValue) - // only used for remote replica; logEndOffsetValue for local replica is kept in log - private[this] var logEndOffsetValue = new AtomicLong(ReplicaManager.UnknownLogEndOffset) - private[this] var logEndOffsetUpdateTimeMsValue: AtomicLong = new AtomicLong(time.milliseconds) + // the high watermark offset value, in non-leader replicas only its message offsets are kept + @volatile private[this] var highWatermarkMetadata: LogOffsetMetadata = new LogOffsetMetadata(initialHighWatermarkValue) + // the log end offset value, kept in all replicas; + // for local replica it is the log's end offset, for remote replicas its value is only updated by follower fetch + @volatile private[this] var logEndOffsetMetadata: LogOffsetMetadata = LogOffsetMetadata.UnknownOffsetMetadata + val topic = partition.topic val partitionId = partition.partitionId - def logEndOffset_=(newLogEndOffset: Long) { - if (!isLocal) { - logEndOffsetValue.set(newLogEndOffset) - logEndOffsetUpdateTimeMsValue.set(time.milliseconds) - trace("Setting log end offset for replica %d for partition [%s,%d] to %d" - .format(brokerId, topic, partitionId, logEndOffsetValue.get())) - } else - throw new KafkaException("Shouldn't set logEndOffset for replica %d partition [%s,%d] since it's local" - .format(brokerId, topic, partitionId)) - - } - - def logEndOffset = { - if (isLocal) - log.get.logEndOffset - else - logEndOffsetValue.get() - } - def isLocal: Boolean = { log match { case Some(l) => true @@ -62,24 +45,56 @@ class Replica(val brokerId: Int, } } - def logEndOffsetUpdateTimeMs = logEndOffsetUpdateTimeMsValue.get() + private[this] val lastCaughtUpTimeMsUnderlying = new AtomicLong(time.milliseconds) + + def lastCaughtUpTimeMs = lastCaughtUpTimeMsUnderlying.get() + + def updateLogReadResult(logReadResult : LogReadResult) { + logEndOffset = logReadResult.info.fetchOffsetMetadata + + /* If the request read up to the log end offset snapshot when the read was initiated, + * set the lastCaughtUpTimeMsUnderlying to the current time. + * This means that the replica is fully caught up. + */ + if(logReadResult.isReadFromLogEnd) { + lastCaughtUpTimeMsUnderlying.set(time.milliseconds) + } + } - def highWatermark_=(newHighWatermark: Long) { + private def logEndOffset_=(newLogEndOffset: LogOffsetMetadata) { if (isLocal) { - trace("Setting hw for replica %d partition [%s,%d] on broker %d to %d" - .format(brokerId, topic, partitionId, brokerId, newHighWatermark)) - highWatermarkValue.set(newHighWatermark) - } else - throw new KafkaException("Unable to set highwatermark for replica %d partition [%s,%d] since it's not local" - .format(brokerId, topic, partitionId)) + throw new KafkaException("Should not set log end offset on partition [%s,%d]'s local replica %d".format(topic, partitionId, brokerId)) + } else { + logEndOffsetMetadata = newLogEndOffset + trace("Setting log end offset for replica %d for partition [%s,%d] to [%s]" + .format(brokerId, topic, partitionId, logEndOffsetMetadata)) + } } - def highWatermark = { + def logEndOffset = if (isLocal) - highWatermarkValue.get() + log.get.logEndOffsetMetadata else - throw new KafkaException("Unable to get highwatermark for replica %d partition [%s,%d] since it's not local" - .format(brokerId, topic, partitionId)) + logEndOffsetMetadata + + def highWatermark_=(newHighWatermark: LogOffsetMetadata) { + if (isLocal) { + highWatermarkMetadata = newHighWatermark + trace("Setting high watermark for replica %d partition [%s,%d] on broker %d to [%s]" + .format(brokerId, topic, partitionId, brokerId, newHighWatermark)) + } else { + throw new KafkaException("Should not set high watermark on partition [%s,%d]'s non-local replica %d".format(topic, partitionId, brokerId)) + } + } + + def highWatermark = highWatermarkMetadata + + def convertHWToLocalOffsetMetadata() = { + if (isLocal) { + highWatermarkMetadata = log.get.convertToOffsetMetadata(highWatermarkMetadata.messageOffset) + } else { + throw new KafkaException("Should not construct complete high watermark on partition [%s,%d]'s non-local replica %d".format(topic, partitionId, brokerId)) + } } override def equals(that: Any): Boolean = { diff --git a/core/src/main/scala/kafka/common/AppInfo.scala b/core/src/main/scala/kafka/common/AppInfo.scala new file mode 100644 index 0000000000000..d642ca555f83c --- /dev/null +++ b/core/src/main/scala/kafka/common/AppInfo.scala @@ -0,0 +1,66 @@ +/** + * 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 java.net.URL +import java.util.jar.{Attributes, Manifest} + +import com.yammer.metrics.core.Gauge +import kafka.metrics.KafkaMetricsGroup + +object AppInfo extends KafkaMetricsGroup { + private var isRegistered = false + private val lock = new Object() + + def registerInfo(): Unit = { + lock.synchronized { + if (isRegistered) { + return + } + } + + try { + val clazz = AppInfo.getClass + val className = clazz.getSimpleName + ".class" + val classPath = clazz.getResource(className).toString + if (!classPath.startsWith("jar")) { + // Class not from JAR + return + } + val manifestPath = classPath.substring(0, classPath.lastIndexOf("!") + 1) + "/META-INF/MANIFEST.MF" + + val mf = new Manifest + mf.read(new URL(manifestPath).openStream()) + val version = mf.getMainAttributes.get(new Attributes.Name("Version")).toString + + newGauge("Version", + new Gauge[String] { + def value = { + version + } + }) + + lock.synchronized { + isRegistered = true + } + } catch { + case e: Exception => + warn("Can't read Kafka version from MANIFEST.MF. Possible cause: %s".format(e)) + } + } +} diff --git a/core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala b/core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala new file mode 100644 index 0000000000000..455d8c64b5a44 --- /dev/null +++ b/core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +class BrokerEndPointNotAvailableException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/common/ClientIdAndBroker.scala b/core/src/main/scala/kafka/common/ClientIdAndBroker.scala index 93223a9c93b55..3b09041d33ac4 100644 --- a/core/src/main/scala/kafka/common/ClientIdAndBroker.scala +++ b/core/src/main/scala/kafka/common/ClientIdAndBroker.scala @@ -8,7 +8,7 @@ package kafka.common * (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 + * 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, @@ -21,6 +21,14 @@ package kafka.common * Convenience case class since (clientId, brokerInfo) pairs are used to create * SyncProducer Request Stats and SimpleConsumer Request and Response Stats. */ -case class ClientIdAndBroker(clientId: String, brokerInfo: String) { - override def toString = "%s-%s".format(clientId, brokerInfo) + +trait ClientIdBroker { +} + +case class ClientIdAndBroker(clientId: String, brokerHost: String, brokerPort: Int) extends ClientIdBroker { + override def toString = "%s-%s-%d".format(clientId, brokerHost, brokerPort) +} + +case class ClientIdAllBrokers(clientId: String) extends ClientIdBroker { + override def toString = "%s-%s".format(clientId, "AllBrokers") } diff --git a/core/src/main/scala/kafka/common/ClientIdAndTopic.scala b/core/src/main/scala/kafka/common/ClientIdAndTopic.scala index 7acf9e76bdd87..5825aad2c8d1a 100644 --- a/core/src/main/scala/kafka/common/ClientIdAndTopic.scala +++ b/core/src/main/scala/kafka/common/ClientIdAndTopic.scala @@ -1,5 +1,3 @@ -package kafka.common - /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -17,11 +15,21 @@ package kafka.common * limitations under the License. */ +package kafka.common + /** * Convenience case class since (clientId, topic) pairs are used in the creation * of many Stats objects. */ -case class ClientIdAndTopic(clientId: String, topic: String) { +trait ClientIdTopic { +} + +case class ClientIdAndTopic(clientId: String, topic: String) extends ClientIdTopic { override def toString = "%s-%s".format(clientId, topic) } +case class ClientIdAllTopics(clientId: String) extends ClientIdTopic { + override def toString = "%s-%s".format(clientId, "AllTopics") +} + + diff --git a/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala b/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala new file mode 100644 index 0000000000000..8e02d264e9447 --- /dev/null +++ b/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +class ConsumerCoordinatorNotAvailableException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/ConsumerReblanceFailedException.scala b/core/src/main/scala/kafka/common/ConsumerRebalanceFailedException.scala similarity index 99% rename from core/src/main/scala/kafka/common/ConsumerReblanceFailedException.scala rename to core/src/main/scala/kafka/common/ConsumerRebalanceFailedException.scala index ae5018daa97bd..2f4c2cf70220a 100644 --- a/core/src/main/scala/kafka/common/ConsumerReblanceFailedException.scala +++ b/core/src/main/scala/kafka/common/ConsumerRebalanceFailedException.scala @@ -23,4 +23,4 @@ package kafka.common */ class ConsumerRebalanceFailedException(message: String) extends RuntimeException(message) { def this() = this(null) -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index b0b5dcedc4dad..c75c68589681b 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.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 @@ -19,11 +19,10 @@ package kafka.common import kafka.message.InvalidMessageException import java.nio.ByteBuffer -import java.lang.Throwable import scala.Predef._ /** - * A bi-directional mapping between error codes and exceptions x + * A bi-directional mapping between error codes and exceptions */ object ErrorMapping { val EmptyByteBuffer = ByteBuffer.allocate(0) @@ -43,8 +42,17 @@ object ErrorMapping { val StaleControllerEpochCode: Short = 11 val OffsetMetadataTooLargeCode: Short = 12 val StaleLeaderEpochCode: Short = 13 + val OffsetsLoadInProgressCode: Short = 14 + val ConsumerCoordinatorNotAvailableCode: Short = 15 + val NotCoordinatorForConsumerCode: Short = 16 + val InvalidTopicCode : Short = 17 + val MessageSetSizeTooLargeCode: Short = 18 + val NotEnoughReplicasCode : Short = 19 + val NotEnoughReplicasAfterAppendCode: Short = 20 + // 21: InvalidRequiredAcks + // 22: IllegalConsumerGeneration - private val exceptionToCode = + private val exceptionToCode = Map[Class[Throwable], Short]( classOf[OffsetOutOfRangeException].asInstanceOf[Class[Throwable]] -> OffsetOutOfRangeCode, classOf[InvalidMessageException].asInstanceOf[Class[Throwable]] -> InvalidMessageCode, @@ -57,18 +65,27 @@ object ErrorMapping { classOf[ReplicaNotAvailableException].asInstanceOf[Class[Throwable]] -> ReplicaNotAvailableCode, classOf[MessageSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSizeTooLargeCode, classOf[ControllerMovedException].asInstanceOf[Class[Throwable]] -> StaleControllerEpochCode, - classOf[OffsetMetadataTooLargeException].asInstanceOf[Class[Throwable]] -> OffsetMetadataTooLargeCode + classOf[OffsetMetadataTooLargeException].asInstanceOf[Class[Throwable]] -> OffsetMetadataTooLargeCode, + classOf[OffsetsLoadInProgressException].asInstanceOf[Class[Throwable]] -> OffsetsLoadInProgressCode, + classOf[ConsumerCoordinatorNotAvailableException].asInstanceOf[Class[Throwable]] -> ConsumerCoordinatorNotAvailableCode, + classOf[NotCoordinatorForConsumerException].asInstanceOf[Class[Throwable]] -> NotCoordinatorForConsumerCode, + classOf[InvalidTopicException].asInstanceOf[Class[Throwable]] -> InvalidTopicCode, + classOf[MessageSetSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSetSizeTooLargeCode, + classOf[NotEnoughReplicasException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasCode, + classOf[NotEnoughReplicasAfterAppendException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasAfterAppendCode ).withDefaultValue(UnknownCode) - + /* invert the mapping */ - private val codeToException = + private val codeToException = (Map[Short, Class[Throwable]]() ++ exceptionToCode.iterator.map(p => (p._2, p._1))).withDefaultValue(classOf[UnknownException]) - + def codeFor(exception: Class[Throwable]): Short = exceptionToCode(exception) - + def maybeThrowException(code: Short) = if(code != 0) throw codeToException(code).newInstance() def exceptionFor(code: Short) : Throwable = codeToException(code).newInstance() + + def exceptionNameFor(code: Short) : String = codeToException(code).getName() } diff --git a/core/src/main/scala/kafka/common/GenerateBrokerIdException.scala b/core/src/main/scala/kafka/common/GenerateBrokerIdException.scala new file mode 100644 index 0000000000000..13784fe50554e --- /dev/null +++ b/core/src/main/scala/kafka/common/GenerateBrokerIdException.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 + +/** + * Thrown when there is a failure to generate a zookeeper sequenceId to use as brokerId + */ +class GenerateBrokerIdException(message: String, cause: Throwable) extends RuntimeException(message, cause) { + def this(message: String) = this(message, null) + def this(cause: Throwable) = this(null, cause) + def this() = this(null, null) +} diff --git a/core/src/main/scala/kafka/common/InconsistentBrokerIdException.scala b/core/src/main/scala/kafka/common/InconsistentBrokerIdException.scala new file mode 100644 index 0000000000000..0c0d1cd731a4d --- /dev/null +++ b/core/src/main/scala/kafka/common/InconsistentBrokerIdException.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 brokerId stored in logDirs is not consistent across logDirs. + */ +class InconsistentBrokerIdException(message: String, cause: Throwable) extends RuntimeException(message, cause) { + def this(message: String) = this(message, null) + def this(cause: Throwable) = this(null, cause) + def this() = this(null, null) +} diff --git a/core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala b/core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala new file mode 100644 index 0000000000000..94a616ed3972a --- /dev/null +++ b/core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +class MessageSetSizeTooLargeException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/common/MessageStreamsExistException.scala b/core/src/main/scala/kafka/common/MessageStreamsExistException.scala new file mode 100644 index 0000000000000..68a2e079ea5b8 --- /dev/null +++ b/core/src/main/scala/kafka/common/MessageStreamsExistException.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.common + +/** + * Indicates a createMessageStreams can't be called more thane once +*/ +class MessageStreamsExistException(message: String, t: Throwable) extends RuntimeException(message, t) { +} diff --git a/core/src/main/scala/kafka/common/NoReplicaOnlineException.scala b/core/src/main/scala/kafka/common/NoReplicaOnlineException.scala index a1e12794978ad..b66c8fc82c15b 100644 --- a/core/src/main/scala/kafka/common/NoReplicaOnlineException.scala +++ b/core/src/main/scala/kafka/common/NoReplicaOnlineException.scala @@ -20,7 +20,8 @@ package kafka.common /** * This exception is thrown by the leader elector in the controller when leader election fails for a partition since - * all the replicas for a partition are offline + * all the leader candidate replicas for a partition are offline; the set of candidates may or may not be limited + * to just the in sync replicas depending upon whether unclean leader election is allowed to occur. */ class NoReplicaOnlineException(message: String, cause: Throwable) extends RuntimeException(message, cause) { def this(message: String) = this(message, null) diff --git a/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala b/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala new file mode 100644 index 0000000000000..1eb74be038eaa --- /dev/null +++ b/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +class NotCoordinatorForConsumerException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala b/core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala new file mode 100644 index 0000000000000..c4f9def6162e9 --- /dev/null +++ b/core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.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 + +/** + * Number of insync replicas for the partition is lower than min.insync.replicas + * This exception is raised when the low ISR size is discovered *after* the message + * was already appended to the log. Producer retries will cause duplicates. + */ +class NotEnoughReplicasAfterAppendException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/common/NotEnoughReplicasException.scala b/core/src/main/scala/kafka/common/NotEnoughReplicasException.scala new file mode 100644 index 0000000000000..bfbe0ee4a5a15 --- /dev/null +++ b/core/src/main/scala/kafka/common/NotEnoughReplicasException.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +/** + * Message was rejected because number of insync replicas for the partition is lower than min.insync.replicas + */ +class NotEnoughReplicasException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 59608a34202b4..deb48b1cee5e2 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -1,5 +1,3 @@ -package kafka.common - /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -17,20 +15,66 @@ package kafka.common * 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) { +package kafka.common + +import org.apache.kafka.common.protocol.Errors + +case class OffsetMetadata(offset: Long, metadata: String = OffsetMetadata.NoMetadata) { + override def toString = "OffsetMetadata[%d,%s]" + .format(offset, + if (metadata != null && metadata.length > 0) metadata else "NO_METADATA") +} + +object OffsetMetadata { + val InvalidOffset: Long = -1L + val NoMetadata: String = "" + + val InvalidOffsetMetadata = OffsetMetadata(OffsetMetadata.InvalidOffset, OffsetMetadata.NoMetadata) +} + +case class OffsetAndMetadata(offsetMetadata: OffsetMetadata, + commitTimestamp: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP, + expireTimestamp: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) { + + def offset = offsetMetadata.offset - def this(tuple: (Long, String, Short)) = this(tuple._1, tuple._2, tuple._3) + def metadata = offsetMetadata.metadata - def asTuple = (offset, metadata, error) + override def toString = "[%s,CommitTime %d,ExpirationTime %d]".format(offsetMetadata, commitTimestamp, expireTimestamp) +} + +object OffsetAndMetadata { + def apply(offset: Long, metadata: String, commitTimestamp: Long, expireTimestamp: Long) = new OffsetAndMetadata(OffsetMetadata(offset, metadata), commitTimestamp, expireTimestamp) + + def apply(offset: Long, metadata: String, timestamp: Long) = new OffsetAndMetadata(OffsetMetadata(offset, metadata), timestamp) + + def apply(offset: Long, metadata: String) = new OffsetAndMetadata(OffsetMetadata(offset, metadata)) + + def apply(offset: Long) = new OffsetAndMetadata(OffsetMetadata(offset, OffsetMetadata.NoMetadata)) +} + +case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short = Errors.NONE.code) { + def offset = offsetMetadata.offset - override def toString = "OffsetAndMetadata[%d,%s,%d]".format(offset, metadata, error) + def metadata = offsetMetadata.metadata + override def toString = "[%s,ErrorCode %d]".format(offsetMetadata, error) } object OffsetMetadataAndError { - val InvalidOffset: Long = -1L; - val NoMetadata: String = ""; + val NoOffset = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.NONE.code) + val OffsetsLoading = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.OFFSET_LOAD_IN_PROGRESS.code) + val UnknownConsumer = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.UNKNOWN_CONSUMER_ID.code) + val NotCoordinatorForGroup = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.NOT_COORDINATOR_FOR_CONSUMER.code) + val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + val IllegalGroupGenerationId = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.ILLEGAL_GENERATION.code) + + def apply(offset: Long) = new OffsetMetadataAndError(OffsetMetadata(offset, OffsetMetadata.NoMetadata), ErrorMapping.NoError) + + def apply(error: Short) = new OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, error) + + def apply(offset: Long, metadata: String, error: Short) = new OffsetMetadataAndError(OffsetMetadata(offset, metadata), error) } + + + diff --git a/core/src/main/scala/kafka/utils/Annotations_2.8.scala b/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala similarity index 63% rename from core/src/main/scala/kafka/utils/Annotations_2.8.scala rename to core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala index 28269eb037109..1c8e96eefc7f0 100644 --- a/core/src/main/scala/kafka/utils/Annotations_2.8.scala +++ b/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.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,22 +15,12 @@ * limitations under the License. */ -package kafka.utils - -/* Some helpful annotations */ +package kafka.common /** - * Indicates that the annotated class is meant to be threadsafe. For an abstract class it is an part of the interface that an implementation - * must respect + * Indicates that offsets are currently being loaded from disk into the cache so offset fetch requests cannot be satisfied. */ -class threadsafe extends StaticAnnotation +class OffsetsLoadInProgressException(message: String) extends RuntimeException(message) { + def this() = this(null) +} -/** - * Indicates that the annotated class is not threadsafe - */ -class nonthreadsafe extends StaticAnnotation - -/** - * Indicates that the annotated class is immutable - */ -class immutable extends StaticAnnotation diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index c1b9f65114c77..32595d6fe4321 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -18,12 +18,16 @@ package kafka.common import util.matching.Regex +import kafka.coordinator.ConsumerCoordinator + object Topic { val legalChars = "[a-zA-Z0-9\\._\\-]" private val maxNameLength = 255 private val rgx = new Regex(legalChars + "+") + val InternalTopics = Set(ConsumerCoordinator.OffsetsTopicName) + def validate(topic: String) { if (topic.length <= 0) throw new InvalidTopicException("topic name is illegal, can't be empty") diff --git a/core/src/main/scala/kafka/common/TopicAlreadyMarkedForDeletionException.scala b/core/src/main/scala/kafka/common/TopicAlreadyMarkedForDeletionException.scala new file mode 100644 index 0000000000000..c83cea96b5de0 --- /dev/null +++ b/core/src/main/scala/kafka/common/TopicAlreadyMarkedForDeletionException.scala @@ -0,0 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.common + +class TopicAlreadyMarkedForDeletionException(message: String) extends RuntimeException(message) { +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/TopicAndPartition.scala b/core/src/main/scala/kafka/common/TopicAndPartition.scala index df3db912f5dae..13a3f2820bbd3 100644 --- a/core/src/main/scala/kafka/common/TopicAndPartition.scala +++ b/core/src/main/scala/kafka/common/TopicAndPartition.scala @@ -1,6 +1,7 @@ package kafka.common import kafka.cluster.{Replica, Partition} +import kafka.utils.Json /** * Licensed to the Apache Software Foundation (ASF) under one or more @@ -24,6 +25,8 @@ import kafka.cluster.{Replica, Partition} */ case class TopicAndPartition(topic: String, partition: Int) { + private val version: Long = 1L + def this(tuple: (String, Int)) = this(tuple._1, tuple._2) def this(partition: Partition) = this(partition.topic, partition.partitionId) @@ -33,5 +36,6 @@ case class TopicAndPartition(topic: String, partition: Int) { def asTuple = (topic, partition) override def toString = "[%s,%d]".format(topic, partition) -} + def toJson = Json.encode(Map("version" -> version, "topic" -> topic, "partition" -> partition)) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala b/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala index 781e551e5b78b..f382d16de9c39 100644 --- a/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala +++ b/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala @@ -17,7 +17,9 @@ package kafka.common /** - * Indicates an unknown topic or a partition id not between 0 and numPartitions-1 + * Indicates one of the following situation: + * 1. Producer does not have the partition metadata for this id upon sending messages + * 2. Broker does not have the specified partition by id upon receiving messages */ class UnknownTopicOrPartitionException(message: String) extends RuntimeException(message) { def this() = this(null) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index e6875d6aa830e..97a56ce7f2acb 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -41,9 +41,15 @@ object ConsumerConfig extends Config { val MirrorTopicsWhitelist = "" val MirrorTopicsBlacklist = "" val MirrorConsumerNumThreads = 1 + val OffsetsChannelBackoffMs = 1000 + val OffsetsChannelSocketTimeoutMs = 10000 + val OffsetsCommitMaxRetries = 5 + val OffsetsStorage = "zookeeper" val MirrorTopicsWhitelistProp = "mirror.topics.whitelist" val MirrorTopicsBlacklistProp = "mirror.topics.blacklist" + val ExcludeInternalTopics = true + val DefaultPartitionAssignmentStrategy = "range" /* select between "range", and "roundrobin" */ val MirrorConsumerNumThreadsProp = "mirror.consumer.numthreads" val DefaultClientId = "" @@ -51,6 +57,8 @@ object ConsumerConfig extends Config { validateClientId(config.clientId) validateGroupId(config.groupId) validateAutoOffsetReset(config.autoOffsetReset) + validateOffsetsStorage(config.offsetsStorage) + validatePartitionAssignmentStrategy(config.partitionAssignmentStrategy) } def validateClientId(clientId: String) { @@ -69,6 +77,24 @@ object ConsumerConfig extends Config { "Valid values are " + OffsetRequest.SmallestTimeString + " and " + OffsetRequest.LargestTimeString) } } + + def validateOffsetsStorage(storage: String) { + storage match { + case "zookeeper" => + case "kafka" => + case _ => throw new InvalidConfigException("Wrong value " + storage + " of offsets.storage in consumer config; " + + "Valid values are 'zookeeper' and 'kafka'") + } + } + + def validatePartitionAssignmentStrategy(strategy: String) { + strategy match { + case "range" => + case "roundrobin" => + case _ => throw new InvalidConfigException("Wrong value " + strategy + " of partition.assignment.strategy in consumer config; " + + "Valid values are 'range' and 'roundrobin'") + } + } } class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) { @@ -86,7 +112,7 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( * Set this explicitly for only testing purpose. */ val consumerId: Option[String] = Option(props.getString("consumer.id", null)) - /** the socket timeout for network requests. The actual timeout set will be max.fetch.wait + socket.timeout.ms. */ + /** the socket timeout for network requests. Its value should be at least fetch.wait.max.ms. */ val socketTimeoutMs = props.getInt("socket.timeout.ms", SocketTimeout) /** the socket receive buffer for network requests */ @@ -115,6 +141,8 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** the maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy fetch.min.bytes */ val fetchWaitMaxMs = props.getInt("fetch.wait.max.ms", MaxFetchWaitMs) + require(fetchWaitMaxMs <= socketTimeoutMs, "socket.timeout.ms should always be at least fetch.wait.max.ms" + + " to prevent unnecessary socket timeouts") /** backoff time between retries during rebalance */ val rebalanceBackoffMs = props.getInt("rebalance.backoff.ms", zkSyncTimeMs) @@ -122,6 +150,27 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** backoff time to refresh the leader of a partition after it loses the current leader */ val refreshLeaderBackoffMs = props.getInt("refresh.leader.backoff.ms", RefreshMetadataBackoffMs) + /** backoff time to reconnect the offsets channel or to retry offset fetches/commits */ + val offsetsChannelBackoffMs = props.getInt("offsets.channel.backoff.ms", OffsetsChannelBackoffMs) + /** socket timeout to use when reading responses for Offset Fetch/Commit requests. This timeout will also be used for + * the ConsumerMetdata requests that are used to query for the offset coordinator. */ + val offsetsChannelSocketTimeoutMs = props.getInt("offsets.channel.socket.timeout.ms", OffsetsChannelSocketTimeoutMs) + + /** Retry the offset commit up to this many times on failure. This retry count only applies to offset commits during + * shut-down. It does not apply to commits from the auto-commit thread. It also does not apply to attempts to query + * for the offset coordinator before committing offsets. i.e., if a consumer metadata request fails for any reason, + * it is retried and that retry does not count toward this limit. */ + val offsetsCommitMaxRetries = props.getInt("offsets.commit.max.retries", OffsetsCommitMaxRetries) + + /** Specify whether offsets should be committed to "zookeeper" (default) or "kafka" */ + val offsetsStorage = props.getString("offsets.storage", OffsetsStorage).toLowerCase + + /** If you are using "kafka" as offsets.storage, you can dual commit offsets to ZooKeeper (in addition to Kafka). This + * is required during migration from zookeeper-based offset storage to kafka-based offset storage. With respect to any + * given consumer group, it is safe to turn this off after all instances within that group have been migrated to + * the new jar that commits offsets to the broker (instead of directly to ZooKeeper). */ + val dualCommitEnabled = props.getBoolean("dual.commit.enabled", if (offsetsStorage == "kafka") true else false) + /* what to do if an offset is out of range. smallest : automatically reset the offset to the smallest offset largest : automatically reset the offset to the largest offset @@ -136,6 +185,12 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( */ val clientId = props.getString("client.id", groupId) + /** Whether messages from internal topics (such as offsets) should be exposed to the consumer. */ + val excludeInternalTopics = props.getBoolean("exclude.internal.topics", ExcludeInternalTopics) + + /** Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin */ + val partitionAssignmentStrategy = props.getString("partition.assignment.strategy", DefaultPartitionAssignmentStrategy) + validate(this) } diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala index 13c3f771e9a96..384be74703050 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala @@ -17,6 +17,9 @@ package kafka.consumer +import kafka.common.{OffsetAndMetadata, TopicAndPartition} +import kafka.javaapi.consumer.ConsumerRebalanceListener + import scala.collection._ import kafka.utils.Logging import kafka.serializer._ @@ -70,7 +73,24 @@ trait ConsumerConnector { /** * Commit the offsets of all broker partitions connected by this connector. */ + def commitOffsets(retryOnFailure: Boolean) + + /** + * KAFKA-1743: This method added for backward compatibility. + */ def commitOffsets + + /** + * Commit offsets from an external offsets map. + * @param offsetsToCommit the offsets to be committed. + */ + def commitOffsets(offsetsToCommit: immutable.Map[TopicAndPartition, OffsetAndMetadata], retryOnFailure: Boolean) + + /** + * Wire in a consumer rebalance listener to be executed when consumer rebalance occurs. + * @param listener The consumer rebalance listener to wire in + */ + def setConsumerRebalanceListener(listener: ConsumerRebalanceListener) /** * Shut down the connector diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala old mode 100644 new mode 100755 index b9e2bea7b442a..49b683f290da2 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -19,13 +19,13 @@ package kafka.consumer import org.I0Itec.zkclient.ZkClient import kafka.server.{BrokerAndInitialOffset, AbstractFetcherThread, AbstractFetcherManager} -import kafka.cluster.{Cluster, Broker} +import kafka.cluster.{BrokerEndPoint, Cluster} +import org.apache.kafka.common.protocol.SecurityProtocol 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.CoreUtils.inLock import kafka.utils.ZkUtils._ import kafka.utils.{ShutdownableThread, SystemTime} import kafka.common.TopicAndPartition @@ -53,7 +53,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, private class LeaderFinderThread(name: String) extends ShutdownableThread(name) { // thread responsible for adding the fetcher to the right broker when leader is available override def doWork() { - val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker] + val leaderForPartitionsMap = new HashMap[TopicAndPartition, BrokerEndPoint] lock.lock() try { while (noLeaderPartitionSet.isEmpty) { @@ -62,7 +62,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, } trace("Partitions without leader %s".format(noLeaderPartitionSet)) - val brokers = getAllBrokersInCluster(zkClient) + val brokers = getAllBrokerEndPointsForChannel(zkClient, SecurityProtocol.PLAINTEXT) val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, brokers, config.clientId, @@ -114,7 +114,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, } } - override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { + override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { new ConsumerFetcherThread( "ConsumerFetcherThread-%s-%d-%d".format(consumerIdString, fetcherId, sourceBroker.id), config, sourceBroker, partitionMap, this) diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index f8c1b4e674f75..33ea728771eb0 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -17,7 +17,7 @@ package kafka.consumer -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import kafka.server.AbstractFetcherThread import kafka.message.ByteBufferMessageSet import kafka.api.{Request, OffsetRequest, FetchResponsePartitionData} @@ -26,11 +26,11 @@ import kafka.common.TopicAndPartition class ConsumerFetcherThread(name: String, val config: ConsumerConfig, - sourceBroker: Broker, + sourceBroker: BrokerEndPoint, partitionMap: Map[TopicAndPartition, PartitionTopicInfo], val consumerFetcherManager: ConsumerFetcherManager) - extends AbstractFetcherThread(name = name, - clientId = config.clientId + "-" + name, + extends AbstractFetcherThread(name = name, + clientId = config.clientId, sourceBroker = sourceBroker, socketTimeout = config.socketTimeoutMs, socketBufferSize = config.socketReceiveBufferBytes, @@ -38,6 +38,7 @@ class ConsumerFetcherThread(name: String, fetcherBrokerId = Request.OrdinaryConsumerId, maxWait = config.fetchWaitMaxMs, minBytes = config.fetchMinBytes, + fetchBackOffMs = config.refreshLeaderBackoffMs, isInterruptible = true) { // process fetched data diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala old mode 100644 new mode 100755 index ac491b4da2583..0c6c810bdc567 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -17,7 +17,7 @@ package kafka.consumer -import kafka.utils.{IteratorTemplate, Logging, Utils} +import kafka.utils.{IteratorTemplate, Logging, CoreUtils} import java.util.concurrent.{TimeUnit, BlockingQueue} import kafka.serializer.Decoder import java.util.concurrent.atomic.AtomicReference @@ -37,7 +37,7 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk val clientId: String) extends IteratorTemplate[MessageAndMetadata[K, V]] with Logging { - private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null) + private val current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null) private var currentTopicInfo: PartitionTopicInfo = null private var consumedOffset: Long = -1L private val consumerTopicStats = ConsumerTopicStatsRegistry.getConsumerTopicStat(clientId) @@ -71,7 +71,6 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk } if(currentDataChunk eq ZookeeperConsumerConnector.shutdownCommand) { debug("Received the shutdown command") - channel.offer(currentDataChunk) return allDone } else { currentTopicInfo = currentDataChunk.topicInfo @@ -105,10 +104,8 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk } def clearCurrentChunk() { - try { - debug("Clearing the current data chunk for this consumer iterator") - current.set(null) - } + debug("Clearing the current data chunk for this consumer iterator") + current.set(null) } } diff --git a/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala b/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala index ff5f470f7aa30..01797ff766a7f 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala @@ -20,12 +20,17 @@ package kafka.consumer import kafka.utils.{Pool, threadsafe, Logging} import java.util.concurrent.TimeUnit import kafka.metrics.KafkaMetricsGroup -import kafka.common.ClientIdAndTopic +import kafka.common.{ClientIdTopic, ClientIdAllTopics, ClientIdAndTopic} @threadsafe -class ConsumerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup { - val messageRate = newMeter(metricId + "MessagesPerSec", "messages", TimeUnit.SECONDS) - val byteRate = newMeter(metricId + "BytesPerSec", "bytes", TimeUnit.SECONDS) +class ConsumerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup { + val tags = metricId match { + case ClientIdAndTopic(clientId, topic) => Map("clientId" -> clientId, "topic" -> topic) + case ClientIdAllTopics(clientId) => Map("clientId" -> clientId) + } + + val messageRate = newMeter("MessagesPerSec", "messages", TimeUnit.SECONDS, tags) + val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, tags) } /** @@ -35,12 +40,12 @@ class ConsumerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup class ConsumerTopicStats(clientId: String) extends Logging { private val valueFactory = (k: ClientIdAndTopic) => new ConsumerTopicMetrics(k) private val stats = new Pool[ClientIdAndTopic, ConsumerTopicMetrics](Some(valueFactory)) - private val allTopicStats = new ConsumerTopicMetrics(new ClientIdAndTopic(clientId, "AllTopics")) // to differentiate from a topic named AllTopics + private val allTopicStats = new ConsumerTopicMetrics(new ClientIdAllTopics(clientId)) // to differentiate from a topic named AllTopics def getConsumerAllTopicStats(): ConsumerTopicMetrics = allTopicStats def getConsumerTopicStats(topic: String): ConsumerTopicMetrics = { - stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic + "-")) + stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic)) } } @@ -54,4 +59,8 @@ object ConsumerTopicStatsRegistry { def getConsumerTopicStat(clientId: String) = { globalStats.getAndMaybePut(clientId) } + + def removeConsumerTopicStat(clientId: String) { + globalStats.remove(clientId) + } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala b/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala index 875eeeb73cba5..3df55e13001ce 100644 --- a/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala +++ b/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala @@ -17,14 +17,23 @@ package kafka.consumer -import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} -import kafka.utils.Pool import java.util.concurrent.TimeUnit -import kafka.common.ClientIdAndBroker -class FetchRequestAndResponseMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGroup { - val requestTimer = new KafkaTimer(newTimer(metricId + "FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) - val requestSizeHist = newHistogram(metricId + "FetchResponseSize") +import kafka.common.{ClientIdAllBrokers, ClientIdBroker, ClientIdAndBroker} +import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} +import kafka.utils.Pool + +class FetchRequestAndResponseMetrics(metricId: ClientIdBroker) extends KafkaMetricsGroup { + val tags = metricId match { + case ClientIdAndBroker(clientId, brokerHost, brokerPort) => + Map("clientId" -> clientId, "brokerHost" -> brokerHost, + "brokerPort" -> brokerPort.toString) + case ClientIdAllBrokers(clientId) => + Map("clientId" -> clientId) + } + + val requestTimer = new KafkaTimer(newTimer("FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, tags)) + val requestSizeHist = newHistogram("FetchResponseSize", biased = true, tags) } /** @@ -32,14 +41,14 @@ class FetchRequestAndResponseMetrics(metricId: ClientIdAndBroker) extends KafkaM * @param clientId ClientId of the given consumer */ class FetchRequestAndResponseStats(clientId: String) { - private val valueFactory = (k: ClientIdAndBroker) => new FetchRequestAndResponseMetrics(k) - private val stats = new Pool[ClientIdAndBroker, FetchRequestAndResponseMetrics](Some(valueFactory)) - private val allBrokersStats = new FetchRequestAndResponseMetrics(new ClientIdAndBroker(clientId, "AllBrokers")) + private val valueFactory = (k: ClientIdBroker) => new FetchRequestAndResponseMetrics(k) + private val stats = new Pool[ClientIdBroker, FetchRequestAndResponseMetrics](Some(valueFactory)) + private val allBrokersStats = new FetchRequestAndResponseMetrics(new ClientIdAllBrokers(clientId)) def getFetchRequestAndResponseAllBrokersStats(): FetchRequestAndResponseMetrics = allBrokersStats - def getFetchRequestAndResponseStats(brokerInfo: String): FetchRequestAndResponseMetrics = { - stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo + "-")) + def getFetchRequestAndResponseStats(brokerHost: String, brokerPort: Int): FetchRequestAndResponseMetrics = { + stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerHost, brokerPort)) } } @@ -53,6 +62,17 @@ object FetchRequestAndResponseStatsRegistry { def getFetchRequestAndResponseStats(clientId: String) = { globalStats.getAndMaybePut(clientId) } + + def removeConsumerFetchRequestAndResponseStats(clientId: String) { + val pattern = (".*" + clientId + ".*").r + val keys = globalStats.keys + for (key <- keys) { + pattern.findFirstIn(key) match { + case Some(_) => globalStats.remove(key) + case _ => + } + } + } } diff --git a/core/src/main/scala/kafka/consumer/KafkaStream.scala b/core/src/main/scala/kafka/consumer/KafkaStream.scala index 31eaf866e3c84..805e91677034e 100644 --- a/core/src/main/scala/kafka/consumer/KafkaStream.scala +++ b/core/src/main/scala/kafka/consumer/KafkaStream.scala @@ -45,4 +45,7 @@ class KafkaStream[K,V](private val queue: BlockingQueue[FetchedDataChunk], iter.clearCurrentChunk() } + override def toString(): String = { + "%s kafka stream".format(clientId) + } } diff --git a/core/src/main/scala/kafka/consumer/PartitionAssignor.scala b/core/src/main/scala/kafka/consumer/PartitionAssignor.scala new file mode 100755 index 0000000000000..849284ad2cfa0 --- /dev/null +++ b/core/src/main/scala/kafka/consumer/PartitionAssignor.scala @@ -0,0 +1,175 @@ +/** + * 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.consumer + +import org.I0Itec.zkclient.ZkClient +import kafka.common.TopicAndPartition +import kafka.utils.{Pool, CoreUtils, ZkUtils, Logging} + +import scala.collection.mutable + +trait PartitionAssignor { + + /** + * Assigns partitions to consumer instances in a group. + * @return An assignment map of partition to this consumer group. This includes assignments for threads that belong + * to the same consumer group. + */ + def assign(ctx: AssignmentContext): Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]] + +} + +object PartitionAssignor { + def createInstance(assignmentStrategy: String) = assignmentStrategy match { + case "roundrobin" => new RoundRobinAssignor() + case _ => new RangeAssignor() + } +} + +class AssignmentContext(group: String, val consumerId: String, excludeInternalTopics: Boolean, zkClient: ZkClient) { + val myTopicThreadIds: collection.Map[String, collection.Set[ConsumerThreadId]] = { + val myTopicCount = TopicCount.constructTopicCount(group, consumerId, zkClient, excludeInternalTopics) + myTopicCount.getConsumerThreadIdsPerTopic + } + + val partitionsForTopic: collection.Map[String, Seq[Int]] = + ZkUtils.getPartitionsForTopics(zkClient, myTopicThreadIds.keySet.toSeq) + + val consumersForTopic: collection.Map[String, List[ConsumerThreadId]] = + ZkUtils.getConsumersPerTopic(zkClient, group, excludeInternalTopics) + + val consumers: Seq[String] = ZkUtils.getConsumersInGroup(zkClient, group).sorted +} + +/** + * The round-robin partition assignor lays out all the available partitions and all the available consumer threads. It + * then proceeds to do a round-robin assignment from partition to consumer thread. If the subscriptions of all consumer + * instances are identical, then the partitions will be uniformly distributed. (i.e., the partition ownership counts + * will be within a delta of exactly one across all consumer threads.) + * + * (For simplicity of implementation) the assignor is allowed to assign a given topic-partition to any consumer instance + * and thread-id within that instance. Therefore, round-robin assignment is allowed only if: + * a) Every topic has the same number of streams within a consumer instance + * b) The set of subscribed topics is identical for every consumer instance within the group. + */ + +class RoundRobinAssignor() extends PartitionAssignor with Logging { + + def assign(ctx: AssignmentContext) = { + + val valueFactory = (topic: String) => new mutable.HashMap[TopicAndPartition, ConsumerThreadId] + val partitionAssignment = + new Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]](Some(valueFactory)) + + if (ctx.consumersForTopic.size > 0) { + // check conditions (a) and (b) + val (headTopic, headThreadIdSet) = (ctx.consumersForTopic.head._1, ctx.consumersForTopic.head._2.toSet) + ctx.consumersForTopic.foreach { case (topic, threadIds) => + val threadIdSet = threadIds.toSet + require(threadIdSet == headThreadIdSet, + "Round-robin assignment is allowed only if all consumers in the group subscribe to the same topics, " + + "AND if the stream counts across topics are identical for a given consumer instance.\n" + + "Topic %s has the following available consumer streams: %s\n".format(topic, threadIdSet) + + "Topic %s has the following available consumer streams: %s\n".format(headTopic, headThreadIdSet)) + } + + val threadAssignor = CoreUtils.circularIterator(headThreadIdSet.toSeq.sorted) + + info("Starting round-robin assignment with consumers " + ctx.consumers) + val allTopicPartitions = ctx.partitionsForTopic.flatMap { case (topic, partitions) => + info("Consumer %s rebalancing the following partitions for topic %s: %s" + .format(ctx.consumerId, topic, partitions)) + partitions.map(partition => { + TopicAndPartition(topic, partition) + }) + }.toSeq.sortWith((topicPartition1, topicPartition2) => { + /* + * Randomize the order by taking the hashcode to reduce the likelihood of all partitions of a given topic ending + * up on one consumer (if it has a high enough stream count). + */ + topicPartition1.toString.hashCode < topicPartition2.toString.hashCode + }) + + allTopicPartitions.foreach(topicPartition => { + val threadId = threadAssignor.next() + // record the partition ownership decision + val assignmentForConsumer = partitionAssignment.getAndMaybePut(threadId.consumer) + assignmentForConsumer += (topicPartition -> threadId) + }) + } + + // assign Map.empty for the consumers which are not associated with topic partitions + ctx.consumers.foreach(consumerId => partitionAssignment.getAndMaybePut(consumerId)) + partitionAssignment + } +} + +/** + * Range partitioning works on a per-topic basis. For each topic, we lay out the available partitions in numeric order + * and the consumer threads in lexicographic order. We then divide the number of partitions by the total number of + * consumer streams (threads) to determine the number of partitions to assign to each consumer. If it does not evenly + * divide, then the first few consumers will have one extra partition. For example, suppose there are two consumers C1 + * and C2 with two streams each, and there are five available partitions (p0, p1, p2, p3, p4). So each consumer thread + * will get at least one partition and the first consumer thread will get one extra partition. So the assignment will be: + * p0 -> C1-0, p1 -> C1-0, p2 -> C1-1, p3 -> C2-0, p4 -> C2-1 + */ +class RangeAssignor() extends PartitionAssignor with Logging { + + def assign(ctx: AssignmentContext) = { + val valueFactory = (topic: String) => new mutable.HashMap[TopicAndPartition, ConsumerThreadId] + val partitionAssignment = + new Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]](Some(valueFactory)) + for (topic <- ctx.myTopicThreadIds.keySet) { + val curConsumers = ctx.consumersForTopic(topic) + val curPartitions: Seq[Int] = ctx.partitionsForTopic(topic) + + val nPartsPerConsumer = curPartitions.size / curConsumers.size + val nConsumersWithExtraPart = curPartitions.size % curConsumers.size + + info("Consumer " + ctx.consumerId + " rebalancing the following partitions: " + curPartitions + + " for topic " + topic + " with consumers: " + curConsumers) + + for (consumerThreadId <- curConsumers) { + val myConsumerPosition = curConsumers.indexOf(consumerThreadId) + assert(myConsumerPosition >= 0) + val startPart = nPartsPerConsumer * myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) + val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1) + + /** + * Range-partition the sorted partitions to consumers for better locality. + * The first few consumers pick up an extra partition, if any. + */ + if (nParts <= 0) + warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic) + else { + for (i <- startPart until startPart + nParts) { + val partition = curPartitions(i) + info(consumerThreadId + " attempting to claim partition " + partition) + // record the partition ownership decision + val assignmentForConsumer = partitionAssignment.getAndMaybePut(consumerThreadId.consumer) + assignmentForConsumer += (TopicAndPartition(topic, partition) -> consumerThreadId) + } + } + } + } + + // assign Map.empty for the consumers which are not associated with topic partitions + ctx.consumers.foreach(consumerId => partitionAssignment.getAndMaybePut(consumerId)) + partitionAssignment + } +} diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index 6dae149adfd4b..7ebc0405d1f30 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -17,10 +17,15 @@ package kafka.consumer + +import java.nio.channels.{AsynchronousCloseException, ClosedByInterruptException} + import kafka.api._ import kafka.network._ import kafka.utils._ import kafka.common.{ErrorMapping, TopicAndPartition} +import org.apache.kafka.common.network.{NetworkReceive, Receive} +import org.apache.kafka.common.utils.Utils._ /** * A consumer of kafka messages @@ -35,7 +40,6 @@ class SimpleConsumer(val host: String, ConsumerConfig.validateClientId(clientId) private val lock = new Object() private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout) - val brokerInfo = "host_%s-port_%s".format(host, port) private val fetchRequestAndResponseStats = FetchRequestAndResponseStatsRegistry.getFetchRequestAndResponseStats(clientId) private var isClosed = false @@ -46,10 +50,8 @@ class SimpleConsumer(val host: String, } private def disconnect() = { - if(blockingChannel.isConnected) { - debug("Disconnecting from " + host + ":" + port) - blockingChannel.disconnect() - } + debug("Disconnecting from " + formatAddress(host, port)) + blockingChannel.disconnect() } private def reconnect() { @@ -57,6 +59,16 @@ class SimpleConsumer(val host: String, connect() } + /** + * Unblock thread by closing channel and triggering AsynchronousCloseException if a read operation is in progress. + * + * This handles a bug found in Java 1.7 and below, where interrupting a thread can not correctly unblock + * the thread from waiting on ReadableByteChannel.read(). + */ + def disconnectToHandleJavaIOBug() = { + disconnect() + } + def close() { lock synchronized { disconnect() @@ -64,16 +76,21 @@ class SimpleConsumer(val host: String, } } - private def sendRequest(request: RequestOrResponse): Receive = { + private def sendRequest(request: RequestOrResponse): NetworkReceive = { lock synchronized { - getOrMakeConnection() - var response: Receive = null + var response: NetworkReceive = null try { + getOrMakeConnection() blockingChannel.send(request) response = blockingChannel.receive() } catch { + case e : ClosedByInterruptException => + throw e + // Should not observe this exception when running Kafka with Java 1.8 + case e: AsynchronousCloseException => + throw e case e : Throwable => - warn("Reconnect due to socket error: %s".format(e.getMessage)) + info("Reconnect due to socket error: %s".format(e.toString)) // retry once try { reconnect() @@ -91,7 +108,12 @@ class SimpleConsumer(val host: String, def send(request: TopicMetadataRequest): TopicMetadataResponse = { val response = sendRequest(request) - TopicMetadataResponse.readFrom(response.buffer) + TopicMetadataResponse.readFrom(response.payload()) + } + + def send(request: ConsumerMetadataRequest): ConsumerMetadataResponse = { + val response = sendRequest(request) + ConsumerMetadataResponse.readFrom(response.payload()) } /** @@ -101,17 +123,17 @@ class SimpleConsumer(val host: String, * @return a set of fetched messages */ def fetch(request: FetchRequest): FetchResponse = { - var response: Receive = null - val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(brokerInfo).requestTimer + var response: NetworkReceive = null + val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestTimer val aggregateTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestTimer aggregateTimer.time { specificTimer.time { response = sendRequest(request) } } - val fetchResponse = FetchResponse.readFrom(response.buffer) + val fetchResponse = FetchResponse.readFrom(response.payload()) val fetchedSize = fetchResponse.sizeInBytes - fetchRequestAndResponseStats.getFetchRequestAndResponseStats(brokerInfo).requestSizeHist.update(fetchedSize) + fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestSizeHist.update(fetchedSize) fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize) fetchResponse } @@ -121,21 +143,27 @@ class SimpleConsumer(val host: String, * @param request a [[kafka.api.OffsetRequest]] object. * @return a [[kafka.api.OffsetResponse]] object. */ - def getOffsetsBefore(request: OffsetRequest) = OffsetResponse.readFrom(sendRequest(request).buffer) + def getOffsetsBefore(request: OffsetRequest) = OffsetResponse.readFrom(sendRequest(request).payload()) /** * Commit offsets for a topic + * Version 0 of the request will commit offsets to Zookeeper and version 1 and above will commit offsets to Kafka. * @param request a [[kafka.api.OffsetCommitRequest]] object. * @return a [[kafka.api.OffsetCommitResponse]] object. */ - def commitOffsets(request: OffsetCommitRequest) = OffsetCommitResponse.readFrom(sendRequest(request).buffer) + def commitOffsets(request: OffsetCommitRequest) = { + // TODO: With KAFKA-1012, we have to first issue a ConsumerMetadataRequest and connect to the coordinator before + // we can commit offsets. + OffsetCommitResponse.readFrom(sendRequest(request).payload()) + } /** * Fetch offsets for a topic + * Version 0 of the request will fetch offsets from Zookeeper and version 1 and above will fetch offsets from Kafka. * @param request a [[kafka.api.OffsetFetchRequest]] object. * @return a [[kafka.api.OffsetFetchResponse]] object. */ - def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).buffer) + def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).payload()) private def getOrMakeConnection() { if(!isClosed && !blockingChannel.isConnected) { diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala old mode 100644 new mode 100755 index e33263378489f..6994c8e89055b --- a/core/src/main/scala/kafka/consumer/TopicCount.scala +++ b/core/src/main/scala/kafka/consumer/TopicCount.scala @@ -19,35 +19,44 @@ package kafka.consumer import scala.collection._ import org.I0Itec.zkclient.ZkClient -import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging} +import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, CoreUtils} import kafka.common.KafkaException private[kafka] trait TopicCount { - def getConsumerThreadIdsPerTopic: Map[String, Set[String]] + def getConsumerThreadIdsPerTopic: Map[String, Set[ConsumerThreadId]] def getTopicCountMap: Map[String, Int] def pattern: String - - protected def makeConsumerThreadIdsPerTopic(consumerIdString: String, - topicCountMap: Map[String, Int]) = { - val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[String]]() + +} + +case class ConsumerThreadId(consumer: String, threadId: Int) extends Ordered[ConsumerThreadId] { + override def toString = "%s-%d".format(consumer, threadId) + + def compare(that: ConsumerThreadId) = toString.compare(that.toString) +} + +private[kafka] object TopicCount extends Logging { + val whiteListPattern = "white_list" + val blackListPattern = "black_list" + val staticPattern = "static" + + def makeThreadId(consumerIdString: String, threadId: Int) = consumerIdString + "-" + threadId + + def makeConsumerThreadIdsPerTopic(consumerIdString: String, + topicCountMap: Map[String, Int]) = { + val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[ConsumerThreadId]]() for ((topic, nConsumers) <- topicCountMap) { - val consumerSet = new mutable.HashSet[String] + val consumerSet = new mutable.HashSet[ConsumerThreadId] assert(nConsumers >= 1) for (i <- 0 until nConsumers) - consumerSet += consumerIdString + "-" + i + consumerSet += ConsumerThreadId(consumerIdString, i) consumerThreadIdsPerTopicMap.put(topic, consumerSet) } consumerThreadIdsPerTopicMap } -} - -private[kafka] object TopicCount extends Logging { - val whiteListPattern = "white_list" - val blackListPattern = "black_list" - val staticPattern = "static" - def constructTopicCount(group: String, consumerId: String, zkClient: ZkClient) : TopicCount = { + def constructTopicCount(group: String, consumerId: String, zkClient: ZkClient, excludeInternalTopics: Boolean) : TopicCount = { val dirs = new ZKGroupDirs(group) val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)._1 var subscriptionPattern: String = null @@ -85,15 +94,15 @@ private[kafka] object TopicCount extends Logging { new Whitelist(regex) else new Blacklist(regex) - new WildcardTopicCount(zkClient, consumerId, filter, numStreams) + new WildcardTopicCount(zkClient, consumerId, filter, numStreams, excludeInternalTopics) } } def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) = new StaticTopicCount(consumerIdString, topicCount) - def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkClient: ZkClient) = - new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams) + def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkClient: ZkClient, excludeInternalTopics: Boolean) = + new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams, excludeInternalTopics) } @@ -101,7 +110,7 @@ private[kafka] class StaticTopicCount(val consumerIdString: String, val topicCountMap: Map[String, Int]) extends TopicCount { - def getConsumerThreadIdsPerTopic = makeConsumerThreadIdsPerTopic(consumerIdString, topicCountMap) + def getConsumerThreadIdsPerTopic = TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, topicCountMap) override def equals(obj: Any): Boolean = { obj match { @@ -119,13 +128,15 @@ private[kafka] class StaticTopicCount(val consumerIdString: String, private[kafka] class WildcardTopicCount(zkClient: ZkClient, consumerIdString: String, topicFilter: TopicFilter, - numStreams: Int) extends TopicCount { + numStreams: Int, + excludeInternalTopics: Boolean) extends TopicCount { def getConsumerThreadIdsPerTopic = { - val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath).filter(topicFilter.isTopicAllowed(_)) - makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*)) + val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath) + .filter(topic => topicFilter.isTopicAllowed(topic, excludeInternalTopics)) + TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*)) } - def getTopicCountMap = Map(topicFilter.regex -> numStreams) + def getTopicCountMap = Map(CoreUtils.JSONEscapeString(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 4f2082360cfbb..5a13540699be4 100644 --- a/core/src/main/scala/kafka/consumer/TopicFilter.scala +++ b/core/src/main/scala/kafka/consumer/TopicFilter.scala @@ -20,6 +20,7 @@ package kafka.consumer import kafka.utils.Logging import java.util.regex.{PatternSyntaxException, Pattern} +import kafka.common.Topic sealed abstract class TopicFilter(rawRegex: String) extends Logging { @@ -41,12 +42,12 @@ sealed abstract class TopicFilter(rawRegex: String) extends Logging { override def toString = regex - def isTopicAllowed(topic: String): Boolean + def isTopicAllowed(topic: String, excludeInternalTopics: Boolean): Boolean } case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) { - override def isTopicAllowed(topic: String) = { - val allowed = topic.matches(regex) + override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = { + val allowed = topic.matches(regex) && !(Topic.InternalTopics.contains(topic) && excludeInternalTopics) debug("%s %s".format( topic, if (allowed) "allowed" else "filtered")) @@ -58,8 +59,8 @@ case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) { } case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) { - override def isTopicAllowed(topic: String) = { - val allowed = !topic.matches(regex) + override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = { + val allowed = (!topic.matches(regex)) && !(Topic.InternalTopics.contains(topic) && excludeInternalTopics) debug("%s %s".format( topic, if (allowed) "allowed" else "filtered")) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala old mode 100644 new mode 100755 index 703b2e22605ca..e42d10488f8df --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -17,24 +17,30 @@ package kafka.consumer +import java.net.InetAddress +import java.util.UUID import java.util.concurrent._ import java.util.concurrent.atomic._ -import locks.ReentrantLock -import collection._ +import java.util.concurrent.locks.ReentrantLock + +import com.yammer.metrics.core.Gauge +import kafka.api._ +import kafka.client.ClientUtils import kafka.cluster._ +import kafka.common._ +import kafka.javaapi.consumer.ConsumerRebalanceListener +import kafka.metrics._ +import kafka.network.BlockingChannel +import kafka.serializer._ +import kafka.utils.CoreUtils.inLock +import kafka.utils.ZkUtils._ import kafka.utils._ import org.I0Itec.zkclient.exception.ZkNodeExistsException -import java.net.InetAddress -import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, IZkChildListener, ZkClient} +import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient} 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._ -import scala.Some + +import scala.collection._ +import scala.collection.JavaConversions._ /** @@ -42,7 +48,7 @@ import scala.Some * * Directories: * 1. Consumer id registry: - * /consumers/[group_id]/ids[consumer_id] -> topic1,...topicN + * /consumers/[group_id]/ids/[consumer_id] -> topic1,...topicN * A consumer has a unique consumer id within a consumer group. A consumer registers its id as an ephemeral znode * and puts all topics that it subscribes to as the value of the znode. The znode is deleted when the client is gone. * A consumer subscribes to event changes of the consumer id registry within its group. @@ -85,8 +91,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private var fetcher: Option[ConsumerFetcherManager] = None private var zkClient: ZkClient = null private var topicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] - private var checkpointedOffsets = new Pool[TopicAndPartition, Long] - private val topicThreadIdAndQueues = new Pool[(String,String), BlockingQueue[FetchedDataChunk]] + private val checkpointedZkOffsets = new Pool[TopicAndPartition, Long] + private val topicThreadIdAndQueues = new Pool[(String, ConsumerThreadId), BlockingQueue[FetchedDataChunk]] private val scheduler = new KafkaScheduler(threads = 1, threadNamePrefix = "kafka-consumer-scheduler-") private val messageStreamCreated = new AtomicBoolean(false) @@ -94,7 +100,16 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private var topicPartitionChangeListener: ZKTopicPartitionChangeListener = null private var loadBalancerListener: ZKRebalancerListener = null + private var offsetsChannel: BlockingChannel = null + private val offsetsChannelLock = new Object + private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null + private var consumerRebalanceListener: ConsumerRebalanceListener = null + + // useful for tracking migration of consumers to store offsets in kafka + private val kafkaCommitMeter = newMeter("KafkaCommitsPerSec", "commits", TimeUnit.SECONDS, Map("clientId" -> config.clientId)) + private val zkCommitMeter = newMeter("ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS, Map("clientId" -> config.clientId)) + private val rebalanceTimer = new KafkaTimer(newTimer("RebalanceRateAndTime", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, Map("clientId" -> config.clientId))) val consumerIdString = { var consumerUuid : String = null @@ -113,39 +128,49 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, connectZk() createFetcher() + ensureOffsetManagerConnected() + if (config.autoCommitEnable) { scheduler.startup info("starting auto committer every " + config.autoCommitIntervalMs + " ms") - scheduler.schedule("kafka-consumer-autocommit", - autoCommit, + scheduler.schedule("kafka-consumer-autocommit", + autoCommit, delay = config.autoCommitIntervalMs, - period = config.autoCommitIntervalMs, + period = config.autoCommitIntervalMs, unit = TimeUnit.MILLISECONDS) } KafkaMetricsReporter.startReporters(config.props) + AppInfo.registerInfo() def this(config: ConsumerConfig) = this(config, true) - - def createMessageStreams(topicCountMap: Map[String,Int]): Map[String, List[KafkaStream[Array[Byte],Array[Byte]]]] = + + def createMessageStreams(topicCountMap: Map[String,Int]): Map[String, List[KafkaStream[Array[Byte],Array[Byte]]]] = createMessageStreams(topicCountMap, new DefaultDecoder(), new DefaultDecoder()) def createMessageStreams[K,V](topicCountMap: Map[String,Int], keyDecoder: Decoder[K], valueDecoder: Decoder[V]) : Map[String, List[KafkaStream[K,V]]] = { if (messageStreamCreated.getAndSet(true)) - throw new RuntimeException(this.getClass.getSimpleName + - " can create message streams at most once") + throw new MessageStreamsExistException(this.getClass.getSimpleName + + " can create message streams at most once",null) consume(topicCountMap, keyDecoder, valueDecoder) } - def createMessageStreamsByFilter[K,V](topicFilter: TopicFilter, - numStreams: Int, - keyDecoder: Decoder[K] = new DefaultDecoder(), + def createMessageStreamsByFilter[K,V](topicFilter: TopicFilter, + numStreams: Int, + keyDecoder: Decoder[K] = new DefaultDecoder(), valueDecoder: Decoder[V] = new DefaultDecoder()) = { val wildcardStreamsHandler = new WildcardStreamsHandler[K,V](topicFilter, numStreams, keyDecoder, valueDecoder) wildcardStreamsHandler.streams } + def setConsumerRebalanceListener(listener: ConsumerRebalanceListener) { + if (messageStreamCreated.get()) + throw new MessageStreamsExistException(this.getClass.getSimpleName + + " can only set consumer rebalance listener before creating streams",null) + consumerRebalanceListener = listener + } + private def createFetcher() { if (enableFetcher) fetcher = Some(new ConsumerFetcherManager(consumerIdString, config, zkClient)) @@ -153,15 +178,27 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def connectZk() { info("Connecting to zookeeper instance at " + config.zkConnect) - zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) } - def shutdown() { - rebalanceLock synchronized { - val canShutdown = isShuttingDown.compareAndSet(false, true); - if (canShutdown) { - info("ZKConsumerConnector shutting down") + // Blocks until the offset manager is located and a channel is established to it. + private def ensureOffsetManagerConnected() { + if (config.offsetsStorage == "kafka") { + if (offsetsChannel == null || !offsetsChannel.isConnected) + offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient, + config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs) + + debug("Connected to offset manager %s:%d.".format(offsetsChannel.host, offsetsChannel.port)) + } + } + def shutdown() { + val canShutdown = isShuttingDown.compareAndSet(false, true) + if (canShutdown) { + info("ZKConsumerConnector shutting down") + val startTime = System.nanoTime() + KafkaMetricsGroup.removeAllConsumerMetrics(config.clientId) + rebalanceLock synchronized { if (wildcardTopicWatcher != null) wildcardTopicWatcher.shutdown() try { @@ -173,16 +210,18 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } sendShutdownToAllQueues() if (config.autoCommitEnable) - commitOffsets() + commitOffsets(true) if (zkClient != null) { zkClient.close() zkClient = null } + + if (offsetsChannel != null) offsetsChannel.disconnect() } catch { case e: Throwable => fatal("error during consumer connector shutdown", e) } - info("ZKConsumerConnector shut down completed") + info("ZKConsumerConnector shutdown completed in " + (System.nanoTime() - startTime) / 1000000 + " ms") } } } @@ -229,7 +268,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def sendShutdownToAllQueues() = { - for (queue <- topicThreadIdAndQueues.values) { + for (queue <- topicThreadIdAndQueues.values.toSet[BlockingQueue[FetchedDataChunk]]) { debug("Clearing up queue") queue.clear() queue.put(ZookeeperConsumerConnector.shutdownCommand) @@ -240,7 +279,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def autoCommit() { trace("auto committing") try { - commitOffsets() + commitOffsets(isAutoCommit = false) } catch { case t: Throwable => @@ -249,30 +288,192 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - def commitOffsets() { - if (zkClient == null) { - error("zk client is null. Cannot commit offsets") - return + def commitOffsetToZooKeeper(topicPartition: TopicAndPartition, offset: Long) { + if (checkpointedZkOffsets.get(topicPartition) != offset) { + val topicDirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic) + updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + topicPartition.partition, offset.toString) + checkpointedZkOffsets.put(topicPartition, offset) + zkCommitMeter.mark() + } + } + + /** + * KAFKA-1743: This method added for backward compatibility. + */ + def commitOffsets { commitOffsets(true) } + + def commitOffsets(isAutoCommit: Boolean) { + + val offsetsToCommit = + immutable.Map(topicRegistry.flatMap { case (topic, partitionTopicInfos) => + partitionTopicInfos.map { case (partition, info) => + TopicAndPartition(info.topic, info.partitionId) -> OffsetAndMetadata(info.getConsumeOffset()) + } + }.toSeq: _*) + + commitOffsets(offsetsToCommit, isAutoCommit) + + } + + def commitOffsets(offsetsToCommit: immutable.Map[TopicAndPartition, OffsetAndMetadata], isAutoCommit: Boolean) { + trace("OffsetMap: %s".format(offsetsToCommit)) + var retriesRemaining = 1 + (if (isAutoCommit) 0 else config.offsetsCommitMaxRetries) // no retries for commits from auto-commit + var done = false + while (!done) { + val committed = offsetsChannelLock synchronized { + // committed when we receive either no error codes or only MetadataTooLarge errors + if (offsetsToCommit.size > 0) { + if (config.offsetsStorage == "zookeeper") { + offsetsToCommit.foreach { case (topicAndPartition, offsetAndMetadata) => + commitOffsetToZooKeeper(topicAndPartition, offsetAndMetadata.offset) + } + true + } else { + val offsetCommitRequest = OffsetCommitRequest(config.groupId, offsetsToCommit, clientId = config.clientId) + ensureOffsetManagerConnected() + try { + kafkaCommitMeter.mark(offsetsToCommit.size) + offsetsChannel.send(offsetCommitRequest) + val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().payload()) + trace("Offset commit response: %s.".format(offsetCommitResponse)) + + val (commitFailed, retryableIfFailed, shouldRefreshCoordinator, errorCount) = { + offsetCommitResponse.commitStatus.foldLeft(false, false, false, 0) { case (folded, (topicPartition, errorCode)) => + + if (errorCode == ErrorMapping.NoError && config.dualCommitEnabled) { + val offset = offsetsToCommit(topicPartition).offset + commitOffsetToZooKeeper(topicPartition, offset) + } + + (folded._1 || // update commitFailed + errorCode != ErrorMapping.NoError, + + folded._2 || // update retryableIfFailed - (only metadata too large is not retryable) + (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.OffsetMetadataTooLargeCode), + + folded._3 || // update shouldRefreshCoordinator + errorCode == ErrorMapping.NotCoordinatorForConsumerCode || + errorCode == ErrorMapping.ConsumerCoordinatorNotAvailableCode, + + // update error count + folded._4 + (if (errorCode != ErrorMapping.NoError) 1 else 0)) + } + } + debug(errorCount + " errors in offset commit response.") + + + if (shouldRefreshCoordinator) { + debug("Could not commit offsets (because offset coordinator has moved or is unavailable).") + offsetsChannel.disconnect() + } + + if (commitFailed && retryableIfFailed) + false + else + true + } + catch { + case t: Throwable => + error("Error while committing offsets.", t) + offsetsChannel.disconnect() + false + } + } + } else { + debug("No updates to offsets since last commit.") + true + } + } + + done = { + retriesRemaining -= 1 + retriesRemaining == 0 || committed + } + + if (!done) { + debug("Retrying offset commit in %d ms".format(config.offsetsChannelBackoffMs)) + Thread.sleep(config.offsetsChannelBackoffMs) + } + } + } + + private def fetchOffsetFromZooKeeper(topicPartition: TopicAndPartition) = { + val dirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic) + val offsetString = readDataMaybeNull(zkClient, dirs.consumerOffsetDir + "/" + topicPartition.partition)._1 + offsetString match { + case Some(offsetStr) => (topicPartition, OffsetMetadataAndError(offsetStr.toLong)) + case None => (topicPartition, OffsetMetadataAndError.NoOffset) } - for ((topic, infos) <- topicRegistry) { - val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) - for (info <- infos.values) { - val newOffset = info.getConsumeOffset - if (newOffset != checkpointedOffsets.get(TopicAndPartition(topic, info.partitionId))) { + } + + private def fetchOffsets(partitions: Seq[TopicAndPartition]) = { + if (partitions.isEmpty) + Some(OffsetFetchResponse(Map.empty)) + else if (config.offsetsStorage == "zookeeper") { + val offsets = partitions.map(fetchOffsetFromZooKeeper) + Some(OffsetFetchResponse(immutable.Map(offsets:_*))) + } else { + val offsetFetchRequest = OffsetFetchRequest(groupId = config.groupId, requestInfo = partitions, clientId = config.clientId) + + var offsetFetchResponseOpt: Option[OffsetFetchResponse] = None + while (!isShuttingDown.get && !offsetFetchResponseOpt.isDefined) { + offsetFetchResponseOpt = offsetsChannelLock synchronized { + ensureOffsetManagerConnected() try { - updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partitionId, newOffset.toString) - checkpointedOffsets.put(TopicAndPartition(topic, info.partitionId), newOffset) - } catch { - case t: Throwable => - // log it and let it go - warn("exception during commitOffsets", t) + offsetsChannel.send(offsetFetchRequest) + val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().payload()) + trace("Offset fetch response: %s.".format(offsetFetchResponse)) + + val (leaderChanged, loadInProgress) = + offsetFetchResponse.requestInfo.foldLeft(false, false) { case(folded, (topicPartition, offsetMetadataAndError)) => + (folded._1 || (offsetMetadataAndError.error == ErrorMapping.NotCoordinatorForConsumerCode), + folded._2 || (offsetMetadataAndError.error == ErrorMapping.OffsetsLoadInProgressCode)) + } + + if (leaderChanged) { + offsetsChannel.disconnect() + debug("Could not fetch offsets (because offset manager has moved).") + None // retry + } + else if (loadInProgress) { + debug("Could not fetch offsets (because offset cache is being loaded).") + None // retry + } + else { + if (config.dualCommitEnabled) { + // if dual-commit is enabled (i.e., if a consumer group is migrating offsets to kafka), then pick the + // maximum between offsets in zookeeper and kafka. + val kafkaOffsets = offsetFetchResponse.requestInfo + val mostRecentOffsets = kafkaOffsets.map { case (topicPartition, kafkaOffset) => + val zkOffset = fetchOffsetFromZooKeeper(topicPartition)._2.offset + val mostRecentOffset = zkOffset.max(kafkaOffset.offset) + (topicPartition, OffsetMetadataAndError(mostRecentOffset, kafkaOffset.metadata, ErrorMapping.NoError)) + } + Some(OffsetFetchResponse(mostRecentOffsets)) + } + else + Some(offsetFetchResponse) + } } - debug("Committed offset " + newOffset + " for topic " + info) + catch { + case e: Exception => + warn("Error while fetching offsets from %s:%d. Possible cause: %s".format(offsetsChannel.host, offsetsChannel.port, e.getMessage)) + offsetsChannel.disconnect() + None // retry + } + } + + if (offsetFetchResponseOpt.isEmpty) { + debug("Retrying offset fetch in %d ms".format(config.offsetsChannelBackoffMs)) + Thread.sleep(config.offsetsChannelBackoffMs) } } + + offsetFetchResponseOpt } } + class ZKSessionExpireListener(val dirs: ZKGroupDirs, val consumerIdString: String, val topicCount: TopicCount, @@ -306,6 +507,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, // The child change watchers will be set inside rebalance when we read the children list. } + override def handleSessionEstablishmentError(error: Throwable): Unit = { + fatal("Could not establish session with zookeeper", error) + } } class ZKTopicPartitionChangeListener(val loadBalancerListener: ZKRebalancerListener) @@ -333,9 +537,22 @@ 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 partitionAssignor = PartitionAssignor.createInstance(config.partitionAssignmentStrategy) + private var isWatcherTriggered = false private val lock = new ReentrantLock private val cond = lock.newCondition() + + @volatile private var allTopicsOwnedPartitionsCount = 0 + newGauge("OwnedPartitionsCount", + new Gauge[Int] { + def value() = allTopicsOwnedPartitionsCount + }, + Map("clientId" -> config.clientId, "groupId" -> config.groupId)) + + private def ownedPartitionsCountMetricTags(topic: String) = Map("clientId" -> config.clientId, "groupId" -> config.groupId, "topic" -> topic) + private val watcherExecutorThread = new Thread(consumerIdString + "_watcher_executor") { override def run() { info("starting watcher executor thread for consumer " + consumerIdString) @@ -384,10 +601,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def releasePartitionOwnership(localTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]])= { info("Releasing partition ownership") for ((topic, infos) <- localTopicRegistry) { - for(partition <- infos.keys) + for(partition <- infos.keys) { deletePartitionOwnershipFromZK(topic, partition) + } + removeMetric("OwnedPartitionsCount", ownedPartitionsCountMetricTags(topic)) localTopicRegistry.remove(topic) } + allTopicsOwnedPartitionsCount = 0 } def resetState() { @@ -396,10 +616,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def syncedRebalance() { rebalanceLock synchronized { - if(isShuttingDown.get()) { - return - } else { + rebalanceTimer.time { for (i <- 0 until config.rebalanceMaxRetries) { + if(isShuttingDown.get()) { + return + } info("begin rebalancing consumer " + consumerIdString + " try #" + i) var done = false var cluster: Cluster = null @@ -409,9 +630,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } catch { case e: Throwable => /** occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating. - * For example, a ZK node can disappear between the time we get all children and the time we try to get - * the value of a child. Just let this go since another rebalance will be triggered. - **/ + * For example, a ZK node can disappear between the time we get all children and the time we try to get + * the value of a child. Just let this go since another rebalance will be triggered. + **/ info("exception during rebalance ", e) } info("end rebalancing consumer " + consumerIdString + " try #" + i) @@ -433,8 +654,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def rebalance(cluster: Cluster): Boolean = { - val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic - val consumersPerTopicMap = getConsumersPerTopic(zkClient, group) + val myTopicThreadIdsMap = TopicCount.constructTopicCount( + group, consumerIdString, zkClient, config.excludeInternalTopics).getConsumerThreadIdsPerTopic val brokers = getAllBrokersInCluster(zkClient) if (brokers.size == 0) { // This can happen in a rare case when there are no brokers available in the cluster when the consumer is started. @@ -445,9 +666,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, true } else { - val partitionsAssignmentPerTopicMap = getPartitionAssignmentForTopics(zkClient, myTopicThreadIdsMap.keySet.toSeq) - val partitionsPerTopicMap = partitionsAssignmentPerTopicMap.map(p => (p._1, p._2.keySet.toSeq.sorted)) - /** * fetchers must be stopped to avoid data duplication, since if the current * rebalancing attempt fails, the partitions that are released could be owned by another consumer. @@ -455,63 +673,87 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * partitions in parallel. So, not stopping the fetchers leads to duplicate data. */ closeFetchers(cluster, kafkaMessageAndMetadataStreams, myTopicThreadIdsMap) - + if (consumerRebalanceListener != null) { + info("Invoking rebalance listener before relasing partition ownerships.") + consumerRebalanceListener.beforeReleasingPartitions( + if (topicRegistry.size == 0) + new java.util.HashMap[String, java.util.Set[java.lang.Integer]] + else + mapAsJavaMap(topicRegistry.map(topics => + topics._1 -> topics._2.keys + ).toMap).asInstanceOf[java.util.Map[String, java.util.Set[java.lang.Integer]]] + ) + } releasePartitionOwnership(topicRegistry) + val assignmentContext = new AssignmentContext(group, consumerIdString, config.excludeInternalTopics, zkClient) + val globalPartitionAssignment = partitionAssignor.assign(assignmentContext) + val partitionAssignment = globalPartitionAssignment.get(assignmentContext.consumerId) + val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]( + valueFactory = Some((topic: String) => new Pool[Int, PartitionTopicInfo])) - var partitionOwnershipDecision = new collection.mutable.HashMap[(String, Int), String]() - val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] - - for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) { - currentTopicRegistry.put(topic, new Pool[Int, PartitionTopicInfo]) + // fetch current offsets for all topic-partitions + val topicPartitions = partitionAssignment.keySet.toSeq - val topicDirs = new ZKGroupTopicDirs(group, topic) - val curConsumers = consumersPerTopicMap.get(topic).get - val curPartitions: Seq[Int] = partitionsPerTopicMap.get(topic).get + val offsetFetchResponseOpt = fetchOffsets(topicPartitions) - val nPartsPerConsumer = curPartitions.size / curConsumers.size - val nConsumersWithExtraPart = curPartitions.size % curConsumers.size + if (isShuttingDown.get || !offsetFetchResponseOpt.isDefined) + false + else { + val offsetFetchResponse = offsetFetchResponseOpt.get + topicPartitions.foreach(topicAndPartition => { + val (topic, partition) = topicAndPartition.asTuple + val offset = offsetFetchResponse.requestInfo(topicAndPartition).offset + val threadId = partitionAssignment(topicAndPartition) + addPartitionTopicInfo(currentTopicRegistry, partition, topic, offset, threadId) + }) - info("Consumer " + consumerIdString + " rebalancing the following partitions: " + curPartitions + - " for topic " + topic + " with consumers: " + curConsumers) + /** + * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt + * A rebalancing attempt is completed successfully only after the fetchers have been started correctly + */ + if(reflectPartitionOwnershipDecision(partitionAssignment)) { + allTopicsOwnedPartitionsCount = partitionAssignment.size + + partitionAssignment.view.groupBy { case(topicPartition, consumerThreadId) => topicPartition.topic } + .foreach { case (topic, partitionThreadPairs) => + newGauge("OwnedPartitionsCount", + new Gauge[Int] { + def value() = partitionThreadPairs.size + }, + ownedPartitionsCountMetricTags(topic)) + } - for (consumerThreadId <- consumerThreadIdSet) { - val myConsumerPosition = curConsumers.indexOf(consumerThreadId) - assert(myConsumerPosition >= 0) - val startPart = nPartsPerConsumer*myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) - val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1) + topicRegistry = currentTopicRegistry + // Invoke beforeStartingFetchers callback if the consumerRebalanceListener is set. + if (consumerRebalanceListener != null) { + info("Invoking rebalance listener before starting fetchers.") - /** - * Range-partition the sorted partitions to consumers for better locality. - * The first few consumers pick up an extra partition, if any. - */ - if (nParts <= 0) - warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic) - else { - for (i <- startPart until startPart + nParts) { - val partition = curPartitions(i) - info(consumerThreadId + " attempting to claim partition " + partition) - addPartitionTopicInfo(currentTopicRegistry, topicDirs, partition, topic, consumerThreadId) - // record the partition ownership decision - partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) + // Partition assignor returns the global partition assignment organized as a map of [TopicPartition, ThreadId] + // per consumer, and we need to re-organize it to a map of [Partition, ThreadId] per topic before passing + // to the rebalance callback. + val partitionAssginmentGroupByTopic = globalPartitionAssignment.values.flatten.groupBy[String] { + case (topicPartition, _) => topicPartition.topic } + val partitionAssigmentMapForCallback = partitionAssginmentGroupByTopic.map({ + case (topic, partitionOwnerShips) => + val partitionOwnershipForTopicScalaMap = partitionOwnerShips.map({ + case (topicAndPartition, consumerThreadId) => + topicAndPartition.partition -> consumerThreadId + }) + topic -> mapAsJavaMap(collection.mutable.Map(partitionOwnershipForTopicScalaMap.toSeq:_*)) + .asInstanceOf[java.util.Map[java.lang.Integer, ConsumerThreadId]] + }) + consumerRebalanceListener.beforeStartingFetchers( + consumerIdString, + mapAsJavaMap(collection.mutable.Map(partitionAssigmentMapForCallback.toSeq:_*)) + ) } + updateFetcher(cluster) + true + } else { + false } } - - /** - * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt - * A rebalancing attempt is completed successfully only after the fetchers have been started correctly - */ - if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) { - info("Updating the cache") - debug("Partitions per topic cache " + partitionsPerTopicMap) - debug("Consumers per topic cache " + consumersPerTopicMap) - topicRegistry = currentTopicRegistry - updateFetcher(cluster) - true - } else { - false - } } } @@ -523,7 +765,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, case Some(f) => f.stopConnections clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, messageStreams) - info("Committing all offsets after clearing the fetcher queues") /** * here, we need to commit offsets before stopping the consumer from returning any more messages * from the current data chunk. Since partition ownership is not yet released, this commit offsets @@ -532,8 +773,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * by the consumer, there will be no more messages returned by this iterator until the rebalancing finishes * successfully and the fetchers restart to fetch more data chunks **/ - if (config.autoCommitEnable) - commitOffsets + if (config.autoCommitEnable) { + info("Committing all offsets after clearing the fetcher queues") + commitOffsets(true) + } case None => } } @@ -555,7 +798,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def closeFetchers(cluster: Cluster, messageStreams: Map[String,List[KafkaStream[_,_]]], - relevantTopicThreadIdsMap: Map[String, Set[String]]) { + relevantTopicThreadIdsMap: Map[String, Set[ConsumerThreadId]]) { // only clear the fetcher queues for certain topic partitions that *might* no longer be served by this consumer // after this rebalancing attempt val queuesTobeCleared = topicThreadIdAndQueues.filter(q => relevantTopicThreadIdsMap.contains(q._1._1)).map(q => q._2) @@ -578,15 +821,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, Int), String]): Boolean = { + private def reflectPartitionOwnershipDecision(partitionAssignment: Map[TopicAndPartition, ConsumerThreadId]): Boolean = { var successfullyOwnedPartitions : List[(String, Int)] = Nil - val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner => - val topic = partitionOwner._1._1 - val partition = partitionOwner._1._2 + val partitionOwnershipSuccessful = partitionAssignment.map { partitionOwner => + val topic = partitionOwner._1.topic + val partition = partitionOwner._1.partition val consumerThreadId = partitionOwner._2 val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic, partition) try { - createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId) + createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId.toString) info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic) successfullyOwnedPartitions ::= (topic, partition) true @@ -609,18 +852,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]], - topicDirs: ZKGroupTopicDirs, partition: Int, - topic: String, consumerThreadId: String) { - val partTopicInfoMap = currentTopicRegistry.get(topic) - - val znode = topicDirs.consumerOffsetDir + "/" + partition - val offsetString = readDataMaybeNull(zkClient, znode)._1 - // If first time starting a consumer, set the initial offset to -1 - val offset = - offsetString match { - case Some(offsetStr) => offsetStr.toLong - case None => PartitionTopicInfo.InvalidOffset - } + partition: Int, topic: String, + offset: Long, consumerThreadId: ConsumerThreadId) { + val partTopicInfoMap = currentTopicRegistry.getAndMaybePut(topic) + val queue = topicThreadIdAndQueues.get((topic, consumerThreadId)) val consumedOffset = new AtomicLong(offset) val fetchedOffset = new AtomicLong(offset) @@ -633,7 +868,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, config.clientId) partTopicInfoMap.put(partition, partTopicInfo) debug(partTopicInfo + " selected new offset " + offset) - checkpointedOffsets.put(TopicAndPartition(topic, partition), offset) + checkpointedZkOffsets.put(TopicAndPartition(topic, partition), offset) } } @@ -662,7 +897,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val topicStreamsMap = loadBalancerListener.kafkaMessageAndMetadataStreams // map of {topic -> Set(thread-1, thread-2, ...)} - val consumerThreadIdsPerTopic: Map[String, Set[String]] = + val consumerThreadIdsPerTopic: Map[String, Set[ConsumerThreadId]] = topicCount.getConsumerThreadIdsPerTopic val allQueuesAndStreams = topicCount match { @@ -692,10 +927,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, topicThreadIdAndQueues.put(topicThreadId, q) debug("Adding topicThreadId %s and queue %s to topicThreadIdAndQueues data structure".format(topicThreadId, q.toString)) newGauge( - config.clientId + "-" + config.groupId + "-" + topicThreadId._1 + "-" + topicThreadId._2 + "-FetchQueueSize", + "FetchQueueSize", new Gauge[Int] { def value = q.size - } + }, + Map("clientId" -> config.clientId, + "topic" -> topicThreadId._1, + "threadId" -> topicThreadId._2.threadId.toString) ) }) @@ -735,10 +973,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private val wildcardQueuesAndStreams = (1 to numStreams) .map(e => { val queue = new LinkedBlockingQueue[FetchedDataChunk](config.queuedMaxMessages) - val stream = new KafkaStream[K,V](queue, - config.consumerTimeoutMs, - keyDecoder, - valueDecoder, + val stream = new KafkaStream[K,V](queue, + config.consumerTimeoutMs, + keyDecoder, + valueDecoder, config.clientId) (queue, stream) }).toList @@ -746,10 +984,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, // bootstrap with existing topics private var wildcardTopics = getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) - .filter(topicFilter.isTopicAllowed) + .filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics)) private val wildcardTopicCount = TopicCount.constructTopicCount( - consumerIdString, topicFilter, numStreams, zkClient) + consumerIdString, topicFilter, numStreams, zkClient, config.excludeInternalTopics) val dirs = new ZKGroupDirs(config.groupId) registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount) @@ -764,7 +1002,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def handleTopicEvent(allTopics: Seq[String]) { debug("Handling topic event") - val updatedTopics = allTopics.filter(topicFilter.isTopicAllowed) + val updatedTopics = allTopics.filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics)) val addedTopics = updatedTopics filterNot (wildcardTopics contains) if (addedTopics.nonEmpty) @@ -792,4 +1030,3 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, wildcardQueuesAndStreams.map(_._2) } } - diff --git a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala index 38f4ec0bd1b38..f74823b514798 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala @@ -18,7 +18,7 @@ package kafka.consumer import scala.collection.JavaConversions._ -import kafka.utils.{ZkUtils, ZKStringSerializer, Logging} +import kafka.utils.{ZkUtils, Logging} import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} import org.apache.zookeeper.Watcher.Event.KeeperState @@ -93,6 +93,10 @@ class ZookeeperTopicEventWatcher(val zkClient: ZkClient, } } } + + override def handleSessionEstablishmentError(error: Throwable): Unit = { + //no-op ZookeeperConsumerConnector should log error. + } } } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala old mode 100644 new mode 100755 index a1ee5a7074121..9f521fa7a54a8 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -16,16 +16,15 @@ */ package kafka.controller -import kafka.network.{Receive, BlockingChannel} -import kafka.utils.{Utils, Logging, ShutdownableThread} +import kafka.network.BlockingChannel +import kafka.utils.{CoreUtils, Logging, ShutdownableThread} +import org.apache.kafka.common.network.NetworkReceive import collection.mutable.HashMap import kafka.cluster.Broker import java.util.concurrent.{LinkedBlockingQueue, BlockingQueue} import kafka.server.KafkaConfig import collection.mutable import kafka.api._ -import org.apache.log4j.Logger -import scala.Some import kafka.common.TopicAndPartition import kafka.api.RequestOrResponse import collection.Set @@ -78,9 +77,10 @@ class ControllerChannelManager (private val controllerContext: ControllerContext } private def addNewBroker(broker: Broker) { - val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize) + val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)]() debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id)) - val channel = new BlockingChannel(broker.host, broker.port, + val brokerEndPoint = broker.getBrokerEndPoint(config.interBrokerSecurityProtocol) + val channel = new BlockingChannel(brokerEndPoint.host, brokerEndPoint.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) @@ -114,55 +114,57 @@ class RequestSendThread(val controllerId: Int, val channel: BlockingChannel) 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) + private val stateChangeLogger = KafkaController.stateChangeLogger connectToBroker(toBroker, channel) override def doWork(): Unit = { val queueItem = queue.take() val request = queueItem._1 val callback = queueItem._2 - var receive: Receive = null + var receive: NetworkReceive = null try { lock synchronized { var isSendSuccessful = false - while(isRunning.get() && !isSendSuccessful) { + 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) + receive = channel.receive() 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. " + + case e: Throwable => // if the send was not successful, reconnect to broker and resend the message + warn(("Controller %d epoch %d fails to send request %s to broker %s. " + "Reconnecting to broker.").format(controllerId, controllerContext.epoch, - RequestKeys.nameForKey(request.requestId.get), request.correlationId, toBroker.toString()), e) + request.toString, toBroker.toString()), e) channel.disconnect() connectToBroker(toBroker, channel) isSendSuccessful = false // backoff before retrying the connection and send - Utils.swallow(Thread.sleep(300)) + CoreUtils.swallowTrace(Thread.sleep(300)) } } - receive = channel.receive() - var response: RequestOrResponse = null - request.requestId.get match { - case RequestKeys.LeaderAndIsrKey => - response = LeaderAndIsrResponse.readFrom(receive.buffer) - case RequestKeys.StopReplicaKey => - response = StopReplicaResponse.readFrom(receive.buffer) - case RequestKeys.UpdateMetadataKey => - response = UpdateMetadataResponse.readFrom(receive.buffer) - } - stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %s" - .format(controllerId, controllerContext.epoch, response.correlationId, toBroker.toString())) + if (receive != null) { + var response: RequestOrResponse = null + request.requestId.get match { + case RequestKeys.LeaderAndIsrKey => + response = LeaderAndIsrResponse.readFrom(receive.payload()) + case RequestKeys.StopReplicaKey => + response = StopReplicaResponse.readFrom(receive.payload()) + case RequestKeys.UpdateMetadataKey => + response = UpdateMetadataResponse.readFrom(receive.payload()) + } + stateChangeLogger.trace("Controller %d epoch %d received response %s for a request sent to broker %s" + .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString)) - if(callback != null) { - callback(response) + if (callback != null) { + callback(response) + } } } } catch { case e: Throwable => - warn("Controller %d fails to send a request to broker %s".format(controllerId, toBroker.toString()), e) + error("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() } @@ -173,7 +175,7 @@ class RequestSendThread(val controllerId: Int, channel.connect() info("Controller %d connected to %s for sending state change requests".format(controllerId, broker.toString())) } catch { - case e => { + case e: Throwable => { channel.disconnect() error("Controller %d's connection to broker %s was unsuccessful".format(controllerId, broker.toString()), e) } @@ -188,7 +190,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val leaderAndIsrRequestMap = new mutable.HashMap[Int, mutable.HashMap[(String, Int), PartitionStateInfo]] val stopReplicaRequestMap = new mutable.HashMap[Int, Seq[StopReplicaRequestInfo]] val updateMetadataRequestMap = new mutable.HashMap[Int, mutable.HashMap[TopicAndPartition, PartitionStateInfo]] - private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) + private val stateChangeLogger = KafkaController.stateChangeLogger def newBatch() { // raise error if the previous batch is not empty @@ -206,12 +208,17 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, replicas: Seq[Int], callback: (RequestOrResponse) => Unit = null) { - brokerIds.filter(b => b >= 0).foreach { brokerId => - leaderAndIsrRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[(String, Int), PartitionStateInfo]) - leaderAndIsrRequestMap(brokerId).put((topic, partition), - PartitionStateInfo(leaderIsrAndControllerEpoch, replicas.toSet)) + val topicAndPartition: TopicAndPartition = TopicAndPartition(topic, partition) + + brokerIds.filter(b => b >= 0).foreach { + brokerId => + leaderAndIsrRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[(String, Int), PartitionStateInfo]) + leaderAndIsrRequestMap(brokerId).put((topic, partition), + PartitionStateInfo(leaderIsrAndControllerEpoch, replicas.toSet)) } - addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq) + + addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, + Set(topicAndPartition)) } def addStopReplicaRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, deletePartition: Boolean, @@ -228,27 +235,48 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging } } - /* Send UpdateMetadataRequest to the given brokers for all partitions except those being deleted as part of delete topic - * - */ + /** Send UpdateMetadataRequest to the given brokers for the given partitions and partitions that are being deleted */ def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int], + partitions: collection.Set[TopicAndPartition] = Set.empty[TopicAndPartition], callback: (RequestOrResponse) => Unit = null) { - val partitionList = controllerContext.partitionLeadershipInfo.keySet.dropWhile( - p => controller.deleteTopicManager.isTopicQueuedUpForDeletion(p.topic)) - partitionList.foreach { partition => + def updateMetadataRequestMapFor(partition: TopicAndPartition, beingDeleted: Boolean) { val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition) leaderIsrAndControllerEpochOpt match { case Some(leaderIsrAndControllerEpoch) => val replicas = controllerContext.partitionReplicaAssignment(partition).toSet - val partitionStateInfo = PartitionStateInfo(leaderIsrAndControllerEpoch, replicas) + val partitionStateInfo = if (beingDeleted) { + val leaderAndIsr = new LeaderAndIsr(LeaderAndIsr.LeaderDuringDelete, leaderIsrAndControllerEpoch.leaderAndIsr.isr) + PartitionStateInfo(LeaderIsrAndControllerEpoch(leaderAndIsr, leaderIsrAndControllerEpoch.controllerEpoch), replicas) + } else { + PartitionStateInfo(leaderIsrAndControllerEpoch, replicas) + } brokerIds.filter(b => b >= 0).foreach { brokerId => updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo]) updateMetadataRequestMap(brokerId).put(partition, partitionStateInfo) } case None => - info("Leader not assigned yet for partition %s. Skip sending udpate metadata request".format(partition)) + info("Leader not yet assigned for partition %s. Skip sending UpdateMetadataRequest.".format(partition)) } } + + val filteredPartitions = { + val givenPartitions = if (partitions.isEmpty) + controllerContext.partitionLeadershipInfo.keySet + else + partitions + if (controller.deleteTopicManager.partitionsToBeDeleted.isEmpty) + givenPartitions + else + givenPartitions -- controller.deleteTopicManager.partitionsToBeDeleted + } + if(filteredPartitions.isEmpty) + brokerIds.filter(b => b >= 0).foreach { brokerId => + updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo]) + } + else + filteredPartitions.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = false)) + + controller.deleteTopicManager.partitionsToBeDeleted.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = true)) } def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int) { @@ -256,7 +284,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val broker = m._1 val partitionStateInfos = m._2.toMap val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet - val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)) + val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map(b => b.getBrokerEndPoint(controller.config.interBrokerSecurityProtocol)) 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" @@ -271,17 +299,19 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging updateMetadataRequestMap.foreach { m => val broker = m._1 val partitionStateInfos = m._2.toMap - val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, correlationId, clientId, - partitionStateInfos, controllerContext.liveOrShuttingDownBrokers) + + val versionId = if (controller.config.interBrokerProtocolVersion.onOrAfter(KAFKA_083)) 1 else 0 + val updateMetadataRequest = new UpdateMetadataRequest(versionId = versionId.toShort, controllerId = controllerId, controllerEpoch = controllerEpoch, + correlationId = correlationId, clientId = clientId, partitionStateInfos = partitionStateInfos, aliveBrokers = controllerContext.liveOrShuttingDownBrokers) 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))) + correlationId, broker, p._1))) controller.sendRequest(broker, updateMetadataRequest, null) } updateMetadataRequestMap.clear() stopReplicaRequestMap foreach { case(broker, replicaInfoList) => - val stopReplicaWithDelete = replicaInfoList.filter(p => p.deletePartition == true).map(i => i.replica).toSet - val stopReplicaWithoutDelete = replicaInfoList.filter(p => p.deletePartition == false).map(i => i.replica).toSet + val stopReplicaWithDelete = replicaInfoList.filter(_.deletePartition).map(_.replica).toSet + val stopReplicaWithoutDelete = replicaInfoList.filterNot(_.deletePartition).map(_.replica).toSet debug("The stop replica request (delete = true) sent to broker %d is %s" .format(broker, stopReplicaWithDelete.mkString(","))) debug("The stop replica request (delete = false) sent to broker %d is %s" diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala old mode 100644 new mode 100755 index d812cb4121d7f..b4fc755641b9b --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -16,28 +16,28 @@ */ package kafka.controller -import collection._ -import collection.Set +import java.util + +import scala.collection._ import com.yammer.metrics.core.Gauge -import java.lang.{IllegalStateException, Object} import java.util.concurrent.TimeUnit +import kafka.admin.AdminUtils import kafka.admin.PreferredReplicaLeaderElectionCommand import kafka.api._ import kafka.cluster.Broker import kafka.common._ +import kafka.log.LogConfig import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} -import kafka.server.{ZookeeperLeaderElector, KafkaConfig} import kafka.utils.ZkUtils._ import kafka.utils._ -import kafka.utils.Utils._ +import kafka.utils.CoreUtils._ import org.apache.zookeeper.Watcher.Event.KeeperState -import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} +import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} import java.util.concurrent.atomic.AtomicInteger -import org.apache.log4j.Logger -import scala.Some -import kafka.common.TopicAndPartition import java.util.concurrent.locks.ReentrantLock +import kafka.server._ +import kafka.common.TopicAndPartition class ControllerContext(val zkClient: ZkClient, val zkSessionTimeout: Int) { @@ -113,22 +113,20 @@ class ControllerContext(val zkClient: ZkClient, } def removeTopic(topic: String) = { - partitionLeadershipInfo = partitionLeadershipInfo.dropWhile(p => p._1.topic.equals(topic)) - partitionReplicaAssignment = partitionReplicaAssignment.dropWhile(p => p._1.topic.equals(topic)) + partitionLeadershipInfo = partitionLeadershipInfo.filter{ case (topicAndPartition, _) => topicAndPartition.topic != topic } + partitionReplicaAssignment = partitionReplicaAssignment.filter{ case (topicAndPartition, _) => topicAndPartition.topic != topic } allTopics -= topic } } -trait KafkaControllerMBean { - def shutdownBroker(id: Int): Set[TopicAndPartition] -} object KafkaController extends Logging { - val MBeanName = "kafka.controller:type=KafkaController,name=ControllerOps" - val stateChangeLogger = "state.change.logger" + val stateChangeLogger = new StateChangeLogger("state.change.logger") val InitialControllerEpoch = 1 val InitialControllerEpochZkVersion = 1 + case class StateChangeLogger(override val loggerName: String) extends Logging + def parseControllerId(controllerInfoString: String): Int = { try { Json.parseFull(controllerInfoString) match { @@ -151,10 +149,10 @@ object KafkaController extends Logging { } } -class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logging with KafkaMetricsGroup with KafkaControllerMBean { +class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerState: BrokerState) extends Logging with KafkaMetricsGroup { this.logIdent = "[Controller " + config.brokerId + "]: " private var isRunning = true - private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) + private val stateChangeLogger = KafkaController.stateChangeLogger val controllerContext = new ControllerContext(zkClient, config.zkSessionTimeoutMs) val partitionStateMachine = new PartitionStateMachine(this) val replicaStateMachine = new ReplicaStateMachine(this) @@ -164,12 +162,15 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // kafka server private val autoRebalanceScheduler = new KafkaScheduler(1) var deleteTopicManager: TopicDeletionManager = null - val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext) + val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext, config) private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext) private val brokerRequestBatch = new ControllerBrokerRequestBatch(this) - registerControllerChangedListener() + + private val partitionReassignedListener = new PartitionsReassignedListener(this) + private val preferredReplicaElectionListener = new PreferredReplicaElectionListener(this) + private val isrChangeNotificationListener = new IsrChangeNotificationListener(this) newGauge( "ActiveControllerCount", @@ -210,7 +211,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg def epoch = controllerContext.epoch - def clientId = "id_%d-host_%s-port_%d".format(config.brokerId, config.hostName, config.port) + def clientId = { + val listeners = config.listeners + val controllerListener = listeners.get(config.interBrokerSecurityProtocol) + "id_%d-host_%s-port_%d".format(config.brokerId, controllerListener.get.host, controllerListener.get.port) + } /** * On clean shutdown, the controller first determines the partitions that the @@ -234,7 +239,6 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg throw new BrokerNotAvailableException("Broker id %d does not exist.".format(id)) controllerContext.shuttingDownBrokerIds.add(id) - debug("All shutting down brokers: " + controllerContext.shuttingDownBrokerIds.mkString(",")) debug("Live brokers: " + controllerContext.liveBrokerIds.mkString(",")) } @@ -247,31 +251,31 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg allPartitionsAndReplicationFactorOnBroker.foreach { case(topicAndPartition, replicationFactor) => - // Move leadership serially to relinquish lock. - inLock(controllerContext.controllerLock) { - controllerContext.partitionLeadershipInfo.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch => - if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id) { - // If the broker leads the topic partition, transition the leader and update isr. Updates zk and - // notifies all affected brokers - partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, - controlledShutdownPartitionLeaderSelector) - } - else { - // Stop the replica first. The state change below initiates ZK changes which should take some time - // before which the stop replica request should be completed (in most cases) - brokerRequestBatch.newBatch() - brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, - topicAndPartition.partition, deletePartition = false) - brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) - - // If the broker is a follower, updates the isr in ZK and notifies the current leader - replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic, - topicAndPartition.partition, id)), OfflineReplica) + // Move leadership serially to relinquish lock. + inLock(controllerContext.controllerLock) { + controllerContext.partitionLeadershipInfo.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch => + if (replicationFactor > 1) { + if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id) { + // If the broker leads the topic partition, transition the leader and update isr. Updates zk and + // notifies all affected brokers + partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, + controlledShutdownPartitionLeaderSelector) + } else { + // Stop the replica first. The state change below initiates ZK changes which should take some time + // before which the stop replica request should be completed (in most cases) + brokerRequestBatch.newBatch() + brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, + topicAndPartition.partition, deletePartition = false) + brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) + + // If the broker is a follower, updates the isr in ZK and notifies the current leader + replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic, + topicAndPartition.partition, id)), OfflineReplica) + } + } } } - } } - def replicatedPartitionsBrokerLeads() = inLock(controllerContext.controllerLock) { trace("All leaders = " + controllerContext.partitionLeadershipInfo.mkString(",")) controllerContext.partitionLeadershipInfo.filter { @@ -299,10 +303,13 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg def onControllerFailover() { if(isRunning) { info("Broker %d starting become controller state transition".format(config.brokerId)) + //read controller epoch from zk + readControllerEpochFromZookeeper() // increment the controller epoch incrementControllerEpoch(zkClient) // before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks registerReassignedPartitionsListener() + registerIsrChangeNotificationListener() registerPreferredReplicaElectionListener() partitionStateMachine.registerListeners() replicaStateMachine.registerListeners() @@ -311,8 +318,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg partitionStateMachine.startup() // register the partition change listeners for all existing topics on failover controllerContext.allTopics.foreach(topic => partitionStateMachine.registerPartitionChangeListener(topic)) - Utils.registerMBean(this, KafkaController.MBeanName) info("Broker %d is ready to serve as the new controller with epoch %d".format(config.brokerId, epoch)) + brokerState.newState(RunningAsController) maybeTriggerPartitionReassignment() maybeTriggerPreferredReplicaElection() /* send partition leadership info to all live brokers */ @@ -321,7 +328,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("starting the partition rebalance scheduler") autoRebalanceScheduler.startup() autoRebalanceScheduler.schedule("partition-rebalance-thread", checkAndTriggerPartitionRebalance, - 5, config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS) + 5, config.leaderImbalanceCheckIntervalSeconds.toLong, TimeUnit.SECONDS) } deleteTopicManager.start() } @@ -334,16 +341,37 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * required to clean up internal controller data structures */ def onControllerResignation() { - inLock(controllerContext.controllerLock) { - autoRebalanceScheduler.shutdown() + // de-register listeners + deregisterIsrChangeNotificationListener() + deregisterReassignedPartitionsListener() + deregisterPreferredReplicaElectionListener() + + // shutdown delete topic manager + if (deleteTopicManager != null) deleteTopicManager.shutdown() - Utils.unregisterMBean(KafkaController.MBeanName) + + // shutdown leader rebalance scheduler + if (config.autoLeaderRebalanceEnable) + autoRebalanceScheduler.shutdown() + + inLock(controllerContext.controllerLock) { + // de-register partition ISR listener for on-going partition reassignment task + deregisterReassignedPartitionsIsrChangeListeners() + // shutdown partition state machine partitionStateMachine.shutdown() + // shutdown replica state machine replicaStateMachine.shutdown() + // shutdown controller channel manager if(controllerContext.controllerChannelManager != null) { controllerContext.controllerChannelManager.shutdown() controllerContext.controllerChannelManager = null } + // reset controller context + controllerContext.epoch=0 + controllerContext.epochZkVersion=0 + brokerState.newState(RunningAsBroker) + + info("Broker %d resigned as the controller".format(config.brokerId)) } } @@ -359,8 +387,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg /** * This callback is invoked by the replica state machine's broker change listener, with the list of newly started * brokers as input. It does the following - - * 1. Triggers the OnlinePartition state change for all new/offline partitions - * 2. It checks whether there are reassigned replicas assigned to any newly started brokers. If + * 1. Sends update metadata request to all live and shutting down brokers + * 2. Triggers the OnlinePartition state change for all new/offline partitions + * 3. It checks whether there are reassigned replicas assigned to any newly started brokers. If * so, it performs the reassignment logic for each topic/partition. * * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point for two reasons: @@ -372,10 +401,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg def onBrokerStartup(newBrokers: Seq[Int]) { info("New broker startup callback for %s".format(newBrokers.mkString(","))) val newBrokersSet = newBrokers.toSet - // send update metadata request for all partitions to the newly restarted brokers. In cases of controlled shutdown - // leaders will not be elected when a new broker comes up. So at least in the common controlled shutdown case, the - // metadata will reach the new brokers faster - sendUpdateMetadataRequest(newBrokers) + // send update metadata request to all live and shutting down brokers. Old brokers will get to know of the new + // broker via this update. + // In cases of controlled shutdown leaders will not be elected when a new broker comes up. So at least in the + // common controlled shutdown case, the metadata will reach the new brokers faster + sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) // 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 val allReplicasOnNewBrokers = controllerContext.replicasOnBrokers(newBrokersSet) @@ -405,6 +435,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * 1. Mark partitions with dead leaders as offline * 2. Triggers the OnlinePartition state change for all new/offline partitions * 3. Invokes the OfflineReplica state change on the input list of newly started brokers + * 4. If no partitions are effected then send UpdateMetadataRequest to live or shutting down brokers * * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point. This is because * the partition state machine will refresh our cache for us when performing leader election for all new/offline @@ -433,9 +464,15 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg if(replicasForTopicsToBeDeleted.size > 0) { // it is required to mark the respective replicas in TopicDeletionFailed state since the replica cannot be // deleted when the broker is down. This will prevent the replica from being in TopicDeletionStarted state indefinitely - // since topic deletion cannot be retried if at least one replica is in TopicDeletionStarted state + // since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state deleteTopicManager.failReplicaDeletion(replicasForTopicsToBeDeleted) } + + // If broker failure did not require leader re-election, inform brokers of failed broker + // Note that during leader re-election, brokers update their metadata + if (partitionsWithoutLeader.isEmpty) { + sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) + } } /** @@ -443,6 +480,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * 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 + * 3. Send metadata request with the new topic to all brokers so they allow requests for that topic to be served */ def onNewTopicCreation(topics: Set[String], newPartitions: Set[TopicAndPartition]) { info("New topic creation callback for %s".format(newPartitions.mkString(","))) @@ -545,7 +583,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("Removed partition %s from the list of reassigned partitions in zookeeper".format(topicAndPartition)) controllerContext.partitionsBeingReassigned.remove(topicAndPartition) //12. After electing leader, the replicas and isr information changes, so resend the update metadata request to every broker - sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) + sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicAndPartition)) // signal delete topic thread if reassignment for some partitions belonging to topics being deleted just completed deleteTopicManager.resumeDeletionForTopics(Set(topicAndPartition.topic)) } @@ -581,8 +619,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // first register ISR change listener watchIsrChangesForReassignedPartition(topic, partition, reassignedPartitionContext) controllerContext.partitionsBeingReassigned.put(topicAndPartition, reassignedPartitionContext) - // halt topic deletion for the partitions being reassigned - deleteTopicManager.haltTopicDeletion(Set(topic)) + // mark topic ineligible for deletion for the partitions being reassigned + deleteTopicManager.markTopicIneligibleForDeletion(Set(topic)) onPartitionReassignment(topicAndPartition, reassignedPartitionContext) } else { // some replica in RAR is not alive. Fail partition reassignment @@ -601,16 +639,16 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def onPreferredReplicaElection(partitions: Set[TopicAndPartition]) { + def onPreferredReplicaElection(partitions: Set[TopicAndPartition], isTriggeredByAutoRebalance: Boolean = false) { info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(","))) try { controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions - deleteTopicManager.haltTopicDeletion(partitions.map(_.topic)) + deleteTopicManager.markTopicIneligibleForDeletion(partitions.map(_.topic)) partitionStateMachine.handleStateChanges(partitions, OnlinePartition, preferredReplicaPartitionLeaderSelector) } catch { case e: Throwable => error("Error completing preferred replica leader election for partitions %s".format(partitions.mkString(",")), e) } finally { - removePartitionsFromPreferredReplicaElection(partitions) + removePartitionsFromPreferredReplicaElection(partitions, isTriggeredByAutoRebalance) deleteTopicManager.resumeDeletionForTopics(partitions.map(_.topic)) } } @@ -622,7 +660,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg */ def startup() = { inLock(controllerContext.controllerLock) { - info("Controller starting up"); + info("Controller starting up") registerSessionExpirationListener() isRunning = true controllerElector.startup @@ -638,16 +676,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg def shutdown() = { inLock(controllerContext.controllerLock) { isRunning = false - partitionStateMachine.shutdown() - replicaStateMachine.shutdown() - if (config.autoLeaderRebalanceEnable) - autoRebalanceScheduler.shutdown() - if(controllerContext.controllerChannelManager != null) { - controllerContext.controllerChannelManager.shutdown() - controllerContext.controllerChannelManager = null - info("Controller shutdown complete") - } } + onControllerResignation() } def sendRequest(brokerId : Int, request : RequestOrResponse, callback: (RequestOrResponse) => Unit = null) = { @@ -748,17 +778,16 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private def initializeTopicDeletion() { val topicsQueuedForDeletion = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.DeleteTopicsPath).toSet - val replicasOnDeadBrokers = controllerContext.partitionReplicaAssignment.filter(r => - r._2.foldLeft(false)((res,r) => res || !controllerContext.liveBrokerIds.contains(r))) - val topicsWithReplicasOnDeadBrokers = replicasOnDeadBrokers.map(_._1.topic).toSet + val topicsWithReplicasOnDeadBrokers = controllerContext.partitionReplicaAssignment.filter { case(partition, replicas) => + replicas.exists(r => !controllerContext.liveBrokerIds.contains(r)) }.keySet.map(_.topic) val topicsForWhichPartitionReassignmentIsInProgress = controllerContext.partitionsUndergoingPreferredReplicaElection.map(_.topic) val topicsForWhichPreferredReplicaElectionIsInProgress = controllerContext.partitionsBeingReassigned.keySet.map(_.topic) - val haltedTopicsForDeletion = topicsWithReplicasOnDeadBrokers | topicsForWhichPartitionReassignmentIsInProgress | + val topicsIneligibleForDeletion = topicsWithReplicasOnDeadBrokers | topicsForWhichPartitionReassignmentIsInProgress | topicsForWhichPreferredReplicaElectionIsInProgress info("List of topics to be deleted: %s".format(topicsQueuedForDeletion.mkString(","))) - info("List of topics halted for deletion: %s".format(haltedTopicsForDeletion.mkString(","))) + info("List of topics ineligible for deletion: %s".format(topicsIneligibleForDeletion.mkString(","))) // initialize the topic deletion manager - deleteTopicManager = new TopicDeletionManager(this, topicsQueuedForDeletion, haltedTopicsForDeletion) + deleteTopicManager = new TopicDeletionManager(this, topicsQueuedForDeletion, topicsIneligibleForDeletion) } private def maybeTriggerPartitionReassignment() { @@ -776,8 +805,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg controllerContext.controllerChannelManager.startup() } - private def updateLeaderAndIsrCache() { - val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, controllerContext.partitionReplicaAssignment.keySet) + def updateLeaderAndIsrCache(topicAndPartitions: Set[TopicAndPartition] = controllerContext.partitionReplicaAssignment.keySet) { + val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, topicAndPartitions) for((topicPartition, leaderIsrAndControllerEpoch) <- leaderAndIsrInfo) controllerContext.partitionLeadershipInfo.put(topicPartition, leaderIsrAndControllerEpoch) } @@ -872,16 +901,49 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } + private def registerIsrChangeNotificationListener() = { + debug("Registering IsrChangeNotificationListener") + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) + zkClient.subscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) + } + + private def deregisterIsrChangeNotificationListener() = { + debug("De-registering IsrChangeNotificationListener") + zkClient.unsubscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener) + } + private def registerReassignedPartitionsListener() = { - zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, new PartitionsReassignedListener(this)) + zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) + } + + private def deregisterReassignedPartitionsListener() = { + zkClient.unsubscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener) } private def registerPreferredReplicaElectionListener() { - zkClient.subscribeDataChanges(ZkUtils.PreferredReplicaLeaderElectionPath, new PreferredReplicaElectionListener(this)) + zkClient.subscribeDataChanges(ZkUtils.PreferredReplicaLeaderElectionPath, preferredReplicaElectionListener) } - private def registerControllerChangedListener() { - zkClient.subscribeDataChanges(ZkUtils.ControllerEpochPath, new ControllerEpochListener(this)) + private def deregisterPreferredReplicaElectionListener() { + zkClient.unsubscribeDataChanges(ZkUtils.PreferredReplicaLeaderElectionPath, preferredReplicaElectionListener) + } + + private def deregisterReassignedPartitionsIsrChangeListeners() { + controllerContext.partitionsBeingReassigned.foreach { + case (topicAndPartition, reassignedPartitionsContext) => + val zkPartitionPath = ZkUtils.getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition) + zkClient.unsubscribeDataChanges(zkPartitionPath, reassignedPartitionsContext.isrChangeListener) + } + } + + private def readControllerEpochFromZookeeper() { + // initialize the controller epoch and zk version by reading from zookeeper + if(ZkUtils.pathExists(controllerContext.zkClient, ZkUtils.ControllerEpochPath)) { + val epochData = ZkUtils.readData(controllerContext.zkClient, ZkUtils.ControllerEpochPath) + controllerContext.epoch = epochData._1.toInt + controllerContext.epochZkVersion = epochData._2.getVersion + info("Initialized controller epoch to %d and zk version %d".format(controllerContext.epoch, controllerContext.epochZkVersion)) + } } def removePartitionFromReassignedPartitions(topicAndPartition: TopicAndPartition) { @@ -913,7 +975,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicAndPartition]) { + def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicAndPartition], + isTriggeredByAutoRebalance : Boolean) { for(partition <- partitionsToBeRemoved) { // check the status val currentLeader = controllerContext.partitionLeadershipInfo(partition).leaderAndIsr.leader @@ -924,7 +987,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg warn("Partition %s failed to complete preferred replica leader election. Leader is %d".format(partition, currentLeader)) } } - ZkUtils.deletePath(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath) + if (!isTriggeredByAutoRebalance) + ZkUtils.deletePath(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath) controllerContext.partitionsUndergoingPreferredReplicaElection --= partitionsToBeRemoved } @@ -933,9 +997,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * metadata requests * @param brokers The brokers that the update metadata request should be sent to */ - def sendUpdateMetadataRequest(brokers: Seq[Int]) { + def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) { brokerRequestBatch.newBatch() - brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers) + brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) } @@ -956,7 +1020,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg var zkWriteCompleteOrUnnecessary = false while (!zkWriteCompleteOrUnnecessary) { // refresh leader and isr from zookeeper again - val leaderIsrAndEpochOpt = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) + val leaderIsrAndEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) zkWriteCompleteOrUnnecessary = leaderIsrAndEpochOpt match { case Some(leaderIsrAndEpoch) => // increment the leader epoch even if the ISR changes val leaderAndIsr = leaderIsrAndEpoch.leaderAndIsr @@ -967,17 +1031,25 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg "controller was elected with epoch %d. Aborting state change by this controller".format(controllerEpoch)) if (leaderAndIsr.isr.contains(replicaId)) { // if the replica to be removed from the ISR is also the leader, set the new leader value to -1 - val newLeader = if(replicaId == leaderAndIsr.leader) -1 else leaderAndIsr.leader + val newLeader = if (replicaId == leaderAndIsr.leader) LeaderAndIsr.NoLeader else leaderAndIsr.leader + var newIsr = leaderAndIsr.isr.filter(b => b != replicaId) + + // if the replica to be removed from the ISR is the last surviving member of the ISR and unclean leader election + // is disallowed for the corresponding topic, then we must preserve the ISR membership so that the replica can + // eventually be restored as the leader. + if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(zkClient, + topicAndPartition.topic)).uncleanLeaderElectionEnable) { + info("Retaining last ISR %d of partition %s since unclean leader election is disabled".format(replicaId, topicAndPartition)) + newIsr = leaderAndIsr.isr + } + val newLeaderAndIsr = new LeaderAndIsr(newLeader, leaderAndIsr.leaderEpoch + 1, - leaderAndIsr.isr.filter(b => b != replicaId), leaderAndIsr.zkVersion + 1) + newIsr, 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 + val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partition, + newLeaderAndIsr, epoch, leaderAndIsr.zkVersion) + newLeaderAndIsr.zkVersion = newVersion finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newLeaderAndIsr, epoch)) controllerContext.partitionLeadershipInfo.put(topicAndPartition, finalLeaderIsrAndControllerEpoch.get) if (updateSucceeded) @@ -1011,7 +1083,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg var zkWriteCompleteOrUnnecessary = false while (!zkWriteCompleteOrUnnecessary) { // refresh leader and isr from zookeeper again - val leaderIsrAndEpochOpt = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) + val leaderIsrAndEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) zkWriteCompleteOrUnnecessary = leaderIsrAndEpochOpt match { case Some(leaderIsrAndEpoch) => val leaderAndIsr = leaderIsrAndEpoch.leaderAndIsr @@ -1025,11 +1097,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg 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) + val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, + partition, newLeaderAndIsr, epoch, leaderAndIsr.zkVersion) + newLeaderAndIsr.zkVersion = newVersion finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newLeaderAndIsr, epoch)) if (updateSucceeded) @@ -1066,6 +1136,10 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg controllerElector.elect } } + + override def handleSessionEstablishmentError(error: Throwable): Unit = { + //no-op handleSessionEstablishmentError in KafkaHealthCheck should System.exit and log the error. + } } private def checkAndTriggerPartitionRebalance(): Unit = { @@ -1089,6 +1163,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg topicsNotInPreferredReplica = topicAndPartitionsForBroker.filter { case(topicPartition, replicas) => { + controllerContext.partitionLeadershipInfo.contains(topicPartition) && controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader != leaderBroker } } @@ -1101,26 +1176,18 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // 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)) { - inLock(controllerContext.controllerLock) { - // 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)) + topicsNotInPreferredReplica.foreach { + case(topicPartition, replicas) => { + inLock(controllerContext.controllerLock) { + // 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 && + !deleteTopicManager.isTopicQueuedUpForDeletion(topicPartition.topic) && + controllerContext.allTopics.contains(topicPartition.topic)) { + onPreferredReplicaElection(Set(topicPartition), true) + } } } } @@ -1237,6 +1304,56 @@ class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic: } } +/** + * Called when leader intimates of isr change + * @param controller + */ +class IsrChangeNotificationListener(controller: KafkaController) extends IZkChildListener with Logging { + var topicAndPartitionSet: Set[TopicAndPartition] = Set() + + override def handleChildChange(parentPath: String, currentChildren: util.List[String]): Unit = { + import scala.collection.JavaConverters._ + + inLock(controller.controllerContext.controllerLock) { + debug("[IsrChangeNotificationListener] Fired!!!") + val childrenAsScala: mutable.Buffer[String] = currentChildren.asScala + val topicAndPartitions: immutable.Set[TopicAndPartition] = childrenAsScala.map(x => getTopicAndPartition(x)).flatten.toSet + controller.updateLeaderAndIsrCache(topicAndPartitions) + processUpdateNotifications(topicAndPartitions) + + // delete processed children + childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient, ZkUtils.TopicConfigChangesPath + "/" + x)) + } + } + + private def processUpdateNotifications(topicAndPartitions: immutable.Set[TopicAndPartition]) { + val liveBrokers: Seq[Int] = controller.controllerContext.liveOrShuttingDownBrokerIds.toSeq + controller.sendUpdateMetadataRequest(liveBrokers, topicAndPartitions) + debug("Sending MetadataRequest to Brokers:" + liveBrokers + " for TopicAndPartitions:" + topicAndPartitions) + } + + private def getTopicAndPartition(child: String): Option[TopicAndPartition] = { + val changeZnode: String = ZkUtils.IsrChangeNotificationPath + "/" + child + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(controller.controllerContext.zkClient, changeZnode) + if (jsonOpt.isDefined) { + val json = Json.parseFull(jsonOpt.get) + + json match { + case Some(m) => + val topicAndPartition = m.asInstanceOf[Map[String, Any]] + val topic = topicAndPartition("topic").asInstanceOf[String] + val partition = topicAndPartition("partition").asInstanceOf[Int] + Some(TopicAndPartition(topic, partition)) + case None => + error("Invalid topic and partition JSON: " + json + " in ZK: " + changeZnode) + None + } + } else { + None + } + } +} + /** * Starts the preferred replica leader election for the list of partitions specified under * /admin/preferred_replica_election - @@ -1265,8 +1382,7 @@ class PreferredReplicaElectionListener(controller: KafkaController) extends IZkD error("Skipping preferred replica election for partitions %s since the respective topics are being deleted" .format(partitionsForTopicsToBeDeleted)) } - else - controller.onPreferredReplicaElection(partitions -- partitionsForTopicsToBeDeleted) + controller.onPreferredReplicaElection(partitions -- partitionsForTopicsToBeDeleted) } } @@ -1279,43 +1395,6 @@ class PreferredReplicaElectionListener(controller: KafkaController) extends IZkD } } -class ControllerEpochListener(controller: KafkaController) extends IZkDataListener with Logging { - this.logIdent = "[ControllerEpochListener on " + controller.config.brokerId + "]: " - val controllerContext = controller.controllerContext - readControllerEpochFromZookeeper() - - /** - * Invoked when a controller updates the epoch value - * @throws Exception On any error. - */ - @throws(classOf[Exception]) - def handleDataChange(dataPath: String, data: Object) { - debug("Controller epoch listener fired with new epoch " + data.toString) - inLock(controllerContext.controllerLock) { - // read the epoch path to get the zk version - readControllerEpochFromZookeeper() - } - } - - /** - * @throws Exception - * On any error. - */ - @throws(classOf[Exception]) - def handleDataDeleted(dataPath: String) { - } - - private def readControllerEpochFromZookeeper() { - // initialize the controller epoch and zk version by reading from zookeeper - if(ZkUtils.pathExists(controllerContext.zkClient, ZkUtils.ControllerEpochPath)) { - val epochData = ZkUtils.readData(controllerContext.zkClient, ZkUtils.ControllerEpochPath) - controllerContext.epoch = epochData._1.toInt - controllerContext.epochZkVersion = epochData._2.getVersion - info("Initialized controller epoch to %d and zk version %d".format(controllerContext.epoch, controllerContext.epochZkVersion)) - } - } -} - case class ReassignedPartitionsContext(var newReplicas: Seq[Int] = Seq.empty, var isrChangeListener: ReassignedPartitionsIsrChangeListener = null) @@ -1325,7 +1404,7 @@ case class PartitionAndReplica(topic: String, partition: Int, replica: Int) { } } -case class LeaderIsrAndControllerEpoch(val leaderAndIsr: LeaderAndIsr, controllerEpoch: Int) { +case class LeaderIsrAndControllerEpoch(leaderAndIsr: LeaderAndIsr, controllerEpoch: Int) { override def toString(): String = { val leaderAndIsrInfo = new StringBuilder leaderAndIsrInfo.append("(Leader:" + leaderAndIsr.leader) diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index fa29bbe82db35..bb6b5c8764522 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -16,9 +16,12 @@ */ package kafka.controller +import kafka.admin.AdminUtils import kafka.api.LeaderAndIsr +import kafka.log.LogConfig import kafka.utils.Logging import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException} +import kafka.server.KafkaConfig trait PartitionLeaderSelector { @@ -37,12 +40,14 @@ trait PartitionLeaderSelector { * 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 + * 2. Else, if unclean leader election for the topic is disabled, it throws a NoReplicaOnlineException. + * 3. Else, it picks some alive broker from the assigned replica list as the new leader and the new isr. + * 4. If no broker in the assigned replica list is alive, it throws a 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 { +class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, config: KafkaConfig) + extends PartitionLeaderSelector with Logging { this.logIdent = "[OfflinePartitionLeaderSelector]: " def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { @@ -54,6 +59,15 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten val currentLeaderIsrZkPathVersion = currentLeaderAndIsr.zkVersion val newLeaderAndIsr = liveBrokersInIsr.isEmpty match { case true => + // Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration + // for unclean leader election. + if (!LogConfig.fromProps(config.originals, AdminUtils.fetchTopicConfig(controllerContext.zkClient, + topicAndPartition.topic)).uncleanLeaderElectionEnable) { + throw new NoReplicaOnlineException(("No broker in ISR for partition " + + "%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) + + " ISR brokers are: [%s]".format(currentLeaderAndIsr.isr.mkString(","))) + } + debug("No broker in ISR is alive for %s. Pick the leader from the alive assigned replicas: %s" .format(topicAndPartition, liveAssignedReplicas.mkString(","))) liveAssignedReplicas.isEmpty match { @@ -69,7 +83,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, List(newLeader), currentLeaderIsrZkPathVersion + 1) } case false => - val newLeader = liveBrokersInIsr.head + val liveReplicasInIsr = liveAssignedReplicas.filter(r => liveBrokersInIsr.contains(r)) + val newLeader = liveReplicasInIsr.head debug("Some broker in ISR is alive for %s. Select %d from ISR %s to be the leader." .format(topicAndPartition, newLeader, liveBrokersInIsr.mkString(","))) new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, liveBrokersInIsr.toList, currentLeaderIsrZkPathVersion + 1) @@ -77,7 +92,7 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten info("Selected new leader and ISR %s for offline partition %s".format(newLeaderAndIsr.toString(), topicAndPartition)) (newLeaderAndIsr, liveAssignedReplicas) case None => - throw new NoReplicaOnlineException("Partition %s doesn't have".format(topicAndPartition) + "replicas assigned to it") + throw new NoReplicaOnlineException("Partition %s doesn't have replicas assigned to it".format(topicAndPartition)) } } } @@ -196,4 +211,4 @@ class NoOpLeaderSelector(controllerContext: ControllerContext) extends Partition warn("I should never have been asked to perform leader election, returning the current LeaderAndIsr and replica assignment.") (currentLeaderAndIsr, controllerContext.partitionReplicaAssignment(topicAndPartition)) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala old mode 100644 new mode 100755 index 57c96b5539f20..b4e7c88a32cf8 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -22,12 +22,11 @@ import collection.mutable.Buffer import java.util.concurrent.atomic.AtomicBoolean import kafka.api.LeaderAndIsr import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException} -import kafka.utils.{Logging, ZkUtils} +import kafka.utils.{Logging, ZkUtils, ReplicationUtils} import org.I0Itec.zkclient.{IZkDataListener, IZkChildListener} import org.I0Itec.zkclient.exception.ZkNodeExistsException -import org.apache.log4j.Logger import kafka.controller.Callbacks.CallbackBuilder -import kafka.utils.Utils._ +import kafka.utils.CoreUtils._ /** * This class represents the state machine for partitions. It defines the states that a partition can be in, and @@ -45,12 +44,16 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { private val controllerContext = controller.controllerContext private val controllerId = controller.config.brokerId private val zkClient = controllerContext.zkClient - var partitionState: mutable.Map[TopicAndPartition, PartitionState] = mutable.Map.empty - val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) + private val partitionState: mutable.Map[TopicAndPartition, PartitionState] = mutable.Map.empty + private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) private val hasStarted = new AtomicBoolean(false) private val noOpPartitionLeaderSelector = new NoOpLeaderSelector(controllerContext) + private val topicChangeListener = new TopicChangeListener() + private val deleteTopicsListener = new DeleteTopicsListener() + private val addPartitionsListener: mutable.Map[String, AddPartitionsListener] = mutable.Map.empty + private val stateChangeLogger = KafkaController.stateChangeLogger + this.logIdent = "[Partition state machine on Controller " + controllerId + "]: " - private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) /** * Invoked on successful controller election. First registers a topic change listener since that triggers all @@ -60,24 +63,45 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { def startup() { // initialize partition state initializePartitionState() + // set started flag hasStarted.set(true) // try to move partitions to online state triggerOnlinePartitionStateChange() + info("Started partition state machine with initial state -> " + partitionState.toString()) } // register topic and partition change listeners def registerListeners() { registerTopicChangeListener() - registerDeleteTopicListener() + if(controller.config.deleteTopicEnable) + registerDeleteTopicListener() + } + + // de-register topic and partition change listeners + def deregisterListeners() { + deregisterTopicChangeListener() + addPartitionsListener.foreach { + case (topic, listener) => + zkClient.unsubscribeDataChanges(ZkUtils.getTopicPath(topic), listener) + } + addPartitionsListener.clear() + if(controller.config.deleteTopicEnable) + deregisterDeleteTopicListener() } /** * Invoked on controller shutdown. */ def shutdown() { + // reset started flag hasStarted.set(false) + // clear partition state partitionState.clear() + // de-register all ZK listeners + deregisterListeners() + + info("Stopped partition state machine") } /** @@ -164,11 +188,11 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { case NewPartition => // pre: partition did not exist before this assertValidPreviousStates(topicAndPartition, List(NonExistentPartition), NewPartition) - assignReplicasToPartitions(topic, partition) partitionState.put(topicAndPartition, NewPartition) val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition).mkString(",") - stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from NotExists to New with assigned replicas %s" - .format(controllerId, controller.epoch, topicAndPartition, assignedReplicas)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from %s to %s with assigned replicas %s" + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState, + assignedReplicas)) // post: partition has been assigned replicas case OnlinePartition => assertValidPreviousStates(topicAndPartition, List(NewPartition, OnlinePartition, OfflinePartition), OnlinePartition) @@ -184,22 +208,22 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } partitionState.put(topicAndPartition, OnlinePartition) val leader = controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader - stateChangeLogger.trace("Controller %d epoch %d changed partition %s from %s to OnlinePartition with leader %d" - .format(controllerId, controller.epoch, topicAndPartition, partitionState(topicAndPartition), leader)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s from %s to %s with leader %d" + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState, leader)) // post: partition has a leader case OfflinePartition => // pre: partition should be in New or Online state assertValidPreviousStates(topicAndPartition, List(NewPartition, OnlinePartition, OfflinePartition), OfflinePartition) // should be called when the leader for a partition is no longer alive - stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from Online to Offline" - .format(controllerId, controller.epoch, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from %s to %s" + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState)) partitionState.put(topicAndPartition, OfflinePartition) // post: partition has no alive leader case NonExistentPartition => // pre: partition should be in Offline state assertValidPreviousStates(topicAndPartition, List(OfflinePartition), NonExistentPartition) - stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from Offline to NotExists" - .format(controllerId, controller.epoch, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from %s to %s" + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState)) partitionState.put(topicAndPartition, NonExistentPartition) // post: partition state is deleted from all brokers and zookeeper } @@ -240,17 +264,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { .format(partitionState(topicAndPartition))) } - /** - * Invoked on the NonExistentPartition->NewPartition state transition to update the controller's cache with the - * partition's replica assignment. - * @param topic The topic of the partition whose replica assignment is to be cached - * @param partition The partition whose replica assignment is to be cached - */ - private def assignReplicasToPartitions(topic: String, partition: Int) { - val assignedReplicas = ZkUtils.getReplicasForPartition(controllerContext.zkClient, topic, partition) - controllerContext.partitionReplicaAssignment += TopicAndPartition(topic, partition) -> assignedReplicas - } - /** * Invoked on the NewPartition->OnlinePartition state change. When a partition is in the New state, it does not have * a leader and isr path in zookeeper. Once the partition moves to the OnlinePartition state, it's leader and isr @@ -288,7 +301,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } catch { case e: ZkNodeExistsException => // read the controller epoch - val leaderIsrAndEpoch = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, + val leaderIsrAndEpoch = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition).get val failMsg = ("encountered error while changing partition %s's state from New to Online since LeaderAndIsr path already " + "exists with value %s and controller epoch %d") @@ -329,9 +342,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } // elect new leader or throw exception val (leaderAndIsr, replicas) = leaderSelector.selectLeader(topicAndPartition, currentLeaderAndIsr) - val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath(zkClient, - ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition), - ZkUtils.leaderAndIsrZkData(leaderAndIsr, controller.epoch), currentLeaderAndIsr.zkVersion) + val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partition, + leaderAndIsr, controller.epoch, currentLeaderAndIsr.zkVersion) newLeaderAndIsr = leaderAndIsr newLeaderAndIsr.zkVersion = newVersion zookeeperPathUpdateSucceeded = updateSucceeded @@ -358,20 +370,34 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } private def registerTopicChangeListener() = { - zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, new TopicChangeListener()) + zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicChangeListener) + } + + private def deregisterTopicChangeListener() = { + zkClient.unsubscribeChildChanges(ZkUtils.BrokerTopicsPath, topicChangeListener) } def registerPartitionChangeListener(topic: String) = { - zkClient.subscribeDataChanges(ZkUtils.getTopicPath(topic), new AddPartitionsListener(topic)) + addPartitionsListener.put(topic, new AddPartitionsListener(topic)) + zkClient.subscribeDataChanges(ZkUtils.getTopicPath(topic), addPartitionsListener(topic)) + } + + def deregisterPartitionChangeListener(topic: String) = { + zkClient.unsubscribeDataChanges(ZkUtils.getTopicPath(topic), addPartitionsListener(topic)) + addPartitionsListener.remove(topic) } private def registerDeleteTopicListener() = { - zkClient.subscribeChildChanges(ZkUtils.DeleteTopicsPath, new DeleteTopicsListener()) + zkClient.subscribeChildChanges(ZkUtils.DeleteTopicsPath, deleteTopicsListener) + } + + private def deregisterDeleteTopicListener() = { + zkClient.unsubscribeChildChanges(ZkUtils.DeleteTopicsPath, deleteTopicsListener) } private def getLeaderIsrAndEpochOrThrowException(topic: String, partition: Int): LeaderIsrAndControllerEpoch = { val topicAndPartition = TopicAndPartition(topic, partition) - ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) match { + ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) match { case Some(currentLeaderIsrAndEpoch) => currentLeaderIsrAndEpoch case None => val failMsg = "LeaderAndIsr information doesn't exist for partition %s in %s state" @@ -438,22 +464,24 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } debug("Delete topics listener fired for topics %s to be deleted".format(topicsToBeDeleted.mkString(","))) val nonExistentTopics = topicsToBeDeleted.filter(t => !controllerContext.allTopics.contains(t)) - if(nonExistentTopics.size > 0) + if(nonExistentTopics.size > 0) { warn("Ignoring request to delete non-existing topics " + nonExistentTopics.mkString(",")) + nonExistentTopics.foreach(topic => ZkUtils.deletePathRecursive(zkClient, ZkUtils.getDeleteTopicPath(topic))) + } topicsToBeDeleted --= nonExistentTopics if(topicsToBeDeleted.size > 0) { info("Starting topic deletion for topics " + topicsToBeDeleted.mkString(",")) - // add topic to deletion list - controller.deleteTopicManager.enqueueTopicsForDeletion(topicsToBeDeleted) - // halt if other state changes are in progress + // mark topic ineligible for deletion if other state changes are in progress topicsToBeDeleted.foreach { topic => val preferredReplicaElectionInProgress = controllerContext.partitionsUndergoingPreferredReplicaElection.map(_.topic).contains(topic) val partitionReassignmentInProgress = controllerContext.partitionsBeingReassigned.keySet.map(_.topic).contains(topic) if(preferredReplicaElectionInProgress || partitionReassignmentInProgress) - controller.deleteTopicManager.haltTopicDeletion(Set(topic)) + controller.deleteTopicManager.markTopicIneligibleForDeletion(Set(topic)) } + // add topic to deletion list + controller.deleteTopicManager.enqueueTopicsForDeletion(topicsToBeDeleted) } } } @@ -486,6 +514,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { else { if (partitionsToBeAdded.size > 0) { info("New partitions to be added %s".format(partitionsToBeAdded)) + controllerContext.partitionReplicaAssignment.++=(partitionsToBeAdded) controller.onNewPartitionCreation(partitionsToBeAdded.keySet.toSet) } } @@ -507,5 +536,3 @@ case object NewPartition extends PartitionState { val state: Byte = 0 } case object OnlinePartition extends PartitionState { val state: Byte = 1 } case object OfflinePartition extends PartitionState { val state: Byte = 2 } case object NonExistentPartition extends PartitionState { val state: Byte = 3 } - - diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala old mode 100644 new mode 100755 index 613aec6f40db5..3a44fdc4e6b35 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -20,11 +20,11 @@ import collection._ import collection.JavaConversions._ import java.util.concurrent.atomic.AtomicBoolean import kafka.common.{TopicAndPartition, StateChangeFailedException} -import kafka.utils.{ZkUtils, Logging} +import kafka.utils.{ZkUtils, ReplicationUtils, Logging} import org.I0Itec.zkclient.IZkChildListener import org.apache.log4j.Logger import kafka.controller.Callbacks._ -import kafka.utils.Utils._ +import kafka.utils.CoreUtils._ /** * This class represents the state machine for replicas. It defines the states that a replica can be in, and @@ -40,7 +40,7 @@ import kafka.utils.Utils._ * 4. ReplicaDeletionStarted: If replica deletion starts, it is moved to this state. Valid previous state is OfflineReplica * 5. ReplicaDeletionSuccessful: If replica responds with no error code in response to a delete replica request, it is * moved to this state. Valid previous state is ReplicaDeletionStarted - * 6. ReplicaDeletionFailed: If replica deletion fails, it is moved to this state. Valid previous state is ReplicaDeletionStarted + * 6. ReplicaDeletionIneligible: If replica deletion fails, it is moved to this state. Valid previous state is ReplicaDeletionStarted * 7. NonExistentReplica: If a replica is deleted successfully, it is moved to this state. Valid previous state is * ReplicaDeletionSuccessful */ @@ -48,11 +48,14 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { private val controllerContext = controller.controllerContext private val controllerId = controller.config.brokerId private val zkClient = controllerContext.zkClient - var replicaState: mutable.Map[PartitionAndReplica, ReplicaState] = mutable.Map.empty - val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) + private val replicaState: mutable.Map[PartitionAndReplica, ReplicaState] = mutable.Map.empty + private val brokerChangeListener = new BrokerChangeListener() + private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) private val hasStarted = new AtomicBoolean(false) + private val stateChangeLogger = KafkaController.stateChangeLogger + this.logIdent = "[Replica state machine on controller " + controller.config.brokerId + "]: " - private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) + /** * Invoked on successful controller election. First registers a broker change listener since that triggers all @@ -62,23 +65,38 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { def startup() { // initialize replica state initializeReplicaState() + // set started flag hasStarted.set(true) // move all Online replicas to Online handleStateChanges(controllerContext.allLiveReplicas(), OnlineReplica) + info("Started replica state machine with initial state -> " + replicaState.toString()) } - // register broker change listener + // register ZK listeners of the replica state machine def registerListeners() { + // register broker change listener registerBrokerChangeListener() } + // de-register ZK listeners of the replica state machine + def deregisterListeners() { + // de-register broker change listener + deregisterBrokerChangeListener() + } + /** * Invoked on controller shutdown. */ def shutdown() { + // reset started flag hasStarted.set(false) + // reset replica state replicaState.clear() + // de-register all ZK listeners + deregisterListeners() + + info("Stopped replica state machine") } /** @@ -115,7 +133,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * --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,ReplicaDeletionFailed -> OfflineReplica + * NewReplica,OnlineReplica,OfflineReplica,ReplicaDeletionIneligible -> 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. @@ -126,7 +144,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * ReplicaDeletionStarted -> ReplicaDeletionSuccessful * -- mark the state of the replica in the state machine * - * ReplicaDeletionStarted -> ReplicaDeletionFailed + * ReplicaDeletionStarted -> ReplicaDeletionIneligible * -- mark the state of the replica in the state machine * * ReplicaDeletionSuccessful -> NonExistentReplica @@ -146,14 +164,14 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { throw new StateChangeFailedException(("Controller %d epoch %d initiated state change of replica %d for partition %s " + "to %s failed because replica state machine has not started") .format(controllerId, controller.epoch, replicaId, topicAndPartition, targetState)) + val currState = replicaState.getOrElseUpdate(partitionAndReplica, NonExistentReplica) try { - replicaState.getOrElseUpdate(partitionAndReplica, NonExistentReplica) val replicaAssignment = controllerContext.partitionReplicaAssignment(topicAndPartition) targetState match { case NewReplica => assertValidPreviousStates(partitionAndReplica, List(NonExistentReplica), targetState) // start replica as a follower to the current leader for its partition - val leaderIsrAndControllerEpochOpt = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) + val leaderIsrAndControllerEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) leaderIsrAndControllerEpochOpt match { case Some(leaderIsrAndControllerEpoch) => if(leaderIsrAndControllerEpoch.leaderAndIsr.leader == replicaId) @@ -165,45 +183,47 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case None => // new leader request will be sent to this replica when one gets elected } replicaState.put(partitionAndReplica, NewReplica) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to NewReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, + targetState)) case ReplicaDeletionStarted => assertValidPreviousStates(partitionAndReplica, List(OfflineReplica), targetState) replicaState.put(partitionAndReplica, ReplicaDeletionStarted) // send stop replica command brokerRequestBatch.addStopReplicaRequestForBrokers(List(replicaId), topic, partition, deletePartition = true, callbacks.stopReplicaResponseCallback) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to ReplicaDeletionStarted" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) - case ReplicaDeletionFailed => + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) + case ReplicaDeletionIneligible => assertValidPreviousStates(partitionAndReplica, List(ReplicaDeletionStarted), targetState) - replicaState.put(partitionAndReplica, ReplicaDeletionFailed) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to ReplicaDeletionFailed" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + replicaState.put(partitionAndReplica, ReplicaDeletionIneligible) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) case ReplicaDeletionSuccessful => assertValidPreviousStates(partitionAndReplica, List(ReplicaDeletionStarted), targetState) replicaState.put(partitionAndReplica, ReplicaDeletionSuccessful) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to ReplicaDeletionSuccessful" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) case NonExistentReplica => assertValidPreviousStates(partitionAndReplica, List(ReplicaDeletionSuccessful), targetState) // remove this replica from the assigned replicas list for its partition val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) controllerContext.partitionReplicaAssignment.put(topicAndPartition, currentAssignedReplicas.filterNot(_ == replicaId)) replicaState.remove(partitionAndReplica) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to NonExistentReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) case OnlineReplica => assertValidPreviousStates(partitionAndReplica, - List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionFailed), targetState) + List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionIneligible), targetState) replicaState(partitionAndReplica) match { case NewReplica => // add this replica to the assigned replicas list for its partition val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) if(!currentAssignedReplicas.contains(replicaId)) controllerContext.partitionReplicaAssignment.put(topicAndPartition, currentAssignedReplicas :+ replicaId) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, + targetState)) case _ => // check if the leader for this partition ever existed controllerContext.partitionLeadershipInfo.get(topicAndPartition) match { @@ -211,8 +231,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId), topic, partition, leaderIsrAndControllerEpoch, replicaAssignment) replicaState.put(partitionAndReplica, OnlineReplica) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) case None => // that means the partition was never in OnlinePartition state, this means the broker never // started a log for that partition and does not have a high watermark value for this partition } @@ -220,7 +240,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { replicaState.put(partitionAndReplica, OnlineReplica) case OfflineReplica => assertValidPreviousStates(partitionAndReplica, - List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionFailed), targetState) + List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionIneligible), 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 @@ -229,12 +249,15 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case Some(currLeaderIsrAndControllerEpoch) => 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) + // send the shrunk ISR state change request to all the remaining alive replicas of the partition. + val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) + if (!controller.deleteTopicManager.isPartitionToBeDeleted(topicAndPartition)) { + brokerRequestBatch.addLeaderAndIsrRequestForBrokers(currentAssignedReplicas.filterNot(_ == replicaId), + topic, partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment) + } replicaState.put(partitionAndReplica, OfflineReplica) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OfflineReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) false case None => true @@ -250,8 +273,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { } catch { case t: Throwable => - stateChangeLogger.error("Controller %d epoch %d initiated state change of replica %d for partition [%s,%d] to %s failed" - .format(controllerId, controller.epoch, replicaId, topic, partition, targetState), t) + stateChangeLogger.error("Controller %d epoch %d initiated state change of replica %d for partition [%s,%d] from %s to %s failed" + .format(controllerId, controller.epoch, replicaId, topic, partition, currState, targetState), t) } } @@ -259,7 +282,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { val replicasForTopic = controller.controllerContext.replicasForTopic(topic) val replicaStatesForTopic = replicasForTopic.map(r => (r, replicaState(r))).toMap debug("Are all replicas for topic %s deleted %s".format(topic, replicaStatesForTopic)) - replicaStatesForTopic.foldLeft(true)((deletionState, r) => deletionState && r._2 == ReplicaDeletionSuccessful) + replicaStatesForTopic.forall(_._2 == ReplicaDeletionSuccessful) } def isAtLeastOneReplicaInDeletionStartedState(topic: String): Boolean = { @@ -272,8 +295,12 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { replicaState.filter(r => r._1.topic.equals(topic) && r._2 == state).keySet } + def isAnyReplicaInState(topic: String, state: ReplicaState): Boolean = { + replicaState.exists(r => r._1.topic.equals(topic) && r._2 == state) + } + def replicasInDeletionStates(topic: String): Set[PartitionAndReplica] = { - val deletionStates = Set(ReplicaDeletionStarted, ReplicaDeletionSuccessful, ReplicaDeletionFailed) + val deletionStates = Set(ReplicaDeletionStarted, ReplicaDeletionSuccessful, ReplicaDeletionIneligible) replicaState.filter(r => r._1.topic.equals(topic) && deletionStates.contains(r._2)).keySet } @@ -286,7 +313,11 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { } private def registerBrokerChangeListener() = { - zkClient.subscribeChildChanges(ZkUtils.BrokerIdsPath, new BrokerChangeListener()) + zkClient.subscribeChildChanges(ZkUtils.BrokerIdsPath, brokerChangeListener) + } + + private def deregisterBrokerChangeListener() = { + zkClient.unsubscribeChildChanges(ZkUtils.BrokerIdsPath, brokerChangeListener) } /** @@ -304,8 +335,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case false => // mark replicas on dead brokers as failed for topic deletion, if they belong to a topic to be deleted. // This is required during controller failover since during controller failover a broker can go down, - // so the replicas on that broker should be moved to ReplicaDeletionFailed to be on the safer side. - replicaState.put(partitionAndReplica, ReplicaDeletionFailed) + // so the replicas on that broker should be moved to ReplicaDeletionIneligible to be on the safer side. + replicaState.put(partitionAndReplica, ReplicaDeletionIneligible) } } } @@ -356,7 +387,5 @@ case object OnlineReplica extends ReplicaState { val state: Byte = 2 } case object OfflineReplica extends ReplicaState { val state: Byte = 3 } case object ReplicaDeletionStarted extends ReplicaState { val state: Byte = 4} case object ReplicaDeletionSuccessful extends ReplicaState { val state: Byte = 5} -case object ReplicaDeletionFailed extends ReplicaState { val state: Byte = 6} +case object ReplicaDeletionIneligible extends ReplicaState { val state: Byte = 6} case object NonExistentReplica extends ReplicaState { val state: Byte = 7 } - - diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala old mode 100644 new mode 100755 index 91a446ddc8aeb..64ecb499f24bc --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -18,10 +18,12 @@ package kafka.controller import collection.mutable import kafka.utils.{ShutdownableThread, Logging, ZkUtils} -import kafka.utils.Utils._ +import kafka.utils.CoreUtils._ import collection.Set import kafka.common.{ErrorMapping, TopicAndPartition} import kafka.api.{StopReplicaResponse, RequestOrResponse} +import java.util.concurrent.locks.ReentrantLock +import java.util.concurrent.atomic.AtomicBoolean /** * This manages the state machine for topic deletion. @@ -30,8 +32,8 @@ import kafka.api.{StopReplicaResponse, RequestOrResponse} * 3. The controller has a background thread that handles topic deletion. The purpose of having this background thread * is to accommodate the TTL feature, when we have it. This thread is signaled whenever deletion for a topic needs to * be started or resumed. Currently, a topic's deletion can be started only by the onPartitionDeletion callback on the - * controller. In the future, it can be triggered based on the configured TTL for the topic. A topic's deletion will - * be halted in the following scenarios - + * controller. In the future, it can be triggered based on the configured TTL for the topic. A topic will be ineligible + * for deletion in the following scenarios - * 3.1 broker hosting one of the replicas for that topic goes down * 3.2 partition reassignment for partitions of that topic is in progress * 3.3 preferred replica election for partitions of that topic is in progress @@ -62,37 +64,51 @@ import kafka.api.{StopReplicaResponse, RequestOrResponse} * it marks the topic for deletion retry. * @param controller * @param initialTopicsToBeDeleted The topics that are queued up for deletion in zookeeper at the time of controller failover - * @param initialHaltedTopicsForDeletion The topics for which deletion is halted due to any of the conditions mentioned in #3 above + * @param initialTopicsIneligibleForDeletion The topics ineligible for deletion due to any of the conditions mentioned in #3 above */ class TopicDeletionManager(controller: KafkaController, initialTopicsToBeDeleted: Set[String] = Set.empty, - initialHaltedTopicsForDeletion: Set[String] = Set.empty) extends Logging { + initialTopicsIneligibleForDeletion: Set[String] = Set.empty) extends Logging { + this.logIdent = "[Topic Deletion Manager " + controller.config.brokerId + "], " val controllerContext = controller.controllerContext val partitionStateMachine = controller.partitionStateMachine val replicaStateMachine = controller.replicaStateMachine - var topicsToBeDeleted: mutable.Set[String] = mutable.Set.empty[String] ++ initialTopicsToBeDeleted - var haltedTopicsForDeletion: mutable.Set[String] = mutable.Set.empty[String] ++ - (initialHaltedTopicsForDeletion & initialTopicsToBeDeleted) - val deleteTopicsCond = controllerContext.controllerLock.newCondition() - var deleteTopicStateChanged: Boolean = false + val topicsToBeDeleted: mutable.Set[String] = mutable.Set.empty[String] ++ initialTopicsToBeDeleted + val partitionsToBeDeleted: mutable.Set[TopicAndPartition] = topicsToBeDeleted.flatMap(controllerContext.partitionsForTopic) + val deleteLock = new ReentrantLock() + val topicsIneligibleForDeletion: mutable.Set[String] = mutable.Set.empty[String] ++ + (initialTopicsIneligibleForDeletion & initialTopicsToBeDeleted) + val deleteTopicsCond = deleteLock.newCondition() + val deleteTopicStateChanged: AtomicBoolean = new AtomicBoolean(false) var deleteTopicsThread: DeleteTopicsThread = null + val isDeleteTopicEnabled = controller.config.deleteTopicEnable /** * Invoked at the end of new controller initiation */ def start() { - deleteTopicsThread = new DeleteTopicsThread() - deleteTopicStateChanged = true - deleteTopicsThread.start() + if (isDeleteTopicEnabled) { + deleteTopicsThread = new DeleteTopicsThread() + if (topicsToBeDeleted.size > 0) + deleteTopicStateChanged.set(true) + deleteTopicsThread.start() + } } /** - * Invoked when the current controller resigns. At this time, all state for topic deletion should be cleared + * Invoked when the current controller resigns. At this time, all state for topic deletion should be cleared. */ def shutdown() { - deleteTopicsThread.shutdown() - topicsToBeDeleted.clear() - haltedTopicsForDeletion.clear() + // Only allow one shutdown to go through + if (isDeleteTopicEnabled && deleteTopicsThread.initiateShutdown()) { + // Resume the topic deletion so it doesn't block on the condition + resumeTopicDeletionThread() + // Await delete topic thread to exit + deleteTopicsThread.awaitShutdown() + topicsToBeDeleted.clear() + partitionsToBeDeleted.clear() + topicsIneligibleForDeletion.clear() + } } /** @@ -102,8 +118,11 @@ class TopicDeletionManager(controller: KafkaController, * @param topics Topics that should be deleted */ def enqueueTopicsForDeletion(topics: Set[String]) { - topicsToBeDeleted ++= topics - resumeTopicDeletionThread() + if(isDeleteTopicEnabled) { + topicsToBeDeleted ++= topics + partitionsToBeDeleted ++= topics.flatMap(controllerContext.partitionsForTopic) + resumeTopicDeletionThread() + } } /** @@ -115,30 +134,34 @@ class TopicDeletionManager(controller: KafkaController, * @param topics Topics for which deletion can be resumed */ def resumeDeletionForTopics(topics: Set[String] = Set.empty) { - val topicsToResumeDeletion = topics & topicsToBeDeleted - if(topicsToResumeDeletion.size > 0) { - haltedTopicsForDeletion --= topicsToResumeDeletion - resumeTopicDeletionThread() + if(isDeleteTopicEnabled) { + val topicsToResumeDeletion = topics & topicsToBeDeleted + if(topicsToResumeDeletion.size > 0) { + topicsIneligibleForDeletion --= topicsToResumeDeletion + resumeTopicDeletionThread() + } } } /** * Invoked when a broker that hosts replicas for topics to be deleted goes down. Also invoked when the callback for * StopReplicaResponse receives an error code for the replicas of a topic to be deleted. As part of this, the replicas - * are moved from ReplicaDeletionStarted to ReplicaDeletionFailed state. Also, the topic is added to the list of topics - * for which deletion is halted until further notice. The delete topic thread is notified so it can retry topic deletion + * are moved from ReplicaDeletionStarted to ReplicaDeletionIneligible state. Also, the topic is added to the list of topics + * ineligible for deletion until further notice. The delete topic thread is notified so it can retry topic deletion * if it has received a response for all replicas of a topic to be deleted * @param replicas Replicas for which deletion has failed */ def failReplicaDeletion(replicas: Set[PartitionAndReplica]) { - val replicasThatFailedToDelete = replicas.filter(r => isTopicQueuedUpForDeletion(r.topic)) - if(replicasThatFailedToDelete.size > 0) { - val topics = replicasThatFailedToDelete.map(_.topic) - debug("Deletion failed for replicas %s. Halting deletion for topics %s" - .format(replicasThatFailedToDelete.mkString(","), topics)) - controller.replicaStateMachine.handleStateChanges(replicasThatFailedToDelete, ReplicaDeletionFailed) - haltTopicDeletion(topics) - resumeTopicDeletionThread() + if(isDeleteTopicEnabled) { + val replicasThatFailedToDelete = replicas.filter(r => isTopicQueuedUpForDeletion(r.topic)) + if(replicasThatFailedToDelete.size > 0) { + val topics = replicasThatFailedToDelete.map(_.topic) + debug("Deletion failed for replicas %s. Halting deletion for topics %s" + .format(replicasThatFailedToDelete.mkString(","), topics)) + controller.replicaStateMachine.handleStateChanges(replicasThatFailedToDelete, ReplicaDeletionIneligible) + markTopicIneligibleForDeletion(topics) + resumeTopicDeletionThread() + } } } @@ -147,25 +170,43 @@ class TopicDeletionManager(controller: KafkaController, * 1. replicas being down * 2. partition reassignment in progress for some partitions of the topic * 3. preferred replica election in progress for some partitions of the topic - * @param topics Topics for which deletion should be halted. No op if the topic is was not previously queued up for deletion + * @param topics Topics that should be marked ineligible for deletion. No op if the topic is was not previously queued up for deletion */ - def haltTopicDeletion(topics: Set[String]) { - val newTopicsToHaltDeletion = topicsToBeDeleted & topics - haltedTopicsForDeletion ++= newTopicsToHaltDeletion - if(newTopicsToHaltDeletion.size > 0) - info("Halted deletion of topics %s".format(newTopicsToHaltDeletion.mkString(","))) + def markTopicIneligibleForDeletion(topics: Set[String]) { + if(isDeleteTopicEnabled) { + val newTopicsToHaltDeletion = topicsToBeDeleted & topics + topicsIneligibleForDeletion ++= newTopicsToHaltDeletion + if(newTopicsToHaltDeletion.size > 0) + info("Halted deletion of topics %s".format(newTopicsToHaltDeletion.mkString(","))) + } } - def isTopicDeletionHalted(topic: String): Boolean = { - haltedTopicsForDeletion.contains(topic) + def isTopicIneligibleForDeletion(topic: String): Boolean = { + if(isDeleteTopicEnabled) { + topicsIneligibleForDeletion.contains(topic) + } else + true } def isTopicDeletionInProgress(topic: String): Boolean = { - controller.replicaStateMachine.isAtLeastOneReplicaInDeletionStartedState(topic) + if(isDeleteTopicEnabled) { + controller.replicaStateMachine.isAtLeastOneReplicaInDeletionStartedState(topic) + } else + false + } + + def isPartitionToBeDeleted(topicAndPartition: TopicAndPartition) = { + if(isDeleteTopicEnabled) { + partitionsToBeDeleted.contains(topicAndPartition) + } else + false } def isTopicQueuedUpForDeletion(topic: String): Boolean = { - topicsToBeDeleted.contains(topic) + if(isDeleteTopicEnabled) { + topicsToBeDeleted.contains(topic) + } else + false } /** @@ -173,19 +214,22 @@ class TopicDeletionManager(controller: KafkaController, * controllerLock should be acquired before invoking this API */ private def awaitTopicDeletionNotification() { - while(!deleteTopicStateChanged) { - info("Waiting for signal to start or continue topic deletion") - deleteTopicsCond.await() + inLock(deleteLock) { + while(deleteTopicsThread.isRunning.get() && !deleteTopicStateChanged.compareAndSet(true, false)) { + debug("Waiting for signal to start or continue topic deletion") + deleteTopicsCond.await() + } } - deleteTopicStateChanged = false } /** * Signals the delete-topic-thread to process topic deletion */ private def resumeTopicDeletionThread() { - deleteTopicStateChanged = true - deleteTopicsCond.signal() + deleteTopicStateChanged.set(true) + inLock(deleteLock) { + deleteTopicsCond.signal() + } } /** @@ -205,26 +249,31 @@ class TopicDeletionManager(controller: KafkaController, * Topic deletion can be retried if - * 1. Topic deletion is not already complete * 2. Topic deletion is currently not in progress for that topic - * 3. Topic deletion is currently halted for that topic + * 3. Topic is currently marked ineligible for deletion * @param topic Topic * @return Whether or not deletion can be retried for the topic */ private def isTopicEligibleForDeletion(topic: String): Boolean = { - topicsToBeDeleted.contains(topic) && (!isTopicDeletionInProgress(topic) && !isTopicDeletionHalted(topic)) + topicsToBeDeleted.contains(topic) && (!isTopicDeletionInProgress(topic) && !isTopicIneligibleForDeletion(topic)) } /** * If the topic is queued for deletion but deletion is not currently under progress, then deletion is retried for that topic - * To ensure a successful retry, reset states for respective replicas from ReplicaDeletionFailed to OfflineReplica state + * To ensure a successful retry, reset states for respective replicas from ReplicaDeletionIneligible to OfflineReplica state *@param topic Topic for which deletion should be retried */ private def markTopicForDeletionRetry(topic: String) { - // reset replica states from ReplicaDeletionFailed to OfflineReplica - val failedReplicas = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionFailed) + // reset replica states from ReplicaDeletionIneligible to OfflineReplica + val failedReplicas = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionIneligible) + info("Retrying delete topic for topic %s since replicas %s were not successfully deleted" + .format(topic, failedReplicas.mkString(","))) controller.replicaStateMachine.handleStateChanges(failedReplicas, OfflineReplica) } private def completeDeleteTopic(topic: String) { + // deregister partition change listener on the deleted topic. This is to prevent the partition change listener + // firing before the new topic listener when a deleted topic gets auto created + partitionStateMachine.deregisterPartitionChangeListener(topic) val replicasForDeletedTopic = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionSuccessful) // controller will remove this replica from the state machine as well as its partition assignment cache replicaStateMachine.handleStateChanges(replicasForDeletedTopic, NonExistentReplica) @@ -233,6 +282,7 @@ class TopicDeletionManager(controller: KafkaController, partitionStateMachine.handleStateChanges(partitionsForDeletedTopic, OfflinePartition) partitionStateMachine.handleStateChanges(partitionsForDeletedTopic, NonExistentPartition) topicsToBeDeleted -= topic + partitionsToBeDeleted.retain(_.topic != topic) controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicPath(topic)) controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicConfigPath(topic)) controllerContext.zkClient.delete(ZkUtils.getDeleteTopicPath(topic)) @@ -241,10 +291,16 @@ class TopicDeletionManager(controller: KafkaController, /** * This callback is invoked by the DeleteTopics thread with the list of topics to be deleted - * It invokes the delete partition callback for all partitions of a topic + * It invokes the delete partition callback for all partitions of a topic. + * The updateMetadataRequest is also going to set the leader for the topics being deleted to + * {@link LeaderAndIsr#LeaderDuringDelete}. This lets each broker know that this topic is being deleted and can be + * removed from their caches. */ private def onTopicDeletion(topics: Set[String]) { info("Topic deletion callback for %s".format(topics.mkString(","))) + // send update metadata so that brokers stop serving data for topics to be deleted + val partitions = topics.flatMap(controllerContext.partitionsForTopic) + controller.sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, partitions) val partitionReplicaAssignmentByTopic = controllerContext.partitionReplicaAssignment.groupBy(p => p._1.topic) topics.foreach { topic => onPartitionDeletion(partitionReplicaAssignmentByTopic(topic).map(_._1).toSet) @@ -257,42 +313,41 @@ class TopicDeletionManager(controller: KafkaController, * the topics are added to the in progress list. As long as a topic is in the in progress list, deletion for that topic * is never retried. A topic is removed from the in progress list when * 1. Either the topic is successfully deleted OR - * 2. No replica for the topic is in ReplicaDeletionStarted state and at least one replica is in ReplicaDeletionFailed state + * 2. No replica for the topic is in ReplicaDeletionStarted state and at least one replica is in ReplicaDeletionIneligible state * If the topic is queued for deletion but deletion is not currently under progress, then deletion is retried for that topic * As part of starting deletion, all replicas are moved to the ReplicaDeletionStarted state where the controller sends * the replicas a StopReplicaRequest (delete=true) * This callback does the following things - - * 1. Send metadata request to all brokers excluding the topics to be deleted - * 2. Move all dead replicas directly to ReplicaDeletionFailed state. Also halt the deletion of respective topics if - * some replicas are dead since it won't complete successfully anyway - * 3. Move all alive replicas to ReplicaDeletionStarted state so they can be deleted successfully + * 1. Move all dead replicas directly to ReplicaDeletionIneligible state. Also mark the respective topics ineligible + * for deletion if some replicas are dead since it won't complete successfully anyway + * 2. Move all alive replicas to ReplicaDeletionStarted state so they can be deleted successfully *@param replicasForTopicsToBeDeleted */ private def startReplicaDeletion(replicasForTopicsToBeDeleted: Set[PartitionAndReplica]) { replicasForTopicsToBeDeleted.groupBy(_.topic).foreach { case(topic, replicas) => - // send update metadata so that brokers stop serving data - controller.sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) var aliveReplicasForTopic = controllerContext.allLiveReplicas().filter(p => p.topic.equals(topic)) val deadReplicasForTopic = replicasForTopicsToBeDeleted -- aliveReplicasForTopic val successfullyDeletedReplicas = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionSuccessful) val replicasForDeletionRetry = aliveReplicasForTopic -- successfullyDeletedReplicas // move dead replicas directly to failed state - replicaStateMachine.handleStateChanges(deadReplicasForTopic, ReplicaDeletionFailed) + replicaStateMachine.handleStateChanges(deadReplicasForTopic, ReplicaDeletionIneligible) // send stop replica to all followers that are not in the OfflineReplica state so they stop sending fetch requests to the leader replicaStateMachine.handleStateChanges(replicasForDeletionRetry, OfflineReplica) debug("Deletion started for replicas %s".format(replicasForDeletionRetry.mkString(","))) controller.replicaStateMachine.handleStateChanges(replicasForDeletionRetry, ReplicaDeletionStarted, new Callbacks.CallbackBuilder().stopReplicaCallback(deleteTopicStopReplicaCallback).build) - if(deadReplicasForTopic.size > 0) - haltTopicDeletion(Set(topic)) + if(deadReplicasForTopic.size > 0) { + debug("Dead Replicas (%s) found for topic %s".format(deadReplicasForTopic.mkString(","), topic)) + markTopicIneligibleForDeletion(Set(topic)) + } } } /** * This callback is invoked by the delete topic callback with the list of partitions for topics to be deleted * It does the following - - * 1. Send UpdateMetadataRequest to all live brokers (that are not shutting down) with all partitions except those for - * which the topics are being deleted. The brokers start rejecting all client requests with UnknownTopicOrPartitionException + * 1. Send UpdateMetadataRequest to all live brokers (that are not shutting down) for partitions that are being + * deleted. The brokers start rejecting all client requests with UnknownTopicOrPartitionException * 2. Move all replicas for the partitions to OfflineReplica state. This will send StopReplicaRequest to the replicas * and LeaderAndIsrRequest to the leader with the shrunk ISR. When the leader replica itself is moved to OfflineReplica state, * it will skip sending the LeaderAndIsrRequest since the leader will be updated to -1 @@ -314,7 +369,7 @@ class TopicDeletionManager(controller: KafkaController, stopReplicaResponse.responseMap.filter(p => p._2 != ErrorMapping.NoError).map(_._1).toSet val replicasInError = partitionsInError.map(p => PartitionAndReplica(p.topic, p.partition, replicaId)) inLock(controllerContext.controllerLock) { - // move all the failed replicas to ReplicaDeletionFailed + // move all the failed replicas to ReplicaDeletionIneligible failReplicaDeletion(replicasInError) if(replicasInError.size != stopReplicaResponse.responseMap.size) { // some replicas could have been successfully deleted @@ -324,16 +379,22 @@ class TopicDeletionManager(controller: KafkaController, } } - class DeleteTopicsThread() extends ShutdownableThread("delete-topics-thread") { + class DeleteTopicsThread() extends ShutdownableThread(name = "delete-topics-thread-" + controller.config.brokerId, isInterruptible = false) { val zkClient = controllerContext.zkClient override def doWork() { + awaitTopicDeletionNotification() + + if (!isRunning.get) + return + inLock(controllerContext.controllerLock) { - awaitTopicDeletionNotification() val topicsQueuedForDeletion = Set.empty[String] ++ topicsToBeDeleted - if(topicsQueuedForDeletion.size > 0) + + if(!topicsQueuedForDeletion.isEmpty) info("Handling deletion for topics " + topicsQueuedForDeletion.mkString(",")) + topicsQueuedForDeletion.foreach { topic => - // if all replicas are marked as deleted successfully, then topic deletion is done + // if all replicas are marked as deleted successfully, then topic deletion is done if(controller.replicaStateMachine.areAllReplicasForTopicDeleted(topic)) { // clear up all state for this topic from controller cache and zookeeper completeDeleteTopic(topic) @@ -348,13 +409,12 @@ class TopicDeletionManager(controller: KafkaController, partitions.mkString(","), topic)) } else { // if you come here, then no replica is in TopicDeletionStarted and all replicas are not in - // TopicDeletionSuccessful. That means, there is at least one failed replica, which means topic deletion - // should be retried - val replicasInTopicDeletionFailedState = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionFailed) - // mark topic for deletion retry - markTopicForDeletionRetry(topic) - info("Retrying delete topic for topic %s since replicas %s were not successfully deleted" - .format(topic, replicasInTopicDeletionFailedState.mkString(","))) + // TopicDeletionSuccessful. That means, that either given topic haven't initiated deletion + // or there is at least one failed replica (which means topic deletion should be retried). + if(controller.replicaStateMachine.isAnyReplicaInState(topic, ReplicaDeletionIneligible)) { + // mark topic for deletion retry + markTopicForDeletionRetry(topic) + } } } // Try delete topic if it is eligible for deletion. @@ -362,12 +422,11 @@ class TopicDeletionManager(controller: KafkaController, info("Deletion of topic %s (re)started".format(topic)) // topic deletion will be kicked off onTopicDeletion(Set(topic)) - } else if(isTopicDeletionHalted(topic)) { - info("Not retrying deletion of topic %s at this time since it is halted".format(topic)) + } else if(isTopicIneligibleForDeletion(topic)) { + info("Not retrying deletion of topic %s at this time since it is marked ineligible for deletion".format(topic)) } } } } } } - diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala new file mode 100644 index 0000000000000..6c2df4c4f9d26 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -0,0 +1,492 @@ +/** + * 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.coordinator + +import kafka.common.{OffsetMetadataAndError, OffsetAndMetadata, TopicAndPartition} +import kafka.message.UncompressedCodec +import kafka.log.LogConfig +import kafka.server._ +import kafka.utils._ +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.JoinGroupRequest + +import org.I0Itec.zkclient.ZkClient +import java.util.concurrent.atomic.AtomicBoolean +import java.util.Properties +import scala.collection.{Map, Seq, immutable} + +case class GroupManagerConfig(consumerMinSessionTimeoutMs: Int, + consumerMaxSessionTimeoutMs: Int) + +/** + * ConsumerCoordinator handles consumer group and consumer offset management. + * + * Each Kafka server instantiates a coordinator which is responsible for a set of + * consumer groups. Consumer groups are assigned to coordinators based on their + * group names. + */ +class ConsumerCoordinator(val brokerId: Int, + val groupConfig: GroupManagerConfig, + val offsetConfig: OffsetManagerConfig, + private val offsetManager: OffsetManager, + zkClient: ZkClient) extends Logging { + + this.logIdent = "[ConsumerCoordinator " + brokerId + "]: " + + private val isActive = new AtomicBoolean(false) + + private var heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat] = null + private var rebalancePurgatory: DelayedOperationPurgatory[DelayedRebalance] = null + private var coordinatorMetadata: CoordinatorMetadata = null + + def this(brokerId: Int, + groupConfig: GroupManagerConfig, + offsetConfig: OffsetManagerConfig, + replicaManager: ReplicaManager, + zkClient: ZkClient, + scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig, + new OffsetManager(offsetConfig, replicaManager, zkClient, scheduler), zkClient) + + def offsetsTopicConfigs: Properties = { + val props = new Properties + props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + props.put(LogConfig.SegmentBytesProp, offsetConfig.offsetsTopicSegmentBytes.toString) + props.put(LogConfig.CompressionTypeProp, UncompressedCodec.name) + props + } + + /** + * NOTE: If a group lock and metadataLock are simultaneously needed, + * be sure to acquire the group lock before metadataLock to prevent deadlock + */ + + /** + * Startup logic executed at the same time when the server starts up. + */ + def startup() { + info("Starting up.") + heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", brokerId) + rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", brokerId) + coordinatorMetadata = new CoordinatorMetadata(brokerId, zkClient, maybePrepareRebalance) + isActive.set(true) + info("Startup complete.") + } + + /** + * Shutdown logic executed at the same time when server shuts down. + * Ordering of actions should be reversed from the startup process. + */ + def shutdown() { + info("Shutting down.") + isActive.set(false) + offsetManager.shutdown() + coordinatorMetadata.shutdown() + heartbeatPurgatory.shutdown() + rebalancePurgatory.shutdown() + info("Shutdown complete.") + } + + def handleJoinGroup(groupId: String, + consumerId: String, + topics: Set[String], + sessionTimeoutMs: Int, + partitionAssignmentStrategy: String, + responseCallback:(Set[TopicAndPartition], String, Int, Short) => Unit) { + if (!isActive.get) { + responseCallback(Set.empty, consumerId, 0, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code) + } else if (!isCoordinatorForGroup(groupId)) { + responseCallback(Set.empty, consumerId, 0, Errors.NOT_COORDINATOR_FOR_CONSUMER.code) + } else if (!PartitionAssignor.strategies.contains(partitionAssignmentStrategy)) { + responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code) + } else if (sessionTimeoutMs < groupConfig.consumerMinSessionTimeoutMs || + sessionTimeoutMs > groupConfig.consumerMaxSessionTimeoutMs) { + responseCallback(Set.empty, consumerId, 0, Errors.INVALID_SESSION_TIMEOUT.code) + } else { + // only try to create the group if the group is not unknown AND + // the consumer id is UNKNOWN, if consumer is specified but group does not + // exist we should reject the request + var group = coordinatorMetadata.getGroup(groupId) + if (group == null) { + if (consumerId != JoinGroupRequest.UNKNOWN_CONSUMER_ID) { + responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_CONSUMER_ID.code) + } else { + group = coordinatorMetadata.addGroup(groupId, partitionAssignmentStrategy) + doJoinGroup(group, consumerId, topics, sessionTimeoutMs, partitionAssignmentStrategy, responseCallback) + } + } else { + doJoinGroup(group, consumerId, topics, sessionTimeoutMs, partitionAssignmentStrategy, responseCallback) + } + } + } + + private def doJoinGroup(group: ConsumerGroupMetadata, + consumerId: String, + topics: Set[String], + sessionTimeoutMs: Int, + partitionAssignmentStrategy: String, + responseCallback:(Set[TopicAndPartition], String, Int, Short) => Unit) { + group synchronized { + if (group.is(Dead)) { + // if the group is marked as dead, it means some other thread has just removed the group + // from the coordinator metadata; this is likely that the group has migrated to some other + // coordinator OR the group is in a transient unstable phase. Let the consumer to retry + // joining without specified consumer id, + responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_CONSUMER_ID.code) + } else if (partitionAssignmentStrategy != group.partitionAssignmentStrategy) { + responseCallback(Set.empty, consumerId, 0, Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code) + } else if (consumerId != JoinGroupRequest.UNKNOWN_CONSUMER_ID && !group.has(consumerId)) { + // if the consumer trying to register with a un-recognized id, send the response to let + // it reset its consumer id and retry + responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_CONSUMER_ID.code) + } else if (group.has(consumerId) && group.is(Stable) && topics == group.get(consumerId).topics) { + /* + * if an existing consumer sends a JoinGroupRequest with no changes while the group is stable, + * just treat it like a heartbeat and return their currently assigned partitions. + */ + val consumer = group.get(consumerId) + completeAndScheduleNextHeartbeatExpiration(group, consumer) + responseCallback(consumer.assignedTopicPartitions, consumerId, group.generationId, Errors.NONE.code) + } else { + val consumer = if (consumerId == JoinGroupRequest.UNKNOWN_CONSUMER_ID) { + // if the consumer id is unknown, register this consumer to the group + val generatedConsumerId = group.generateNextConsumerId + val consumer = addConsumer(generatedConsumerId, topics, sessionTimeoutMs, group) + maybePrepareRebalance(group) + consumer + } else { + val consumer = group.get(consumerId) + if (topics != consumer.topics) { + // existing consumer changed its subscribed topics + updateConsumer(group, consumer, topics) + maybePrepareRebalance(group) + consumer + } else { + // existing consumer rejoining a group due to rebalance + consumer + } + } + + consumer.awaitingRebalanceCallback = responseCallback + + if (group.is(PreparingRebalance)) + rebalancePurgatory.checkAndComplete(ConsumerGroupKey(group.groupId)) + } + } + } + + def handleHeartbeat(groupId: String, + consumerId: String, + generationId: Int, + responseCallback: Short => Unit) { + if (!isActive.get) { + responseCallback(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code) + } else if (!isCoordinatorForGroup(groupId)) { + responseCallback(Errors.NOT_COORDINATOR_FOR_CONSUMER.code) + } else { + val group = coordinatorMetadata.getGroup(groupId) + if (group == null) { + // if the group is marked as dead, it means some other thread has just removed the group + // from the coordinator metadata; this is likely that the group has migrated to some other + // coordinator OR the group is in a transient unstable phase. Let the consumer to retry + // joining without specified consumer id, + responseCallback(Errors.UNKNOWN_CONSUMER_ID.code) + } else { + group synchronized { + if (group.is(Dead)) { + responseCallback(Errors.UNKNOWN_CONSUMER_ID.code) + } else if (!group.has(consumerId)) { + responseCallback(Errors.UNKNOWN_CONSUMER_ID.code) + } else if (generationId != group.generationId || !group.is(Stable)) { + responseCallback(Errors.ILLEGAL_GENERATION.code) + } else { + val consumer = group.get(consumerId) + completeAndScheduleNextHeartbeatExpiration(group, consumer) + responseCallback(Errors.NONE.code) + } + } + } + } + } + + def handleCommitOffsets(groupId: String, + consumerId: String, + generationId: Int, + offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], + responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { + if (!isActive.get) { + responseCallback(offsetMetadata.mapValues(_ => Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code)) + } else if (!isCoordinatorForGroup(groupId)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.NOT_COORDINATOR_FOR_CONSUMER.code)) + } else { + val group = coordinatorMetadata.getGroup(groupId) + if (group == null) { + // if the group does not exist, it means this group is not relying + // on Kafka for partition management, and hence never send join-group + // request to the coordinator before; in this case blindly commit the offsets + offsetManager.storeOffsets(groupId, consumerId, generationId, offsetMetadata, responseCallback) + } else { + group synchronized { + if (group.is(Dead)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_CONSUMER_ID.code)) + } else if (!group.has(consumerId)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_CONSUMER_ID.code)) + } else if (generationId != group.generationId) { + responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code)) + } else if (!offsetMetadata.keySet.subsetOf(group.get(consumerId).assignedTopicPartitions)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code)) + } else { + offsetManager.storeOffsets(groupId, consumerId, generationId, offsetMetadata, responseCallback) + } + } + } + } + } + + def handleFetchOffsets(groupId: String, + partitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = { + if (!isActive.get) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap + } else if (!isCoordinatorForGroup(groupId)) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap + } else { + val group = coordinatorMetadata.getGroup(groupId) + if (group == null) { + // if the group does not exist, it means this group is not relying + // on Kafka for partition management, and hence never send join-group + // request to the coordinator before; in this case blindly fetch the offsets + offsetManager.getOffsets(groupId, partitions) + } else { + group synchronized { + if (group.is(Dead)) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownConsumer)}.toMap + } else { + offsetManager.getOffsets(groupId, partitions) + } + } + } + } + } + + def handleGroupImmigration(offsetTopicPartitionId: Int) = { + // TODO we may need to add more logic in KAFKA-2017 + offsetManager.loadOffsetsFromLog(offsetTopicPartitionId) + } + + def handleGroupEmigration(offsetTopicPartitionId: Int) = { + // TODO we may need to add more logic in KAFKA-2017 + offsetManager.removeOffsetsFromCacheForPartition(offsetTopicPartitionId) + } + + /** + * Complete existing DelayedHeartbeats for the given consumer and schedule the next one + */ + private def completeAndScheduleNextHeartbeatExpiration(group: ConsumerGroupMetadata, consumer: ConsumerMetadata) { + // complete current heartbeat expectation + consumer.latestHeartbeat = SystemTime.milliseconds + val consumerKey = ConsumerKey(consumer.groupId, consumer.consumerId) + heartbeatPurgatory.checkAndComplete(consumerKey) + + // reschedule the next heartbeat expiration deadline + val newHeartbeatDeadline = consumer.latestHeartbeat + consumer.sessionTimeoutMs + val delayedHeartbeat = new DelayedHeartbeat(this, group, consumer, newHeartbeatDeadline, consumer.sessionTimeoutMs) + heartbeatPurgatory.tryCompleteElseWatch(delayedHeartbeat, Seq(consumerKey)) + } + + private def addConsumer(consumerId: String, + topics: Set[String], + sessionTimeoutMs: Int, + group: ConsumerGroupMetadata) = { + val consumer = new ConsumerMetadata(consumerId, group.groupId, topics, sessionTimeoutMs) + val topicsToBind = topics -- group.topics + group.add(consumer.consumerId, consumer) + coordinatorMetadata.bindGroupToTopics(group.groupId, topicsToBind) + consumer + } + + private def removeConsumer(group: ConsumerGroupMetadata, consumer: ConsumerMetadata) { + group.remove(consumer.consumerId) + val topicsToUnbind = consumer.topics -- group.topics + coordinatorMetadata.unbindGroupFromTopics(group.groupId, topicsToUnbind) + } + + private def updateConsumer(group: ConsumerGroupMetadata, consumer: ConsumerMetadata, topics: Set[String]) { + val topicsToBind = topics -- group.topics + group.remove(consumer.consumerId) + val topicsToUnbind = consumer.topics -- group.topics + group.add(consumer.consumerId, consumer) + consumer.topics = topics + coordinatorMetadata.bindAndUnbindGroupFromTopics(group.groupId, topicsToBind, topicsToUnbind) + } + + private def maybePrepareRebalance(group: ConsumerGroupMetadata) { + group synchronized { + if (group.canRebalance) + prepareRebalance(group) + } + } + + private def prepareRebalance(group: ConsumerGroupMetadata) { + group.transitionTo(PreparingRebalance) + info("Preparing to rebalance group %s with old generation %s".format(group.groupId, group.generationId)) + + val rebalanceTimeout = group.rebalanceTimeout + val delayedRebalance = new DelayedRebalance(this, group, rebalanceTimeout) + val consumerGroupKey = ConsumerGroupKey(group.groupId) + rebalancePurgatory.tryCompleteElseWatch(delayedRebalance, Seq(consumerGroupKey)) + } + + private def rebalance(group: ConsumerGroupMetadata) { + assert(group.notYetRejoinedConsumers == List.empty[ConsumerMetadata]) + + group.transitionTo(Rebalancing) + group.generationId += 1 + + info("Rebalancing group %s with new generation %s".format(group.groupId, group.generationId)) + + val assignedPartitionsPerConsumer = reassignPartitions(group) + trace("Rebalance for group %s generation %s has assigned partitions: %s" + .format(group.groupId, group.generationId, assignedPartitionsPerConsumer)) + + group.transitionTo(Stable) + info("Stabilized group %s generation %s".format(group.groupId, group.generationId)) + } + + private def onConsumerHeartbeatExpired(group: ConsumerGroupMetadata, consumer: ConsumerMetadata) { + trace("Consumer %s in group %s has failed".format(consumer.consumerId, group.groupId)) + removeConsumer(group, consumer) + maybePrepareRebalance(group) + } + + private def reassignPartitions(group: ConsumerGroupMetadata) = { + val assignor = PartitionAssignor.createInstance(group.partitionAssignmentStrategy) + val topicsPerConsumer = group.topicsPerConsumer + val partitionsPerTopic = coordinatorMetadata.partitionsPerTopic + val assignedPartitionsPerConsumer = assignor.assign(topicsPerConsumer, partitionsPerTopic) + assignedPartitionsPerConsumer.foreach { case (consumerId, partitions) => + group.get(consumerId).assignedTopicPartitions = partitions + } + assignedPartitionsPerConsumer + } + + def tryCompleteRebalance(group: ConsumerGroupMetadata, forceComplete: () => Boolean) = { + group synchronized { + if (group.notYetRejoinedConsumers == List.empty[ConsumerMetadata]) + forceComplete() + else false + } + } + + def onExpirationRebalance() { + // TODO: add metrics for rebalance timeouts + } + + def onCompleteRebalance(group: ConsumerGroupMetadata) { + group synchronized { + val failedConsumers = group.notYetRejoinedConsumers + if (group.isEmpty || !failedConsumers.isEmpty) { + failedConsumers.foreach { failedConsumer => + removeConsumer(group, failedConsumer) + // TODO: cut the socket connection to the consumer + } + + if (group.isEmpty) { + group.transitionTo(Dead) + info("Group %s generation %s is dead and removed".format(group.groupId, group.generationId)) + coordinatorMetadata.removeGroup(group.groupId, group.topics) + } + } + if (!group.is(Dead)) { + // assign partitions to existing consumers of the group according to the partitioning strategy + rebalance(group) + + // trigger the awaiting join group response callback for all the consumers after rebalancing + for (consumer <- group.allConsumers) { + assert(consumer.awaitingRebalanceCallback != null) + consumer.awaitingRebalanceCallback(consumer.assignedTopicPartitions, consumer.consumerId, group.generationId, Errors.NONE.code) + consumer.awaitingRebalanceCallback = null + completeAndScheduleNextHeartbeatExpiration(group, consumer) + } + } + } + } + + def tryCompleteHeartbeat(group: ConsumerGroupMetadata, consumer: ConsumerMetadata, heartbeatDeadline: Long, forceComplete: () => Boolean) = { + group synchronized { + if (shouldKeepConsumerAlive(consumer, heartbeatDeadline)) + forceComplete() + else false + } + } + + def onExpirationHeartbeat(group: ConsumerGroupMetadata, consumer: ConsumerMetadata, heartbeatDeadline: Long) { + group synchronized { + if (!shouldKeepConsumerAlive(consumer, heartbeatDeadline)) + onConsumerHeartbeatExpired(group, consumer) + } + } + + def onCompleteHeartbeat() { + // TODO: add metrics for complete heartbeats + } + + def partitionFor(group: String): Int = offsetManager.partitionFor(group) + + private def shouldKeepConsumerAlive(consumer: ConsumerMetadata, heartbeatDeadline: Long) = + consumer.awaitingRebalanceCallback != null || consumer.latestHeartbeat + consumer.sessionTimeoutMs > heartbeatDeadline + + private def isCoordinatorForGroup(groupId: String) = offsetManager.leaderIsLocal(offsetManager.partitionFor(groupId)) +} + +object ConsumerCoordinator { + + val OffsetsTopicName = "__consumer_offsets" + + def create(config: KafkaConfig, + zkClient: ZkClient, + replicaManager: ReplicaManager, + kafkaScheduler: KafkaScheduler): ConsumerCoordinator = { + val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + loadBufferSize = config.offsetsLoadBufferSize, + offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, + offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions = config.offsetsTopicPartitions, + offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, + offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, + offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) + val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs, + consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs) + + new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkClient, kafkaScheduler) + } + + def create(config: KafkaConfig, + zkClient: ZkClient, + offsetManager: OffsetManager): ConsumerCoordinator = { + val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + loadBufferSize = config.offsetsLoadBufferSize, + offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, + offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions = config.offsetsTopicPartitions, + offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, + offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, + offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) + val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs, + consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs) + + new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, offsetManager, zkClient) + } +} diff --git a/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala b/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala new file mode 100644 index 0000000000000..0e3657ff0e473 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala @@ -0,0 +1,133 @@ +/** + * 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.coordinator + +import kafka.utils.nonthreadsafe + +import java.util.UUID + +import collection.mutable + +private[coordinator] sealed trait GroupState { def state: Byte } + +/** + * Consumer group is preparing to rebalance + * + * action: respond to heartbeats with an ILLEGAL GENERATION error code + * transition: some consumers have joined by the timeout => Rebalancing + * all consumers have left the group => Dead + */ +private[coordinator] case object PreparingRebalance extends GroupState { val state: Byte = 1 } + +/** + * Consumer group is rebalancing + * + * action: compute the group's partition assignment + * send the join-group response with new partition assignment when rebalance is complete + * transition: partition assignment has been computed => Stable + */ +private[coordinator] case object Rebalancing extends GroupState { val state: Byte = 2 } + +/** + * Consumer group is stable + * + * action: respond to consumer heartbeats normally + * transition: consumer failure detected via heartbeat => PreparingRebalance + * consumer join-group received => PreparingRebalance + * zookeeper topic watcher fired => PreparingRebalance + */ +private[coordinator] case object Stable extends GroupState { val state: Byte = 3 } + +/** + * Consumer group has no more members + * + * action: none + * transition: none + */ +private[coordinator] case object Dead extends GroupState { val state: Byte = 4 } + + +private object ConsumerGroupMetadata { + private val validPreviousStates: Map[GroupState, Set[GroupState]] = + Map(Dead -> Set(PreparingRebalance), + Stable -> Set(Rebalancing), + PreparingRebalance -> Set(Stable), + Rebalancing -> Set(PreparingRebalance)) +} + +/** + * Group contains the following metadata: + * + * Membership metadata: + * 1. Consumers registered in this group + * 2. Partition assignment strategy for this group + * + * State metadata: + * 1. group state + * 2. generation id + */ +@nonthreadsafe +private[coordinator] class ConsumerGroupMetadata(val groupId: String, + val partitionAssignmentStrategy: String) { + + private val consumers = new mutable.HashMap[String, ConsumerMetadata] + private var state: GroupState = Stable + var generationId = 0 + + def is(groupState: GroupState) = state == groupState + def has(consumerId: String) = consumers.contains(consumerId) + def get(consumerId: String) = consumers(consumerId) + + def add(consumerId: String, consumer: ConsumerMetadata) { + consumers.put(consumerId, consumer) + } + + def remove(consumerId: String) { + consumers.remove(consumerId) + } + + def isEmpty = consumers.isEmpty + + def topicsPerConsumer = consumers.mapValues(_.topics).toMap + + def topics = consumers.values.flatMap(_.topics).toSet + + def notYetRejoinedConsumers = consumers.values.filter(_.awaitingRebalanceCallback == null).toList + + def allConsumers = consumers.values.toList + + def rebalanceTimeout = consumers.values.foldLeft(0) {(timeout, consumer) => + timeout.max(consumer.sessionTimeoutMs) + } + + // TODO: decide if ids should be predictable or random + def generateNextConsumerId = UUID.randomUUID().toString + + def canRebalance = state == Stable + + def transitionTo(groupState: GroupState) { + assertValidTransition(groupState) + state = groupState + } + + private def assertValidTransition(targetState: GroupState) { + if (!ConsumerGroupMetadata.validPreviousStates(targetState).contains(state)) + throw new IllegalStateException("Group %s should be in the %s states before moving to %s state. Instead it is in %s state" + .format(groupId, ConsumerGroupMetadata.validPreviousStates(targetState).mkString(","), targetState, state)) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/coordinator/ConsumerMetadata.scala b/core/src/main/scala/kafka/coordinator/ConsumerMetadata.scala new file mode 100644 index 0000000000000..d5486cf9c256c --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/ConsumerMetadata.scala @@ -0,0 +1,49 @@ +/** + * 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.coordinator + +import kafka.common.TopicAndPartition +import kafka.utils.nonthreadsafe + +/** + * Consumer metadata contains the following metadata: + * + * Heartbeat metadata: + * 1. negotiated heartbeat session timeout + * 2. timestamp of the latest heartbeat + * + * Subscription metadata: + * 1. subscribed topics + * 2. assigned partitions for the subscribed topics + * + * In addition, it also contains the following state information: + * + * 1. Awaiting rebalance callback: when the consumer group is in the prepare-rebalance state, + * its rebalance callback will be kept in the metadata if the + * consumer has sent the join group request + */ +@nonthreadsafe +private[coordinator] class ConsumerMetadata(val consumerId: String, + val groupId: String, + var topics: Set[String], + val sessionTimeoutMs: Int) { + + var awaitingRebalanceCallback: (Set[TopicAndPartition], String, Int, Short) => Unit = null + var assignedTopicPartitions = Set.empty[TopicAndPartition] + var latestHeartbeat: Long = -1 +} diff --git a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala new file mode 100644 index 0000000000000..2920320d2903d --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala @@ -0,0 +1,225 @@ +/** + * 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.coordinator + +import kafka.server.KafkaConfig +import kafka.utils.CoreUtils.{inReadLock, inWriteLock} +import kafka.utils.{threadsafe, ZkUtils, Logging} + +import org.I0Itec.zkclient.{ZkClient, IZkDataListener} + +import java.util.concurrent.locks.ReentrantReadWriteLock + +import scala.collection.mutable + +/** + * CoordinatorMetadata manages group and topic metadata. + * It delegates all group logic to the callers. + */ +@threadsafe +private[coordinator] class CoordinatorMetadata(brokerId: Int, + zkClient: ZkClient, + maybePrepareRebalance: ConsumerGroupMetadata => Unit) { + + /** + * NOTE: If a group lock and metadataLock are simultaneously needed, + * be sure to acquire the group lock before metadataLock to prevent deadlock + */ + private val metadataLock = new ReentrantReadWriteLock() + + /** + * These should be guarded by metadataLock + */ + private val groups = new mutable.HashMap[String, ConsumerGroupMetadata] + private val groupsPerTopic = new mutable.HashMap[String, Set[String]] + private val topicPartitionCounts = new mutable.HashMap[String, Int] + private val topicPartitionChangeListeners = new mutable.HashMap[String, TopicPartitionChangeListener] + + def shutdown() { + inWriteLock(metadataLock) { + topicPartitionChangeListeners.keys.foreach(deregisterTopicPartitionChangeListener) + topicPartitionChangeListeners.clear() + groups.clear() + groupsPerTopic.clear() + topicPartitionCounts.clear() + } + } + + def partitionsPerTopic = { + inReadLock(metadataLock) { + topicPartitionCounts.toMap + } + } + + /** + * Get the group associated with the given groupId, or null if not found + */ + def getGroup(groupId: String) = { + inReadLock(metadataLock) { + groups.get(groupId).orNull + } + } + + /** + * Add a group or get the group associated with the given groupId if it already exists + */ + def addGroup(groupId: String, partitionAssignmentStrategy: String) = { + inWriteLock(metadataLock) { + groups.getOrElseUpdate(groupId, new ConsumerGroupMetadata(groupId, partitionAssignmentStrategy)) + } + } + + /** + * Remove all metadata associated with the group, including its topics + * @param groupId the groupId of the group we are removing + * @param topicsForGroup topics that consumers in the group were subscribed to + */ + def removeGroup(groupId: String, topicsForGroup: Set[String]) { + inWriteLock(metadataLock) { + topicsForGroup.foreach(topic => unbindGroupFromTopics(groupId, topicsForGroup)) + groups.remove(groupId) + } + } + + /** + * Add the given group to the set of groups interested in + * topic partition changes for the given topics + */ + def bindGroupToTopics(groupId: String, topics: Set[String]) { + inWriteLock(metadataLock) { + require(groups.contains(groupId), "CoordinatorMetadata can only bind existing groups") + topics.foreach(topic => bindGroupToTopic(groupId, topic)) + } + } + + /** + * Remove the given group from the set of groups interested in + * topic partition changes for the given topics + */ + def unbindGroupFromTopics(groupId: String, topics: Set[String]) { + inWriteLock(metadataLock) { + require(groups.contains(groupId), "CoordinatorMetadata can only unbind existing groups") + topics.foreach(topic => unbindGroupFromTopic(groupId, topic)) + } + } + + /** + * Add the given group to the set of groups interested in the topicsToBind and + * remove the given group from the set of groups interested in the topicsToUnbind + */ + def bindAndUnbindGroupFromTopics(groupId: String, topicsToBind: Set[String], topicsToUnbind: Set[String]) { + inWriteLock(metadataLock) { + require(groups.contains(groupId), "CoordinatorMetadata can only update topic bindings for existing groups") + topicsToBind.foreach(topic => bindGroupToTopic(groupId, topic)) + topicsToUnbind.foreach(topic => unbindGroupFromTopic(groupId, topic)) + } + } + + private def isListeningToTopic(topic: String) = topicPartitionChangeListeners.contains(topic) + + private def bindGroupToTopic(groupId: String, topic: String) { + if (isListeningToTopic(topic)) { + val currentGroupsForTopic = groupsPerTopic(topic) + groupsPerTopic.put(topic, currentGroupsForTopic + groupId) + } + else { + groupsPerTopic.put(topic, Set(groupId)) + topicPartitionCounts.put(topic, getTopicPartitionCountFromZK(topic)) + registerTopicPartitionChangeListener(topic) + } + } + + private def unbindGroupFromTopic(groupId: String, topic: String) { + if (isListeningToTopic(topic)) { + val remainingGroupsForTopic = groupsPerTopic(topic) - groupId + if (remainingGroupsForTopic.isEmpty) { + // no other group cares about the topic, so erase all metadata associated with the topic + groupsPerTopic.remove(topic) + topicPartitionCounts.remove(topic) + deregisterTopicPartitionChangeListener(topic) + } else { + groupsPerTopic.put(topic, remainingGroupsForTopic) + } + } + } + + private def getTopicPartitionCountFromZK(topic: String) = { + val topicData = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq(topic)) + topicData(topic).size + } + + private def registerTopicPartitionChangeListener(topic: String) { + val listener = new TopicPartitionChangeListener + topicPartitionChangeListeners.put(topic, listener) + zkClient.subscribeDataChanges(ZkUtils.getTopicPath(topic), listener) + } + + private def deregisterTopicPartitionChangeListener(topic: String) { + val listener = topicPartitionChangeListeners(topic) + zkClient.unsubscribeDataChanges(ZkUtils.getTopicPath(topic), listener) + topicPartitionChangeListeners.remove(topic) + } + + /** + * Zookeeper listener to handle topic partition changes + */ + class TopicPartitionChangeListener extends IZkDataListener with Logging { + this.logIdent = "[TopicPartitionChangeListener on Coordinator " + brokerId + "]: " + + override def handleDataChange(dataPath: String, data: Object) { + info("Handling data change for path: %s data: %s".format(dataPath, data)) + val topic = topicFromDataPath(dataPath) + val numPartitions = getTopicPartitionCountFromZK(topic) + + val groupsToRebalance = inWriteLock(metadataLock) { + /* + * This condition exists because a consumer can leave and modify CoordinatorMetadata state + * while ZkClient begins handling the data change but before we acquire the metadataLock. + */ + if (isListeningToTopic(topic)) { + topicPartitionCounts.put(topic, numPartitions) + groupsPerTopic(topic).map(groupId => groups(groupId)) + } + else Set.empty[ConsumerGroupMetadata] + } + groupsToRebalance.foreach(maybePrepareRebalance) + } + + override def handleDataDeleted(dataPath: String) { + info("Handling data delete for path: %s".format(dataPath)) + val topic = topicFromDataPath(dataPath) + val groupsToRebalance = inWriteLock(metadataLock) { + /* + * This condition exists because a consumer can leave and modify CoordinatorMetadata state + * while ZkClient begins handling the data delete but before we acquire the metadataLock. + */ + if (isListeningToTopic(topic)) { + topicPartitionCounts.put(topic, 0) + groupsPerTopic(topic).map(groupId => groups(groupId)) + } + else Set.empty[ConsumerGroupMetadata] + } + groupsToRebalance.foreach(maybePrepareRebalance) + } + + private def topicFromDataPath(dataPath: String) = { + val nodes = dataPath.split("/") + nodes.last + } + } +} diff --git a/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala new file mode 100644 index 0000000000000..70a710cfed6b4 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala @@ -0,0 +1,35 @@ +/** + * 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.coordinator + +import kafka.server.DelayedOperation + +/** + * Delayed heartbeat operations that are added to the purgatory for session timeout checking. + * Heartbeats are paused during rebalance. + */ +private[coordinator] class DelayedHeartbeat(consumerCoordinator: ConsumerCoordinator, + group: ConsumerGroupMetadata, + consumer: ConsumerMetadata, + heartbeatDeadline: Long, + sessionTimeout: Long) + extends DelayedOperation(sessionTimeout) { + override def tryComplete(): Boolean = consumerCoordinator.tryCompleteHeartbeat(group, consumer, heartbeatDeadline, forceComplete) + override def onExpiration() = consumerCoordinator.onExpirationHeartbeat(group, consumer, heartbeatDeadline) + override def onComplete() = consumerCoordinator.onCompleteHeartbeat() +} diff --git a/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala b/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala new file mode 100644 index 0000000000000..8247d33e5bd48 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala @@ -0,0 +1,40 @@ +/** + * 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.coordinator + +import kafka.server.DelayedOperation + +/** + * Delayed rebalance operations that are added to the purgatory when group is preparing for rebalance + * + * Whenever a join-group request is received, check if all known consumers have requested + * to re-join the group; if yes, complete this operation to proceed rebalance. + * + * When the operation has expired, any known consumers that have not requested to re-join + * the group are marked as failed, and complete this operation to proceed rebalance with + * the rest of the group. + */ +private[coordinator] class DelayedRebalance(consumerCoordinator: ConsumerCoordinator, + group: ConsumerGroupMetadata, + sessionTimeout: Long) + extends DelayedOperation(sessionTimeout) { + + override def tryComplete(): Boolean = consumerCoordinator.tryCompleteRebalance(group, forceComplete) + override def onExpiration() = consumerCoordinator.onExpirationRebalance() + override def onComplete() = consumerCoordinator.onCompleteRebalance(group) +} diff --git a/core/src/main/scala/kafka/coordinator/PartitionAssignor.scala b/core/src/main/scala/kafka/coordinator/PartitionAssignor.scala new file mode 100644 index 0000000000000..8499bf86f1de2 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/PartitionAssignor.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.coordinator + +import kafka.common.TopicAndPartition +import kafka.utils.CoreUtils + +private[coordinator] trait PartitionAssignor { + /** + * Assigns partitions to consumers in a group. + * @return A mapping from consumer to assigned partitions. + */ + def assign(topicsPerConsumer: Map[String, Set[String]], + partitionsPerTopic: Map[String, Int]): Map[String, Set[TopicAndPartition]] + + protected def fill[K, V](vsPerK: Map[K, Set[V]], expectedKs: Set[K]): Map[K, Set[V]] = { + val unfilledKs = expectedKs -- vsPerK.keySet + vsPerK ++ unfilledKs.map(k => (k, Set.empty[V])) + } + + protected def aggregate[K, V](pairs: Seq[(K, V)]): Map[K, Set[V]] = { + pairs + .groupBy { case (k, v) => k } + .map { case (k, kvPairs) => (k, kvPairs.map(_._2).toSet) } + } + + protected def invert[K, V](vsPerK: Map[K, Set[V]]): Map[V, Set[K]] = { + val vkPairs = vsPerK.toSeq.flatMap { case (k, vs) => vs.map(v => (v, k)) } + aggregate(vkPairs) + } +} + +private[coordinator] object PartitionAssignor { + val strategies = Set("range", "roundrobin") + + def createInstance(strategy: String) = strategy match { + case "roundrobin" => new RoundRobinAssignor() + case _ => new RangeAssignor() + } +} + +/** + * The roundrobin assignor lays out all the available partitions and all the available consumers. It + * then proceeds to do a roundrobin assignment from partition to consumer. If the subscriptions of all consumer + * instances are identical, then the partitions will be uniformly distributed. (i.e., the partition ownership counts + * will be within a delta of exactly one across all consumers.) + * + * For example, suppose there are two consumers C0 and C1, two topics t0 and t1, and each topic has 3 partitions, + * resulting in partitions t0p0, t0p1, t0p2, t1p0, t1p1, and t1p2. + * + * The assignment will be: + * C0 -> [t0p0, t0p2, t1p1] + * C1 -> [t0p1, t1p0, t1p2] + */ +private[coordinator] class RoundRobinAssignor extends PartitionAssignor { + override def assign(topicsPerConsumer: Map[String, Set[String]], + partitionsPerTopic: Map[String, Int]): Map[String, Set[TopicAndPartition]] = { + val consumers = topicsPerConsumer.keys.toSeq.sorted + val topics = topicsPerConsumer.values.flatten.toSeq.distinct.sorted + + val allTopicPartitions = topics.flatMap { topic => + val numPartitionsForTopic = partitionsPerTopic(topic) + (0 until numPartitionsForTopic).map(partition => TopicAndPartition(topic, partition)) + } + + var consumerAssignor = CoreUtils.circularIterator(consumers) + val consumerPartitionPairs = allTopicPartitions.map { topicAndPartition => + consumerAssignor = consumerAssignor.dropWhile(consumerId => !topicsPerConsumer(consumerId).contains(topicAndPartition.topic)) + val consumer = consumerAssignor.next() + (consumer, topicAndPartition) + } + fill(aggregate(consumerPartitionPairs), topicsPerConsumer.keySet) + } +} + +/** + * The range assignor works on a per-topic basis. For each topic, we lay out the available partitions in numeric order + * and the consumers in lexicographic order. We then divide the number of partitions by the total number of + * consumers to determine the number of partitions to assign to each consumer. If it does not evenly + * divide, then the first few consumers will have one extra partition. + * + * For example, suppose there are two consumers C0 and C1, two topics t0 and t1, and each topic has 3 partitions, + * resulting in partitions t0p0, t0p1, t0p2, t1p0, t1p1, and t1p2. + * + * The assignment will be: + * C0 -> [t0p0, t0p1, t1p0, t1p1] + * C1 -> [t0p2, t1p2] + */ +private[coordinator] class RangeAssignor extends PartitionAssignor { + override def assign(topicsPerConsumer: Map[String, Set[String]], + partitionsPerTopic: Map[String, Int]): Map[String, Set[TopicAndPartition]] = { + val consumersPerTopic = invert(topicsPerConsumer) + val consumerPartitionPairs = consumersPerTopic.toSeq.flatMap { case (topic, consumersForTopic) => + val numPartitionsForTopic = partitionsPerTopic(topic) + + val numPartitionsPerConsumer = numPartitionsForTopic / consumersForTopic.size + val consumersWithExtraPartition = numPartitionsForTopic % consumersForTopic.size + + consumersForTopic.toSeq.sorted.zipWithIndex.flatMap { case (consumerForTopic, consumerIndex) => + val startPartition = numPartitionsPerConsumer * consumerIndex + consumerIndex.min(consumersWithExtraPartition) + val numPartitions = numPartitionsPerConsumer + (if (consumerIndex + 1 > consumersWithExtraPartition) 0 else 1) + + // The first few consumers pick up an extra partition, if any. + (startPartition until startPartition + numPartitions) + .map(partition => (consumerForTopic, TopicAndPartition(topic, partition))) + } + } + fill(aggregate(consumerPartitionPairs), topicsPerConsumer.keySet) + } +} diff --git a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala new file mode 100644 index 0000000000000..4345a8e80914d --- /dev/null +++ b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala @@ -0,0 +1,47 @@ +/** + * 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 java.nio.ByteBuffer +import kafka.cluster.BrokerEndPoint + +class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadataResponse) { + + def errorCode = underlying.errorCode + + def coordinator: BrokerEndPoint = { + import kafka.javaapi.Implicits._ + underlying.coordinatorOpt + } + + override def equals(other: Any) = canEqual(other) && { + val otherConsumerMetadataResponse = other.asInstanceOf[kafka.javaapi.ConsumerMetadataResponse] + this.underlying.equals(otherConsumerMetadataResponse.underlying) + } + + def canEqual(other: Any) = other.isInstanceOf[kafka.javaapi.ConsumerMetadataResponse] + + override def hashCode = underlying.hashCode + + override def toString = underlying.toString + +} + +object ConsumerMetadataResponse { + def readFrom(buffer: ByteBuffer) = new ConsumerMetadataResponse(kafka.api.ConsumerMetadataResponse.readFrom(buffer)) +} diff --git a/core/src/main/scala/kafka/javaapi/Implicits.scala b/core/src/main/scala/kafka/javaapi/Implicits.scala index 8baf4d468027a..c69b0a3c3d87e 100644 --- a/core/src/main/scala/kafka/javaapi/Implicits.scala +++ b/core/src/main/scala/kafka/javaapi/Implicits.scala @@ -47,9 +47,4 @@ private[javaapi] object Implicits extends Logging { } } - // used explicitly by ByteBufferMessageSet constructor as due to SI-4141 which affects Scala 2.8.1, implicits are not visible in constructors - implicit def javaListToScalaBuffer[A](l: java.util.List[A]) = { - import scala.collection.JavaConversions._ - l: collection.mutable.Buffer[A] - } } diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala index 57b9d2a9c1974..456c3c423a1dd 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala @@ -17,18 +17,17 @@ package kafka.javaapi -import kafka.common.{TopicAndPartition, OffsetMetadataAndError} -import collection.JavaConversions -import java.nio.ByteBuffer +import kafka.common.{OffsetAndMetadata, TopicAndPartition} class OffsetCommitRequest(groupId: String, - requestInfo: java.util.Map[TopicAndPartition, OffsetMetadataAndError], - versionId: Short, + requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata], correlationId: Int, - clientId: String) { + clientId: String, + versionId: Short) { val underlying = { - val scalaMap: Map[TopicAndPartition, OffsetMetadataAndError] = { - import JavaConversions._ + val scalaMap: collection.immutable.Map[TopicAndPartition, OffsetAndMetadata] = { + import collection.JavaConversions._ + requestInfo.toMap } kafka.api.OffsetCommitRequest( @@ -40,6 +39,15 @@ class OffsetCommitRequest(groupId: String, ) } + def this(groupId: String, + requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata], + correlationId: Int, + clientId: String) { + + // by default bind to version 0 so that it commits to Zookeeper + this(groupId, requestInfo, correlationId, clientId, 0) + } + override def toString = underlying.toString diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala index 570bf3173f485..b22232980e5ab 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala @@ -17,6 +17,8 @@ package kafka.javaapi +import java.nio.ByteBuffer + import kafka.common.TopicAndPartition import collection.JavaConversions @@ -24,8 +26,15 @@ class OffsetCommitResponse(private val underlying: kafka.api.OffsetCommitRespons def errors: java.util.Map[TopicAndPartition, Short] = { import JavaConversions._ - underlying.requestInfo + underlying.commitStatus } + def hasError = underlying.hasError + + def errorCode(topicAndPartition: TopicAndPartition) = underlying.commitStatus(topicAndPartition) + } +object OffsetCommitResponse { + def readFrom(buffer: ByteBuffer) = new OffsetCommitResponse(kafka.api.OffsetCommitResponse.readFrom(buffer)) +} diff --git a/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala index 5b4f4bbcdd606..818ae33034312 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala @@ -28,6 +28,14 @@ class OffsetFetchRequest(groupId: String, correlationId: Int, clientId: String) { + def this(groupId: String, + requestInfo: java.util.List[TopicAndPartition], + correlationId: Int, + clientId: String) { + // by default bind to version 0 so that it fetches from ZooKeeper + this(groupId, requestInfo, 0, correlationId, clientId) + } + val underlying = { val scalaSeq = { import JavaConversions._ diff --git a/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala index 60924d2fe543d..c4bdb128717ee 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala @@ -17,6 +17,8 @@ package kafka.javaapi +import java.nio.ByteBuffer + import kafka.common.{TopicAndPartition, OffsetMetadataAndError} import collection.JavaConversions @@ -29,3 +31,6 @@ class OffsetFetchResponse(private val underlying: kafka.api.OffsetFetchResponse) } +object OffsetFetchResponse { + def readFrom(buffer: ByteBuffer) = new OffsetFetchResponse(kafka.api.OffsetFetchResponse.readFrom(buffer)) +} diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala index d08c3f4af51e7..4ef832177d017 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala @@ -16,7 +16,7 @@ */ package kafka.javaapi -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import scala.collection.JavaConversions private[javaapi] object MetadataListImplicits { @@ -44,23 +44,25 @@ class TopicMetadata(private val underlying: kafka.api.TopicMetadata) { def errorCode: Short = underlying.errorCode def sizeInBytes: Int = underlying.sizeInBytes + + override def toString = underlying.toString } class PartitionMetadata(private val underlying: kafka.api.PartitionMetadata) { def partitionId: Int = underlying.partitionId - def leader: Broker = { + def leader: BrokerEndPoint = { import kafka.javaapi.Implicits._ underlying.leader } - def replicas: java.util.List[Broker] = { + def replicas: java.util.List[BrokerEndPoint] = { import JavaConversions._ underlying.replicas } - def isr: java.util.List[Broker] = { + def isr: java.util.List[BrokerEndPoint] = { import JavaConversions._ underlying.isr } @@ -68,5 +70,6 @@ class PartitionMetadata(private val underlying: kafka.api.PartitionMetadata) { def errorCode: Short = underlying.errorCode def sizeInBytes: Int = 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 7e6da164a26b1..568d0ac0b2ade 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -16,18 +16,17 @@ */ package kafka.javaapi -import kafka.api._ import java.nio.ByteBuffer + +import kafka.api._ + 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, + val correlationId: Int, val clientId: String, val topics: java.util.List[String]) - extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey), correlationId) { + extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) { val underlying: kafka.api.TopicMetadataRequest = { import scala.collection.JavaConversions._ diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataResponse.scala index 252a0c9d6c12b..335906055a694 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataResponse.scala @@ -23,4 +23,15 @@ class TopicMetadataResponse(private val underlying: kafka.api.TopicMetadataRespo import kafka.javaapi.MetadataListImplicits._ underlying.topicsMetadata } + + override def equals(other: Any) = canEqual(other) && { + val otherTopicMetadataResponse = other.asInstanceOf[kafka.javaapi.TopicMetadataResponse] + this.underlying.equals(otherTopicMetadataResponse.underlying) + } + + def canEqual(other: Any) = other.isInstanceOf[kafka.javaapi.TopicMetadataResponse] + + override def hashCode = underlying.hashCode + + override def toString = underlying.toString } diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java index c45c80399c21f..ca74ca8abf034 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java @@ -18,19 +18,22 @@ package kafka.javaapi.consumer; +import java.util.List; +import java.util.Map; + +import kafka.common.OffsetAndMetadata; +import kafka.common.TopicAndPartition; import kafka.consumer.KafkaStream; import kafka.consumer.TopicFilter; import kafka.serializer.Decoder; -import java.util.List; -import java.util.Map; - public interface ConsumerConnector { /** * Create a list of MessageStreams of type T for each topic. * * @param topicCountMap a map of (topic, #streams) pair - * @param decoder a decoder that converts from Message to T + * @param keyDecoder a decoder that decodes the message key + * @param valueDecoder a decoder that decodes the message itself * @return a map of (topic, list of KafkaStream) pairs. * The number of items in the list is #streams. Each stream supports * an iterator over message/metadata pairs. @@ -62,6 +65,15 @@ public interface ConsumerConnector { * Commit the offsets of all broker partitions connected by this connector. */ public void commitOffsets(); + public void commitOffsets(boolean retryOnFailure); + + /** + * Commit offsets using the provided offsets map + * + * @param offsetsToCommit a map containing the offset to commit for each partition. + * @param retryOnFailure enable retries on the offset commit if it fails. + */ + public void commitOffsets(Map offsetsToCommit, boolean retryOnFailure); /** * Shut down the connector diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java new file mode 100644 index 0000000000000..288ebd9b03ca1 --- /dev/null +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java @@ -0,0 +1,51 @@ +/** + * 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.consumer; + + +import kafka.consumer.ConsumerThreadId; +import java.util.Map; +import java.util.Set; + +/** + * This listener is used for execution of tasks defined by user when a consumer rebalance + * occurs in {@link kafka.consumer.ZookeeperConsumerConnector} + */ +public interface ConsumerRebalanceListener { + + /** + * This method is called after all the fetcher threads are stopped but before the + * ownership of partitions are released. Depending on whether auto offset commit is + * enabled or not, offsets may or may not have been committed. + * This listener is initially added to prevent duplicate messages on consumer rebalance + * in mirror maker, where offset auto commit is disabled to prevent data loss. It could + * also be used in more general cases. + * @param partitionOwnership The partition this consumer currently owns. + */ + public void beforeReleasingPartitions(Map> partitionOwnership); + + /** + * This method is called after the new partition assignment is finished but before fetcher + * threads start. A map of new global partition assignment is passed in as parameter. + * @param consumerId The consumer Id string of the consumer invoking this callback. + * @param globalPartitionAssignment A Map[topic, Map[Partition, ConsumerThreadId]]. It is the global partition + * assignment of this consumer group. + */ + public void beforeStartingFetchers(String consumerId, Map> globalPartitionAssignment); + +} diff --git a/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala index 0ab0195dc9f66..abf6069d2033b 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala @@ -80,7 +80,7 @@ class SimpleConsumer(val host: String, } /** - * Commit offsets for a topic + * Commit offsets for a topic to Zookeeper * @param request a [[kafka.javaapi.OffsetCommitRequest]] object. * @return a [[kafka.javaapi.OffsetCommitResponse]] object. */ @@ -90,7 +90,7 @@ class SimpleConsumer(val host: String, } /** - * Fetch offsets for a topic + * Fetch offsets for a topic from Zookeeper * @param request a [[kafka.javaapi.OffsetFetchRequest]] object. * @return a [[kafka.javaapi.OffsetFetchResponse]] object. */ diff --git a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala index 58e83f6119d32..bfd8d37b51e17 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala @@ -18,9 +18,9 @@ package kafka.javaapi.consumer import kafka.serializer._ import kafka.consumer._ -import scala.collection.mutable -import scala.collection.JavaConversions - +import kafka.common.{OffsetAndMetadata, TopicAndPartition, MessageStreamsExistException} +import scala.collection.{immutable, mutable, JavaConversions} +import java.util.concurrent.atomic.AtomicBoolean /** * This class handles the consumers interaction with zookeeper @@ -63,6 +63,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, extends ConsumerConnector { private val underlying = new kafka.consumer.ZookeeperConsumerConnector(config, enableFetcher) + private val messageStreamCreated = new AtomicBoolean(false) def this(config: ConsumerConfig) = this(config, true) @@ -73,6 +74,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, valueDecoder: Decoder[V]) : java.util.Map[String,java.util.List[KafkaStream[K,V]]] = { + if (messageStreamCreated.getAndSet(true)) + throw new MessageStreamsExistException(this.getClass.getSimpleName + + " can create message streams at most once",null) val scalaTopicCountMap: Map[String, Int] = { import JavaConversions._ Map.empty[String, Int] ++ (topicCountMap.asInstanceOf[java.util.Map[String, Int]]: mutable.Map[String, Int]) @@ -87,23 +91,35 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } ret } - + def createMessageStreams(topicCountMap: java.util.Map[String,java.lang.Integer]): java.util.Map[String,java.util.List[KafkaStream[Array[Byte],Array[Byte]]]] = createMessageStreams(topicCountMap, new DefaultDecoder(), new DefaultDecoder()) - + def createMessageStreamsByFilter[K,V](topicFilter: TopicFilter, numStreams: Int, keyDecoder: Decoder[K], valueDecoder: Decoder[V]) = { import JavaConversions._ underlying.createMessageStreamsByFilter(topicFilter, numStreams, keyDecoder, valueDecoder) } - def createMessageStreamsByFilter(topicFilter: TopicFilter, numStreams: Int) = + def createMessageStreamsByFilter(topicFilter: TopicFilter, numStreams: Int) = createMessageStreamsByFilter(topicFilter, numStreams, new DefaultDecoder(), new DefaultDecoder()) - - def createMessageStreamsByFilter(topicFilter: TopicFilter) = + + def createMessageStreamsByFilter(topicFilter: TopicFilter) = createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder(), new DefaultDecoder()) - + def commitOffsets() { - underlying.commitOffsets + underlying.commitOffsets(true) + } + + def commitOffsets(retryOnFailure: Boolean) { + underlying.commitOffsets(retryOnFailure) + } + + def commitOffsets(offsetsToCommit: java.util.Map[TopicAndPartition, OffsetAndMetadata], retryOnFailure: Boolean) { + underlying.commitOffsets(offsetsToCommit.asInstanceOf[immutable.Map[TopicAndPartition, OffsetAndMetadata]], retryOnFailure) + } + + def setConsumerRebalanceListener(consumerRebalanceListener: ConsumerRebalanceListener) { + underlying.setConsumerRebalanceListener(consumerRebalanceListener) } def shutdown() { diff --git a/core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala index fecee8d5f7b32..df3027958af23 100644 --- a/core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala @@ -17,17 +17,16 @@ package kafka.javaapi.message import java.util.concurrent.atomic.AtomicLong -import scala.reflect.BeanProperty import java.nio.ByteBuffer import kafka.message._ -import kafka.javaapi.Implicits.javaListToScalaBuffer -class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends MessageSet { +import scala.collection.JavaConverters._ + +class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet { private val underlying: kafka.message.ByteBufferMessageSet = new kafka.message.ByteBufferMessageSet(buffer) def this(compressionCodec: CompressionCodec, messages: java.util.List[Message]) { - // due to SI-4141 which affects Scala 2.8.1, implicits are not visible in constructors and must be used explicitly - this(new kafka.message.ByteBufferMessageSet(compressionCodec, new AtomicLong(0), javaListToScalaBuffer(messages).toSeq : _*).buffer) + this(new kafka.message.ByteBufferMessageSet(compressionCodec, new AtomicLong(0), messages.asScala: _*).buffer) } def this(messages: java.util.List[Message]) { @@ -36,6 +35,8 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message def validBytes: Int = underlying.validBytes + def getBuffer = buffer + override def iterator: java.util.Iterator[MessageAndOffset] = new java.util.Iterator[MessageAndOffset] { val underlyingIterator = underlying.iterator override def hasNext(): Boolean = { diff --git a/core/src/main/scala/kafka/log/CleanerConfig.scala b/core/src/main/scala/kafka/log/CleanerConfig.scala index fa946ada92602..782bc9adb2274 100644 --- a/core/src/main/scala/kafka/log/CleanerConfig.scala +++ b/core/src/main/scala/kafka/log/CleanerConfig.scala @@ -29,13 +29,13 @@ package kafka.log * @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 +case class CleanerConfig(numThreads: Int = 1, + dedupeBufferSize: Long = 4*1024*1024L, + dedupeBufferLoadFactor: Double = 0.9d, + ioBufferSize: Int = 1024*1024, + maxMessageSize: Int = 32*1024*1024, + maxIoBytesPerSecond: Double = Double.MaxValue, + backOffMs: Long = 15 * 1000, + enableCleaner: Boolean = true, + hashAlgorithm: String = "MD5") { +} diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala old mode 100644 new mode 100755 index e1f8b979c3e6f..39361fe942b20 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -54,7 +54,7 @@ class FileMessageSet private[kafka](@volatile var file: File, /* if this is not a slice, update the file pointer to the end of the file */ if (!isSlice) /* set the file position to the last byte in the file */ - channel.position(channel.size) + channel.position(math.min(channel.size().toInt, end)) /** * Create a file message set with no slicing. @@ -66,12 +66,25 @@ class FileMessageSet private[kafka](@volatile var file: File, * Create a file message set with no slicing */ def this(file: File) = - this(file, Utils.openChannel(file, mutable = true)) + this(file, FileMessageSet.openChannel(file, mutable = true)) + + /** + * Create a file message set with no slicing, and with initFileSize and preallocate. + * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize + * with one value (for example 512 * 1024 *1024 ) can improve the kafka produce performance. + * If it's new file and preallocate is true, end will be set to 0. Otherwise set to Int.MaxValue. + */ + def this(file: File, fileAlreadyExists: Boolean, initFileSize: Int, preallocate: Boolean) = + this(file, + channel = FileMessageSet.openChannel(file, mutable = true, fileAlreadyExists, initFileSize, preallocate), + start = 0, + end = ( if ( !fileAlreadyExists && preallocate ) 0 else Int.MaxValue), + isSlice = false) /** * Create a file message set with mutable option */ - def this(file: File, mutable: Boolean) = this(file, Utils.openChannel(file, mutable)) + def this(file: File, mutable: Boolean) = this(file, FileMessageSet.openChannel(file, mutable)) /** * Create a slice view of the file message set that begins and ends at the given byte offsets @@ -223,15 +236,23 @@ class FileMessageSet private[kafka](@volatile var file: File, */ def close() { flush() + trim() channel.close() } + /** + * Trim file when close or roll to next file + */ + def trim() { + truncateTo(sizeInBytes()) + } + /** * Delete this message set from the filesystem * @return True iff this message set was deleted. */ def delete(): Boolean = { - Utils.swallow(channel.close()) + CoreUtils.swallow(channel.close()) file.delete() } @@ -255,8 +276,8 @@ class FileMessageSet private[kafka](@volatile var file: File, /** * Read from the underlying file into the buffer starting at the given position */ - def readInto(buffer: ByteBuffer, position: Int): ByteBuffer = { - channel.read(buffer, position) + def readInto(buffer: ByteBuffer, relativePosition: Int): ByteBuffer = { + channel.read(buffer, relativePosition + this.start) buffer.flip() buffer } @@ -272,6 +293,37 @@ class FileMessageSet private[kafka](@volatile var file: File, } } + +object FileMessageSet +{ + /** + * Open a channel for the given file + * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize + * with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance. + * @param file File path + * @param mutable mutable + * @param fileAlreadyExists File already exists or not + * @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024 + * @param preallocate Pre allocate file or not, gotten from configuration. + */ + def openChannel(file: File, mutable: Boolean, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): FileChannel = { + if (mutable) { + if (fileAlreadyExists) + new RandomAccessFile(file, "rw").getChannel() + else { + if (preallocate) { + val randomAccessFile = new RandomAccessFile(file, "rw") + randomAccessFile.setLength(initFileSize) + randomAccessFile.getChannel() + } + else + new RandomAccessFile(file, "rw").getChannel() + } + } + else + new FileInputStream(file).getChannel() + } +} object LogFlushStats extends KafkaMetricsGroup { val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index b3ab5220a66a2..e5e80079645ce 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -17,17 +17,36 @@ package kafka.log -import java.io.{IOException, File} -import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap} -import java.util.concurrent.atomic._ import kafka.utils._ -import scala.collection.JavaConversions -import java.text.NumberFormat import kafka.message._ import kafka.common._ import kafka.metrics.KafkaMetricsGroup +import kafka.server.{LogOffsetMetadata, FetchDataInfo, BrokerTopicStats} + +import java.io.{IOException, File} +import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap} +import java.util.concurrent.atomic._ +import java.text.NumberFormat +import scala.collection.JavaConversions + import com.yammer.metrics.core.Gauge +object LogAppendInfo { + val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, NoCompressionCodec, -1, -1, false) +} + +/** + * 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 shallowCount The number of shallow messages + * @param validBytes The number of valid bytes + * @param sourceCodec The source codec used in the message set (send by the producer) + * @param targetCodec The target codec of the message set(after applying the broker compression configuration if any) + * @param offsetsMonotonic Are the offsets in this message set monotonically increasing + */ +case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, sourceCodec: CompressionCodec, targetCodec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) + /** * An append-only log for storing messages. @@ -48,33 +67,60 @@ import com.yammer.metrics.core.Gauge class Log(val dir: File, @volatile var config: LogConfig, @volatile var recoveryPoint: Long = 0L, - val scheduler: Scheduler, + 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 /* last time it was flushed */ private val lastflushedTime = new AtomicLong(time.milliseconds) + def initFileSize() : Int = { + if (config.preallocate) + config.segmentSize + else + 0 + } + /* the actual segments of the log */ 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()) + @volatile var nextOffsetMetadata = new LogOffsetMetadata(activeSegment.nextOffset(), activeSegment.baseOffset, activeSegment.size.toInt) - val topicAndPartition: TopicAndPartition = Log.parseTopicPartitionName(name) + val topicAndPartition: TopicAndPartition = Log.parseTopicPartitionName(dir) info("Completed load of log %s with log end offset %d".format(name, logEndOffset)) - newGauge(name + "-" + "NumLogSegments", - new Gauge[Int] { def value = numberOfSegments }) + val tags = Map("topic" -> topicAndPartition.topic, "partition" -> topicAndPartition.partition.toString) + + newGauge("NumLogSegments", + new Gauge[Int] { + def value = numberOfSegments + }, + tags) - newGauge(name + "-" + "LogEndOffset", - new Gauge[Long] { def value = logEndOffset }) + newGauge("LogStartOffset", + new Gauge[Long] { + def value = logStartOffset + }, + tags) + + newGauge("LogEndOffset", + new Gauge[Long] { + def value = logEndOffset + }, + tags) + + newGauge("Size", + new Gauge[Long] { + def value = size + }, + tags) /** The name of this log */ def name = dir.getName() @@ -83,9 +129,10 @@ class Log(val dir: File, private def loadSegments() { // create the log directory if it doesn't exist dir.mkdirs() + var swapFiles = Set[File]() // first do a pass through the files in the log directory and remove any temporary files - // and complete any interrupted swap operations + // and find any interrupted swap operations for(file <- dir.listFiles if file.isFile) { if(!file.canRead) throw new IOException("Could not read file " + file) @@ -95,21 +142,16 @@ class Log(val dir: File, 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 a log, delete the .index file, complete the swap operation later // if an index just delete it, it will be rebuilt - val baseName = new File(Utils.replaceSuffix(file.getPath, SwapFileSuffix, "")) + val baseName = new File(CoreUtils.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)) + val index = new File(CoreUtils.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)) + swapFiles += file } } } @@ -127,36 +169,75 @@ class Log(val dir: File, } 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 indexFile = Log.indexFilename(dir, start) val segment = new LogSegment(dir = dir, startOffset = start, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, - time = time) - if(!hasIndex) { + rollJitterMs = config.randomSegmentJitter, + time = time, + fileAlreadyExists = true) + + if(indexFile.exists()) { + try { + segment.index.sanityCheck() + } catch { + case e: java.lang.IllegalArgumentException => + warn("Found an corrupted index file, %s, deleting and rebuilding index...".format(indexFile.getAbsolutePath)) + indexFile.delete() + segment.recover(config.maxMessageSize) + } + } + else { 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) } } + + // Finally, complete any interrupted swap operations. To be crash-safe, + // log files that are replaced by the swap segment should be renamed to .deleted + // before the swap file is restored as the new segment file. + for (swapFile <- swapFiles) { + val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, "")) + val fileName = logFile.getName + val startOffset = fileName.substring(0, fileName.length - LogFileSuffix.length).toLong + val indexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, IndexFileSuffix) + SwapFileSuffix) + val index = new OffsetIndex(file = indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize) + val swapSegment = new LogSegment(new FileMessageSet(file = swapFile), + index = index, + baseOffset = startOffset, + indexIntervalBytes = config.indexInterval, + rollJitterMs = config.randomSegmentJitter, + time = time) + info("Found log file %s from interrupted swap operation, repairing.".format(swapFile.getPath)) + swapSegment.recover(config.maxMessageSize) + val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.nextOffset) + replaceSegments(swapSegment, oldSegments.toSeq, isRecoveredSwapFile = true) + } if(logSegments.size == 0) { // no existing segments, create a new mutable segment beginning at offset 0 - segments.put(0, new LogSegment(dir = dir, + segments.put(0L, new LogSegment(dir = dir, startOffset = 0, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, - time = time)) + rollJitterMs = config.randomSegmentJitter, + time = time, + fileAlreadyExists = false, + initFileSize = this.initFileSize(), + preallocate = config.preallocate)) } else { recoverLog() // reset the index size of the currently active log segment to allow more entries activeSegment.index.resize(config.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 updateLogEndOffset(messageOffset: Long) { + nextOffsetMetadata = new LogOffsetMetadata(messageOffset, activeSegment.baseOffset, activeSegment.size.toInt) } private def recoverLog() { @@ -232,47 +313,58 @@ class Log(val dir: File, return appendInfo // trim any invalid bytes or partial messages before appending it to the on-disk log - var validMessages = trimInvalidBytes(messages) + var validMessages = trimInvalidBytes(messages, appendInfo) try { // they are valid, insert them in the log lock synchronized { - appendInfo.firstOffset = nextOffset.get - - // maybe roll the log if this segment is full - val segment = maybeRoll() + appendInfo.firstOffset = nextOffsetMetadata.messageOffset if(assignOffsets) { - // assign offsets to the messageset - val offset = new AtomicLong(nextOffset.get) + // assign offsets to the message set + val offset = new AtomicLong(nextOffsetMetadata.messageOffset) try { - validMessages = validMessages.assignOffsets(offset, appendInfo.codec) + validMessages = validMessages.validateMessagesAndAssignOffsets(offset, appendInfo.sourceCodec, appendInfo.targetCodec, config.compact) } 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) + if(!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset) throw new IllegalArgumentException("Out of order offsets found in " + messages) } - // 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) + // re-validate message sizes since after re-compression some may exceed the limit for(messageAndOffset <- validMessages.shallowIterator) { - if(MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) + if(MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) { + // we record the original message set size instead of trimmed size + // to be consistent with pre-compression bytesRejectedRate recording + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes) throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d." .format(MessageSet.entrySize(messageAndOffset.message), config.maxMessageSize)) + } + } + + // check messages set size may be exceed config.segmentSize + if(validMessages.sizeInBytes > config.segmentSize) { + throw new MessageSetSizeTooLargeException("Message set size is %d bytes which exceeds the maximum configured segment size of %d." + .format(validMessages.sizeInBytes, config.segmentSize)) } + + // maybe roll the log if this segment is full + val segment = maybeRoll(validMessages.sizeInBytes) + // now append to the log segment.append(appendInfo.firstOffset, validMessages) // increment the log end offset - nextOffset.set(appendInfo.lastOffset + 1) + updateLogEndOffset(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)) + .format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validMessages)) if(unflushedMessages >= config.flushInterval) flush() @@ -284,18 +376,11 @@ class Log(val dir: File, } } - /** 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 + *
        2. each message size is valid *
        * * Also compute the following quantities: @@ -303,14 +388,16 @@ class Log(val dir: File, *
      • First offset in the message set *
      • Last offset in the message set *
      • Number of messages + *
      • Number of valid bytes *
      • Whether the offsets are monotonically increasing *
      • Whether any compression codec is used (if many are used, then the last one is given) * */ private def analyzeAndValidateMessageSet(messages: ByteBufferMessageSet): LogAppendInfo = { - var messageCount = 0 + var shallowMessageCount = 0 + var validBytesCount = 0 var firstOffset, lastOffset = -1L - var codec: CompressionCodec = NoCompressionCodec + var sourceCodec: CompressionCodec = NoCompressionCodec var monotonic = true for(messageAndOffset <- messages.shallowIterator) { // update the first offset if on the first message @@ -322,25 +409,42 @@ class Log(val dir: File, // update the last offset seen lastOffset = messageAndOffset.offset - // check the validity of the message by checking CRC val m = messageAndOffset.message + + // Check if the message sizes are valid. + val messageSize = MessageSet.entrySize(m) + if(messageSize > config.maxMessageSize) { + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes) + throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d." + .format(messageSize, config.maxMessageSize)) + } + + // check the validity of the message by checking CRC m.ensureValid() - messageCount += 1; - + + shallowMessageCount += 1 + validBytesCount += messageSize + val messageCodec = m.compressionCodec if(messageCodec != NoCompressionCodec) - codec = messageCodec + sourceCodec = messageCodec } - LogAppendInfo(firstOffset, lastOffset, codec, messageCount, monotonic) + + // Apply broker-side compression if any + val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec) + + LogAppendInfo(firstOffset, lastOffset, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic) } - + /** * Trim any invalid bytes from the end of this message set (if there are any) * @param messages The message set to trim + * @param info The general information of the message set * @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 + private def trimInvalidBytes(messages: ByteBufferMessageSet, info: LogAppendInfo): ByteBufferMessageSet = { + val messageSetValidBytes = info.validBytes if(messageSetValidBytes < 0) throw new InvalidMessageSizeException("Illegal length of message set " + messageSetValidBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests") if(messageSetValidBytes == messages.sizeInBytes) { @@ -355,20 +459,21 @@ class Log(val dir: File, /** * 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 + * @return The fetch data information including fetch starting offset metadata and messages read */ - def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None): MessageSet = { + def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None): FetchDataInfo = { trace("Reading %d bytes from offset %d in log %s of length %d bytes".format(maxLength, startOffset, name, size)) // check if the offset is valid and in range - val next = nextOffset.get + val next = nextOffsetMetadata.messageOffset if(startOffset == next) - return MessageSet.Empty + return FetchDataInfo(nextOffsetMetadata, MessageSet.Empty) var entry = segments.floorEntry(startOffset) @@ -380,15 +485,31 @@ class Log(val dir: File, // 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) + val fetchInfo = entry.getValue.read(startOffset, maxOffset, maxLength) + if(fetchInfo == null) { entry = segments.higherEntry(entry.getKey) - else - return messages + } else { + return fetchInfo + } } - // okay we are beyond the end of the last segment but less than the log end offset - MessageSet.Empty + // okay we are beyond the end of the last segment with no data fetched although the start offset is in range, + // this can happen when all messages with offset larger than start offsets have been deleted. + // In this case, we will return the empty set with log end offset metadata + FetchDataInfo(nextOffsetMetadata, MessageSet.Empty) + } + + /** + * Given a message offset, find its corresponding offset metadata in the log. + * If the message offset is out of range, return unknown offset metadata + */ + def convertToOffsetMetadata(offset: Long): LogOffsetMetadata = { + try { + val fetchDataInfo = read(offset, 1) + fetchDataInfo.fetchOffsetMetadata + } catch { + case e: OffsetOutOfRangeException => LogOffsetMetadata.UnknownOffsetMetadata + } } /** @@ -401,7 +522,7 @@ class Log(val dir: File, // 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 deletable = logSegments.takeWhile(s => predicate(s) && (s.baseOffset != lastSegment.baseOffset || s.size > 0)) val numToDelete = deletable.size if(numToDelete > 0) { lock synchronized { @@ -420,19 +541,37 @@ class Log(val dir: File, */ def size: Long = logSegments.map(_.size).sum + /** + * The earliest message offset in the log + */ + def logStartOffset: Long = logSegments.head.baseOffset + + /** + * The offset metadata of the next message that will be appended to the log + */ + def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata + /** * The offset of the next message that will be appended to the log */ - def logEndOffset: Long = nextOffset.get + def logEndOffset: Long = nextOffsetMetadata.messageOffset /** - * Roll the log over to a new empty log segment if necessary + * Roll the log over to a new empty log segment if necessary. + * + * @param messagesSize The messages set size in bytes + * logSegment will be rolled if one of the following conditions met + *
          + *
        1. The logSegment is full + *
        2. The maxTime has elapsed + *
        3. The index is full + *
        * @return The currently active segment after (perhaps) rolling to a new segment */ - private def maybeRoll(): LogSegment = { + private def maybeRoll(messagesSize: Int): LogSegment = { val segment = activeSegment - if (segment.size > config.segmentSize || - segment.size > 0 && time.milliseconds - segment.created > config.segmentMs || + if (segment.size > config.segmentSize - messagesSize || + segment.size > 0 && time.milliseconds - segment.created > config.segmentMs - segment.rollJitterMs || segment.index.isFull) { debug("Rolling new log segment in %s (log_size = %d/%d, index_size = %d/%d, age_ms = %d/%d)." .format(name, @@ -441,7 +580,7 @@ class Log(val dir: File, segment.index.entries, segment.index.maxEntries, time.milliseconds - segment.created, - config.segmentMs)) + config.segmentMs - segment.rollJitterMs)) roll() } else { segment @@ -466,13 +605,20 @@ class Log(val dir: File, segments.lastEntry() match { case null => - case entry => entry.getValue.index.trimToValidSize() + case entry => { + entry.getValue.index.trimToValidSize() + entry.getValue.log.trim() + } } val segment = new LogSegment(dir, startOffset = newOffset, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, - time = time) + rollJitterMs = config.randomSegmentJitter, + time = time, + fileAlreadyExists = false, + initFileSize = initFileSize, + preallocate = config.preallocate) 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)) @@ -520,9 +666,10 @@ class Log(val dir: File, */ private[log] def delete() { lock synchronized { + removeLogMetrics() logSegments.foreach(_.delete()) segments.clear() - Utils.rm(dir) + CoreUtils.rm(dir) } } @@ -545,7 +692,7 @@ class Log(val dir: File, val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset) deletable.foreach(deleteSegment(_)) activeSegment.truncateTo(targetOffset) - this.nextOffset.set(targetOffset) + updateLogEndOffset(targetOffset) this.recoveryPoint = math.min(targetOffset, this.recoveryPoint) } } @@ -564,8 +711,12 @@ class Log(val dir: File, newOffset, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, - time = time)) - this.nextOffset.set(newOffset) + rollJitterMs = config.randomSegmentJitter, + time = time, + fileAlreadyExists = false, + initFileSize = initFileSize, + preallocate = config.preallocate)) + updateLogEndOffset(newOffset) this.recoveryPoint = math.min(newOffset, this.recoveryPoint) } } @@ -642,14 +793,32 @@ class Log(val dir: File, * 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. * + * The sequence of operations is: + *
          + *
        1. Cleaner creates new segment with suffix .cleaned and invokes replaceSegments(). + * If broker crashes at this point, the clean-and-swap operation is aborted and + * the .cleaned file is deleted on recovery in loadSegments(). + *
        2. New segment is renamed .swap. If the broker crashes after this point before the whole + * operation is completed, the swap operation is resumed on recovery as described in the next step. + *
        3. Old segment files are renamed to .deleted and asynchronous delete is scheduled. + * If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments(). + * replaceSegments() is then invoked to complete the swap with newSegment recreated from + * the .swap file and oldSegments containing segments which were not renamed before the crash. + *
        4. Swap segment is renamed to replace the existing segment, completing this operation. + * If the broker crashes, any .deleted files which may be left behind are deleted + * on recovery in loadSegments(). + *
        + * * @param newSegment The new log segment to add to the log * @param oldSegments The old log segments to delete from the log + * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash */ - private[log] def replaceSegments(newSegment: LogSegment, oldSegments: Seq[LogSegment]) { + private[log] def replaceSegments(newSegment: LogSegment, oldSegments: Seq[LogSegment], isRecoveredSwapFile : Boolean = false) { 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) + if (!isRecoveredSwapFile) + newSegment.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix) addSegment(newSegment) // delete the old files @@ -664,7 +833,16 @@ class Log(val dir: File, newSegment.changeFileSuffixes(Log.SwapFileSuffix, "") } } - + + /** + * remove deleted log metrics + */ + private[log] def removeLogMetrics(): Unit = { + removeMetric("NumLogSegments", tags) + removeMetric("LogStartOffset", tags) + removeMetric("LogEndOffset", tags) + removeMetric("Size", tags) + } /** * 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 @@ -732,9 +910,25 @@ object Log { /** * Parse the topic and partition out of the directory name of a log */ - def parseTopicPartitionName(name: String): TopicAndPartition = { + def parseTopicPartitionName(dir: File): TopicAndPartition = { + val name: String = dir.getName + if (name == null || name.isEmpty || !name.contains('-')) { + throwException(dir) + } val index = name.lastIndexOf('-') - TopicAndPartition(name.substring(0,index), name.substring(index+1).toInt) + val topic: String = name.substring(0, index) + val partition: String = name.substring(index + 1) + if (topic.length < 1 || partition.length < 1) { + throwException(dir) + } + TopicAndPartition(topic, partition.toInt) + } + + def throwException(dir: File) { + throw new KafkaException("Found directory " + dir.getCanonicalPath + ", " + + "'" + dir.getName + "' is not in the form of topic-partition\n" + + "If a directory does not contain Kafka topic data it should not exist in Kafka's log " + + "directory") } } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 64046472a1f88..b36ea0dd7f954 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -17,15 +17,18 @@ package kafka.log -import scala.collection._ -import scala.math +import java.io.{DataOutputStream, File} import java.nio._ import java.util.Date -import java.io.File +import java.util.concurrent.{CountDownLatch, TimeUnit} + +import com.yammer.metrics.core.Gauge import kafka.common._ import kafka.message._ +import kafka.metrics.KafkaMetricsGroup import kafka.utils._ -import java.lang.IllegalStateException + +import scala.collection._ /** * The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy. @@ -63,19 +66,42 @@ import java.lang.IllegalStateException 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); + time: Time = SystemTime) extends Logging with KafkaMetricsGroup { + + /* for managing the state of partitions being cleaned. package-private to allow access in tests */ + private[log] 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, + "cleaner-io", + "bytes", time = time) /* the threads */ private val cleaners = (0 until config.numThreads).map(new CleanerThread(_)) + /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */ + newGauge("max-buffer-utilization-percent", + new Gauge[Int] { + def value: Int = cleaners.map(_.lastStats).map(100 * _.bufferUtilization).max.toInt + }) + /* a metric to track the recopy rate of each thread's last cleaning */ + newGauge("cleaner-recopy-percent", + new Gauge[Int] { + def value: Int = { + val stats = cleaners.map(_.lastStats) + val recopyRate = stats.map(_.bytesWritten).sum.toDouble / math.max(stats.map(_.bytesRead).sum, 1) + (100 * recopyRate).toInt + } + }) + /* a metric to track the maximum cleaning time for the last cleaning from each thread */ + newGauge("max-clean-time-secs", + new Gauge[Int] { + def value: Int = cleaners.map(_.lastStats).map(_.elapsedSecs).max.toInt + }) + /** * Start the background cleaning */ @@ -100,6 +126,13 @@ class LogCleaner(val config: CleanerConfig, cleanerManager.abortCleaning(topicAndPartition) } + /** + * Update checkpoint file, removing topics and partitions that no longer exist + */ + def updateCheckpoints(dataDir: File) { + cleanerManager.updateCheckpoints(dataDir, update=None) + } + /** * 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. @@ -116,8 +149,7 @@ class LogCleaner(val config: CleanerConfig, } /** - * TODO: - * For testing, a way to know when work has completed. This method blocks until the + * 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 = { @@ -131,6 +163,9 @@ class LogCleaner(val config: CleanerConfig, */ private class CleanerThread(threadId: Int) extends ShutdownableThread(name = "kafka-log-cleaner-thread-" + threadId, isInterruptible = false) { + + override val loggerName = classOf[LogCleaner].getName + if(config.dedupeBufferSize / config.numThreads > Int.MaxValue) warn("Cannot use more than 2G of cleaner buffer space per cleaner thread, ignoring excess buffer space...") @@ -144,6 +179,9 @@ class LogCleaner(val config: CleanerConfig, time = time, checkDone = checkDone) + @volatile var lastStats: CleanerStats = new CleanerStats() + private val backOffWaitLatch = new CountDownLatch(1) + private def checkDone(topicAndPartition: TopicAndPartition) { if (!isRunning.get()) throw new ThreadShutdownException @@ -157,6 +195,13 @@ class LogCleaner(val config: CleanerConfig, cleanOrSleep() } + + override def shutdown() = { + initiateShutdown() + backOffWaitLatch.countDown() + awaitShutdown() + } + /** * Clean a log if there is a dirty log available, otherwise sleep for a bit */ @@ -164,13 +209,13 @@ class LogCleaner(val config: CleanerConfig, cleanerManager.grabFilthiestLog() match { case None => // there are no cleanable logs, sleep a while - time.sleep(config.backOffMs) + backOffWaitLatch.await(config.backOffMs, TimeUnit.MILLISECONDS) 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) + recordStats(cleaner.id, cleanable.log.name, cleanable.firstDirtyOffset, endOffset, cleaner.stats) } catch { case pe: LogCleaningAbortedException => // task can be aborted, let it go. } finally { @@ -182,17 +227,20 @@ class LogCleaner(val config: CleanerConfig, /** * Log out statistics on a single run of the cleaner. */ - def logStats(id: Int, name: String, from: Long, to: Long, stats: CleanerStats) { + def recordStats(id: Int, name: String, from: Long, to: Long, stats: CleanerStats) { + this.lastStats = stats + cleaner.statsUnderlying.swap 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) + + "%n\tLog cleaner thread %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) + + 100 * stats.elapsedIndexSecs/stats.elapsedSecs) + + "\tBuffer utilization: %.1f%%%n".format(100 * stats.bufferUtilization) + "\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) + @@ -201,6 +249,9 @@ class LogCleaner(val config: CleanerConfig, "\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) + if (stats.invalidMessagesRead > 0) { + warn("\tFound %d invalid messages during compaction.".format(stats.invalidMessagesRead)) + } } } @@ -215,19 +266,22 @@ class LogCleaner(val config: CleanerConfig, * @param time The time instance */ private[log] class Cleaner(val id: Int, - offsetMap: OffsetMap, + val offsetMap: OffsetMap, ioBufferSize: Int, maxIoBufferSize: Int, dupBufferLoadFactor: Double, throttler: Throttler, time: Time, checkDone: (TopicAndPartition) => Unit) extends Logging { + + override val loggerName = classOf[LogCleaner].getName this.logIdent = "Cleaner " + id + ": " - /* stats on this cleaning */ - val stats = new CleanerStats(time) - + /* cleaning stats - one instance for the current (or next) cleaning cycle and one for the last completed cycle */ + val statsUnderlying = (new CleanerStats(time), new CleanerStats(time)) + def stats = statsUnderlying._1 + /* buffer used for read i/o */ private var readBuffer = ByteBuffer.allocate(ioBufferSize) @@ -264,8 +318,12 @@ private[log] class Cleaner(val id: Int, 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) + + // record buffer utilization + stats.bufferUtilization = offsetMap.utilization stats.allDone() + endOffset } @@ -275,7 +333,6 @@ private[log] class Cleaner(val id: Int, * @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, @@ -287,9 +344,9 @@ private[log] class Cleaner(val id: Int, logFile.delete() val indexFile = new File(segments.head.index.file.getPath + Log.CleanedFileSuffix) indexFile.delete() - val messages = new FileMessageSet(logFile) + val messages = new FileMessageSet(logFile, fileAlreadyExists = false, initFileSize = log.initFileSize(), preallocate = log.config.preallocate) 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) + val cleaned = new LogSegment(messages, index, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time) try { // clean segments into the new destination segment @@ -316,7 +373,7 @@ private[log] class Cleaner(val id: Int, } catch { case e: LogCleaningAbortedException => cleaned.delete() - throw e + throw e } } @@ -329,7 +386,6 @@ private[log] class Cleaner(val id: Int, * @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) { @@ -343,27 +399,30 @@ private[log] class Cleaner(val id: Int, throttler.maybeThrottle(messages.sizeInBytes) // check each message to see if it is to be retained var messagesRead = 0 - for (entry <- messages) { - messagesRead += 1 + for (entry <- messages.shallowIterator) { 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 (entry.message.compressionCodec == NoCompressionCodec) { + if (shouldRetainMessage(source, map, retainDeletes, entry)) { + ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset) + stats.recopyMessage(size) + } + messagesRead += 1 + } else { + val messages = ByteBufferMessageSet.deepIterator(entry.message) + val retainedMessages = messages.filter(messageAndOffset => { + messagesRead += 1 + shouldRetainMessage(source, map, retainDeletes, messageAndOffset) + }).toSeq + + if (retainedMessages.nonEmpty) + compressMessages(writeBuffer, entry.message.compressionCodec, retainedMessages) } } + + position += messages.validBytes // if any messages are to be retained, write them out - if(writeBuffer.position > 0) { + if (writeBuffer.position > 0) { writeBuffer.flip() val retained = new ByteBufferMessageSet(writeBuffer) dest.append(retained.head.offset, retained) @@ -371,12 +430,62 @@ private[log] class Cleaner(val id: Int, } // 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) + if (readBuffer.limit > 0 && messagesRead == 0) growBuffers() } restoreBuffers() } - + + private def compressMessages(buffer: ByteBuffer, compressionCodec: CompressionCodec, messages: Seq[MessageAndOffset]) { + val messagesIterable = messages.toIterable.map(_.message) + if (messages.isEmpty) { + MessageSet.Empty.sizeInBytes + } else if (compressionCodec == NoCompressionCodec) { + for(messageOffset <- messages) + ByteBufferMessageSet.writeMessage(buffer, messageOffset.message, messageOffset.offset) + MessageSet.messageSetSize(messagesIterable) + } else { + var offset = -1L + val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messagesIterable) / 2, 1024), 1 << 16)) + messageWriter.write(codec = compressionCodec) { outputStream => + val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream)) + try { + for (messageOffset <- messages) { + val message = messageOffset.message + offset = messageOffset.offset + output.writeLong(offset) + output.writeInt(message.size) + output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit) + } + } finally { + output.close() + } + } + ByteBufferMessageSet.writeMessage(buffer, messageWriter, offset) + stats.recopyMessage(messageWriter.size + MessageSet.LogOverhead) + } + } + + private def shouldRetainMessage(source: kafka.log.LogSegment, + map: kafka.log.OffsetMap, + retainDeletes: Boolean, + entry: kafka.message.MessageAndOffset): Boolean = { + val key = entry.message.key + if (key != null) { + 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 + !redundant && !obsoleteDelete + } else { + stats.invalidMessage() + false + } + } + /** * Double the I/O buffer capacity */ @@ -420,7 +529,8 @@ private[log] class Cleaner(val id: Int, segs = segs.tail while(!segs.isEmpty && logSize + segs.head.size < maxSize && - indexSize + segs.head.index.sizeInBytes < maxIndexSize) { + indexSize + segs.head.index.sizeInBytes < maxIndexSize && + segs.head.index.lastOffset - group.last.index.baseOffset <= Int.MaxValue) { group = segs.head :: group logSize += segs.head.size indexSize += segs.head.index.sizeInBytes @@ -449,11 +559,17 @@ private[log] class Cleaner(val id: Int, // 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) { + val maxDesiredMapSize = (map.slots * this.dupBufferLoadFactor).toInt + var full = false + for (segment <- dirty if !full) { checkDone(log.topicAndPartition) - if(segment.baseOffset <= minStopOffset || map.utilization < this.dupBufferLoadFactor) + val segmentSize = segment.nextOffset() - segment.baseOffset + + require(segmentSize <= maxDesiredMapSize, "%d messages in segment %s/%s but offset map can fit only %d. You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads".format(segmentSize, log.name, segment.log.file.getName, maxDesiredMapSize)) + if (map.size + segmentSize <= maxDesiredMapSize) offset = buildOffsetMapForSegment(log.topicAndPartition, segment, map) + else + full = true } info("Offset map for log %s complete.".format(log.name)) offset @@ -478,13 +594,14 @@ private[log] class Cleaner(val id: Int, 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) + if (message.hasKey) + map.put(message.key, entry.offset) offset = entry.offset - stats.indexMessage(size) + stats.indexMessagesRead(1) } + position += messages.validBytes + stats.indexBytesRead(messages.validBytes) + // if we didn't read even one complete message, our read buffer may be too small if(position == startPosition) growBuffers() @@ -498,28 +615,37 @@ private[log] class Cleaner(val id: Int, * 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 + var startTime, mapCompleteTime, endTime, bytesRead, bytesWritten, mapBytesRead, mapMessagesRead, messagesRead, + messagesWritten, invalidMessagesRead = 0L + var bufferUtilization = 0.0d clear() def readMessage(size: Int) { messagesRead += 1 bytesRead += size } + + def invalidMessage() { + invalidMessagesRead += 1 + } def recopyMessage(size: Int) { messagesWritten += 1 bytesWritten += size } - - def indexMessage(size: Int) { - mapMessagesRead += 1 + + def indexMessagesRead(size: Int) { + mapMessagesRead += size + } + + def indexBytesRead(size: Int) { mapBytesRead += size } - + def indexDone() { mapCompleteTime = time.milliseconds } - + def allDone() { endTime = time.milliseconds } @@ -537,7 +663,9 @@ private case class CleanerStats(time: Time = SystemTime) { mapBytesRead = 0L mapMessagesRead = 0L messagesRead = 0L + invalidMessagesRead = 0L messagesWritten = 0L + bufferUtilization = 0.0d } } @@ -550,4 +678,4 @@ private case class LogToClean(topicPartition: TopicAndPartition, log: Log, first 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 old mode 100644 new mode 100755 index 1612c8dbcaba8..f6795d3ffdb85 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -18,11 +18,13 @@ package kafka.log import java.io.File +import kafka.metrics.KafkaMetricsGroup +import com.yammer.metrics.core.Gauge import kafka.utils.{Logging, Pool} import kafka.server.OffsetCheckpoint import collection.mutable import java.util.concurrent.locks.ReentrantLock -import kafka.utils.Utils._ +import kafka.utils.CoreUtils._ import java.util.concurrent.TimeUnit import kafka.common.{LogCleaningAbortedException, TopicAndPartition} @@ -39,17 +41,28 @@ private[log] case object LogCleaningPaused extends LogCleaningState * 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 { +private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[TopicAndPartition, Log]) extends Logging with KafkaMetricsGroup { + + override val loggerName = classOf[LogCleaner].getName + + // package-private for testing + private[log] val offsetCheckpointFile = "cleaner-offset-checkpoint" + /* 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 + private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, offsetCheckpointFile)))).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() + + /* a gauge for tracking the cleanable ratio of the dirtiest log */ + @volatile private var dirtiestLogCleanableRatio = 0.0 + newGauge("max-dirty-percent", new Gauge[Int] { def value = (100 * dirtiestLogCleanableRatio).toInt }) /** * @return the position processed for all logs. @@ -65,15 +78,35 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To 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) { + val dirtyLogs = logs.filter { + case (topicAndPartition, log) => log.config.compact // skip any logs marked for delete rather than dedupe + }.filterNot { + case (topicAndPartition, log) => inProgress.contains(topicAndPartition) // skip any logs already in-progress + }.map { + case (topicAndPartition, log) => // create a LogToClean instance for each + // if the log segments are abnormally truncated and hence the checkpointed offset + // is no longer valid, reset to the log starting offset and log the error event + val logStartOffset = log.logSegments.head.baseOffset + val firstDirtyOffset = { + val offset = lastClean.getOrElse(topicAndPartition, logStartOffset) + if (offset < logStartOffset) { + error("Resetting first dirty offset to log start offset %d since the checkpointed offset %d is invalid." + .format(logStartOffset, offset)) + logStartOffset + } else { + offset + } + } + LogToClean(topicAndPartition, log, firstDirtyOffset) + }.filter(ltc => ltc.totalBytes > 0) // skip any empty logs + + this.dirtiestLogCleanableRatio = if (!dirtyLogs.isEmpty) dirtyLogs.max.cleanableRatio else 0 + // and must meet the minimum threshold for dirty byte ratio + val cleanableLogs = dirtyLogs.filter(ltc => ltc.cleanableRatio > ltc.log.config.minCleanableRatio) + if(cleanableLogs.isEmpty) { None } else { - val filthiest = dirtyLogs.max + val filthiest = cleanableLogs.max inProgress.put(filthiest.topicPartition, LogCleaningInProgress) Some(filthiest) } @@ -89,8 +122,8 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To inLock(lock) { abortAndPauseCleaning(topicAndPartition) resumeCleaning(topicAndPartition) - info("The cleaning for partition %s is aborted".format(topicAndPartition)) } + info("The cleaning for partition %s is aborted".format(topicAndPartition)) } /** @@ -113,13 +146,14 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To 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)) + throw new IllegalStateException("Compaction for partition %s cannot be aborted and paused since it is 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)) } + info("The cleaning for partition %s is aborted and paused".format(topicAndPartition)) } /** @@ -129,30 +163,32 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To inLock(lock) { inProgress.get(topicAndPartition) match { case None => - throw new IllegalStateException(("Partiiton %s can't be resumed since it's never paused").format(topicAndPartition)) + throw new IllegalStateException("Compaction for partition %s cannot be resumed since it is not 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)) + throw new IllegalStateException("Compaction for partition %s cannot be resumed since it is in %s state." + .format(topicAndPartition, s)) } } } - info("The cleaning for partition %s is resumed".format(topicAndPartition)) + info("Compaction 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 = { + private def isCleaningInState(topicAndPartition: TopicAndPartition, expectedState: LogCleaningState): Boolean = { inProgress.get(topicAndPartition) match { - case None => return false + case None => false case Some(state) => if (state == expectedState) - return true + true else - return false + false } } @@ -166,6 +202,14 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To } } + def updateCheckpoints(dataDir: File, update: Option[(TopicAndPartition,Long)]) { + inLock(lock) { + val checkpoint = checkpoints(dataDir) + val existing = checkpoint.read().filterKeys(logs.keys) ++ update + checkpoint.write(existing) + } + } + /** * Save out the endOffset and remove the given log from the in-progress set, if not aborted. */ @@ -173,15 +217,13 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To inLock(lock) { inProgress(topicAndPartition) match { case LogCleaningInProgress => - val checkpoint = checkpoints(dataDir) - val offsets = checkpoint.read() + ((topicAndPartition, endOffset)) - checkpoint.write(offsets) + updateCheckpoints(dataDir,Option(topicAndPartition, endOffset)) 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)) + throw new IllegalStateException("In-progress partition %s cannot 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 old mode 100644 new mode 100755 index 0b32aeeffcd9d..fc41132d2bf29 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -18,134 +18,179 @@ package kafka.log import java.util.Properties +import kafka.server.KafkaConfig +import org.apache.kafka.common.utils.Utils import scala.collection._ -import kafka.common._ +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} +import kafka.message.BrokerCompressionCodec +import kafka.message.Message -/** - * 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 Defaults { + val SegmentSize = kafka.server.Defaults.LogSegmentBytes + val SegmentMs = kafka.server.Defaults.LogRollHours * 60 * 60 * 1000L + val SegmentJitterMs = kafka.server.Defaults.LogRollJitterHours * 60 * 60 * 1000L + val FlushInterval = kafka.server.Defaults.LogFlushIntervalMessages + val FlushMs = kafka.server.Defaults.LogFlushSchedulerIntervalMs + val RetentionSize = kafka.server.Defaults.LogRetentionBytes + val RetentionMs = kafka.server.Defaults.LogRetentionHours * 60 * 60 * 1000L + val MaxMessageSize = kafka.server.Defaults.MessageMaxBytes + val MaxIndexSize = kafka.server.Defaults.LogIndexSizeMaxBytes + val IndexInterval = kafka.server.Defaults.LogIndexIntervalBytes + val FileDeleteDelayMs = kafka.server.Defaults.LogDeleteDelayMs + val DeleteRetentionMs = kafka.server.Defaults.LogCleanerDeleteRetentionMs + val MinCleanableDirtyRatio = kafka.server.Defaults.LogCleanerMinCleanRatio + val Compact = kafka.server.Defaults.LogCleanupPolicy + val UncleanLeaderElectionEnable = kafka.server.Defaults.UncleanLeaderElectionEnable + val MinInSyncReplicas = kafka.server.Defaults.MinInSyncReplicas + val CompressionType = kafka.server.Defaults.CompressionType + val PreAllocateEnable = kafka.server.Defaults.LogPreAllocateEnable +} + +case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props) { + + val segmentSize = getInt(LogConfig.SegmentBytesProp) + val segmentMs = getLong(LogConfig.SegmentMsProp) + val segmentJitterMs = getLong(LogConfig.SegmentJitterMsProp) + val maxIndexSize = getInt(LogConfig.SegmentIndexBytesProp) + val flushInterval = getLong(LogConfig.FlushMessagesProp) + val flushMs = getLong(LogConfig.FlushMsProp) + val retentionSize = getLong(LogConfig.RetentionBytesProp) + val retentionMs = getLong(LogConfig.RetentionMsProp) + val maxMessageSize = getInt(LogConfig.MaxMessageBytesProp) + val indexInterval = getInt(LogConfig.IndexIntervalBytesProp) + val fileDeleteDelayMs = getLong(LogConfig.FileDeleteDelayMsProp) + val deleteRetentionMs = getLong(LogConfig.DeleteRetentionMsProp) + val minCleanableRatio = getDouble(LogConfig.MinCleanableDirtyRatioProp) + val compact = getString(LogConfig.CleanupPolicyProp).toLowerCase != LogConfig.Delete + val uncleanLeaderElectionEnable = getBoolean(LogConfig.UncleanLeaderElectionEnableProp) + val minInSyncReplicas = getInt(LogConfig.MinInSyncReplicasProp) + val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase + val preallocate = getBoolean(LogConfig.PreAllocateEnableProp) + + def randomSegmentJitter: Long = + if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs) } object LogConfig { + + def main(args: Array[String]) { + System.out.println(configDef.toHtmlTable) + } + + val Delete = "delete" + val Compact = "compact" + val SegmentBytesProp = "segment.bytes" val SegmentMsProp = "segment.ms" + val SegmentJitterMsProp = "segment.jitter.ms" val SegmentIndexBytesProp = "segment.index.bytes" val FlushMessagesProp = "flush.messages" val FlushMsProp = "flush.ms" val RetentionBytesProp = "retention.bytes" - val RententionMsProp = "retention.ms" + val RetentionMsProp = "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") + val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" + val MinInSyncReplicasProp = "min.insync.replicas" + val CompressionTypeProp = "compression.type" + val PreAllocateEnableProp = "preallocate" + + val SegmentSizeDoc = "The hard maximum for the size of a segment file in the log" + val SegmentMsDoc = "The soft maximum on the amount of time before a new log segment is rolled" + val SegmentJitterMsDoc = "The maximum random jitter subtracted from segmentMs to avoid thundering herds of segment" + + " rolling" + val FlushIntervalDoc = "The number of messages that can be written to the log before a flush is forced" + val FlushMsDoc = "The amount of time the log can have dirty data before a flush is forced" + val RetentionSizeDoc = "The approximate total number of bytes this log can use" + val RetentionMsDoc = "The approximate maximum age of the last segment that is retained" + val MaxIndexSizeDoc = "The maximum size of an index file" + val MaxMessageSizeDoc = "The maximum size of a message" + val IndexIntervalDoc = "The approximate number of bytes between index entries" + val FileDeleteDelayMsDoc = "The time to wait before deleting a file from the filesystem" + val DeleteRetentionMsDoc = "The time to retain delete markers in the log. Only applicable for logs that are being" + + " compacted." + val MinCleanableRatioDoc = "The ratio of bytes that are available for cleaning to the bytes already cleaned" + val CompactDoc = "Should old segments in this log be deleted or deduplicated?" + val UncleanLeaderElectionEnableDoc = "Indicates whether unclean leader election is enabled" + val MinInSyncReplicasDoc = "If number of insync replicas drops below this number, we stop accepting writes with" + + " -1 (or all) required acks" + val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the " + + "standard compression codecs ('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to " + + "no compression; and 'producer' which means retain the original compression codec set by the producer." + val PreAllocateEnableDoc ="Should pre allocate file when create new segment?" + + private val configDef = { + import ConfigDef.Range._ + import ConfigDef.ValidString._ + import ConfigDef.Type._ + import ConfigDef.Importance._ + import java.util.Arrays.asList + + new ConfigDef() + .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(Message.MinHeaderSize), MEDIUM, SegmentSizeDoc) + .define(SegmentMsProp, LONG, Defaults.SegmentMs, atLeast(0), MEDIUM, SegmentMsDoc) + .define(SegmentJitterMsProp, LONG, Defaults.SegmentJitterMs, atLeast(0), MEDIUM, SegmentJitterMsDoc) + .define(SegmentIndexBytesProp, INT, Defaults.MaxIndexSize, atLeast(0), MEDIUM, MaxIndexSizeDoc) + .define(FlushMessagesProp, LONG, Defaults.FlushInterval, atLeast(0), MEDIUM, FlushIntervalDoc) + .define(FlushMsProp, LONG, Defaults.FlushMs, atLeast(0), MEDIUM, FlushMsDoc) + // can be negative. See kafka.log.LogManager.cleanupSegmentsToMaintainSize + .define(RetentionBytesProp, LONG, Defaults.RetentionSize, MEDIUM, RetentionSizeDoc) + // can be negative. See kafka.log.LogManager.cleanupExpiredSegments + .define(RetentionMsProp, LONG, Defaults.RetentionMs, MEDIUM, RetentionMsDoc) + .define(MaxMessageBytesProp, INT, Defaults.MaxMessageSize, atLeast(0), MEDIUM, MaxMessageSizeDoc) + .define(IndexIntervalBytesProp, INT, Defaults.IndexInterval, atLeast(0), MEDIUM, IndexIntervalDoc) + .define(DeleteRetentionMsProp, LONG, Defaults.DeleteRetentionMs, atLeast(0), MEDIUM, DeleteRetentionMsDoc) + .define(FileDeleteDelayMsProp, LONG, Defaults.FileDeleteDelayMs, atLeast(0), MEDIUM, FileDeleteDelayMsDoc) + .define(MinCleanableDirtyRatioProp, DOUBLE, Defaults.MinCleanableDirtyRatio, between(0, 1), MEDIUM, + MinCleanableRatioDoc) + .define(CleanupPolicyProp, STRING, Defaults.Compact, in(Compact, Delete), MEDIUM, + CompactDoc) + .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, + MEDIUM, UncleanLeaderElectionEnableDoc) + .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc) + .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc) + .define(PreAllocateEnableProp, BOOLEAN, Defaults.PreAllocateEnable, + MEDIUM, PreAllocateEnableDoc) + } + + def apply(): LogConfig = LogConfig(new Properties()) + + def configNames() = { + import JavaConversions._ + configDef.names().toList.sorted } - + + /** * Create a log config instance using the given properties and defaults */ - def fromProps(defaults: Properties, overrides: Properties): LogConfig = { - val props = new Properties(defaults) + def fromProps(defaults: java.util.Map[_ <: Object, _ <: Object], overrides: Properties): LogConfig = { + val props = new Properties() + props.putAll(defaults) props.putAll(overrides) - fromProps(props) + LogConfig(props) } - + /** * Check that property names are valid */ def validateNames(props: Properties) { import JavaConversions._ + val names = configDef.names() for(name <- props.keys) - require(LogConfig.ConfigNames.contains(name), "Unknown configuration \"%s\".".format(name)) + require(names.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. + * Check that the given properties contain only valid log config names and that all values can be parsed and are valid */ def validate(props: Properties) { validateNames(props) - LogConfig.fromProps(LogConfig().toProps, props) // check that we can parse the values + configDef.parse(props) } - + } - - \ 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 old mode 100644 new mode 100755 index 10062af1e02af..69386c17153e5 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -22,7 +22,8 @@ import java.util.concurrent.TimeUnit import kafka.utils._ import scala.collection._ import kafka.common.{TopicAndPartition, KafkaException} -import kafka.server.OffsetCheckpoint +import kafka.server.{RecoveringFromUncleanShutdown, BrokerState, OffsetCheckpoint} +import java.util.concurrent.{Executors, ExecutorService, ExecutionException, Future} /** * The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning. @@ -39,12 +40,13 @@ class LogManager(val logDirs: Array[File], val topicConfigs: Map[String, LogConfig], val defaultConfig: LogConfig, val cleanerConfig: CleanerConfig, + ioThreads: Int, val flushCheckMs: Long, val flushCheckpointMs: Long, val retentionCheckMs: Long, scheduler: Scheduler, + val brokerState: BrokerState, private val time: Time) extends Logging { - val RecoveryPointCheckpointFile = "recovery-point-offset-checkpoint" val LockFile = ".lock" val InitialTaskDelayMs = 30*1000 @@ -52,11 +54,12 @@ class LogManager(val logDirs: Array[File], private val logs = new Pool[TopicAndPartition, Log]() createAndValidateLogDirs(logDirs) - private var dirLocks = lockLogDirs(logDirs) + private val dirLocks = lockLogDirs(logDirs) private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, RecoveryPointCheckpointFile)))).toMap - loadLogs(logDirs) - - private val cleaner: LogCleaner = + loadLogs() + + // public, so we can access this from kafka.admin.DeleteTopicTest + val cleaner: LogCleaner = if(cleanerConfig.enableCleaner) new LogCleaner(cleanerConfig, logDirs, logs, time = time) else @@ -101,33 +104,79 @@ class LogManager(val logDirs: Array[File], /** * Recover and load all logs in the given data directories */ - private def loadLogs(dirs: Seq[File]) { - for(dir <- dirs) { - 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) { - info("Loading log '" + dir.getName + "'") - val topicPartition = Log.parseTopicPartitionName(dir.getName) - val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig) - val log = new Log(dir, - 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)) + private def loadLogs(): Unit = { + info("Loading logs.") + + val threadPools = mutable.ArrayBuffer.empty[ExecutorService] + val jobs = mutable.Map.empty[File, Seq[Future[_]]] + + for (dir <- this.logDirs) { + val pool = Executors.newFixedThreadPool(ioThreads) + threadPools.append(pool) + + val cleanShutdownFile = new File(dir, Log.CleanShutdownFile) + + if (cleanShutdownFile.exists) { + debug( + "Found clean shutdown file. " + + "Skipping recovery for all logs in data directory: " + + dir.getAbsolutePath) + } else { + // log recovery itself is being performed by `Log` class during initialization + brokerState.newState(RecoveringFromUncleanShutdown) + } + + var recoveryPoints = Map[TopicAndPartition, Long]() + try { + recoveryPoints = this.recoveryPointCheckpoints(dir).read + } catch { + case e: Exception => { + warn("Error occured while reading recovery-point-offset-checkpoint file of directory " + dir, e) + warn("Resetting the recovery checkpoint to 0") + } + } + + val jobsForDir = for { + dirContent <- Option(dir.listFiles).toList + logDir <- dirContent if logDir.isDirectory + } yield { + CoreUtils.runnable { + debug("Loading log '" + logDir.getName + "'") + + val topicPartition = Log.parseTopicPartitionName(logDir) + val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig) + val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) + + val current = new Log(logDir, config, logRecoveryPoint, scheduler, time) + val previous = this.logs.put(topicPartition, current) + + if (previous != null) { + throw new IllegalArgumentException( + "Duplicate log directories found: %s, %s!".format( + current.dir.getAbsolutePath, previous.dir.getAbsolutePath)) } } - cleanShutDownFile.delete() } + + jobs(cleanShutdownFile) = jobsForDir.map(pool.submit).toSeq + } + + + try { + for ((cleanShutdownFile, dirJobs) <- jobs) { + dirJobs.foreach(_.get) + cleanShutdownFile.delete() + } + } catch { + case e: ExecutionException => { + error("There was an error in one of the threads during logs loading: " + e.getCause) + throw e.getCause + } + } finally { + threadPools.foreach(_.shutdown()) } + + info("Logs loading complete.") } /** @@ -157,31 +206,69 @@ class LogManager(val logDirs: Array[File], if(cleanerConfig.enableCleaner) cleaner.startup() } - + /** * Close all the logs */ def shutdown() { - debug("Shutting down.") + info("Shutting down.") + + val threadPools = mutable.ArrayBuffer.empty[ExecutorService] + val jobs = mutable.Map.empty[File, Seq[Future[_]]] + + // stop the cleaner first + if (cleaner != null) { + CoreUtils.swallow(cleaner.shutdown()) + } + + // close logs in each dir + for (dir <- this.logDirs) { + debug("Flushing and closing logs at " + dir) + + val pool = Executors.newFixedThreadPool(ioThreads) + threadPools.append(pool) + + val logsInDir = logsByDir.getOrElse(dir.toString, Map()).values + + val jobsForDir = logsInDir map { log => + CoreUtils.runnable { + // flush the log to ensure latest possible recovery point + log.flush() + log.close() + } + } + + jobs(dir) = jobsForDir.map(pool.submit).toSeq + } + + 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())) + for ((dir, dirJobs) <- jobs) { + dirJobs.foreach(_.get) + + // update the last flush point + debug("Updating recovery points at " + dir) + checkpointLogsInDir(dir) + + // mark that the shutdown was clean by creating marker file + debug("Writing clean shutdown marker at " + dir) + CoreUtils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile()) + } + } catch { + case e: ExecutionException => { + error("There was an error in one of the threads during LogManager shutdown: " + e.getCause) + throw e.getCause + } } finally { + threadPools.foreach(_.shutdown()) // regardless of whether the close succeeded, we need to unlock the data directories dirLocks.foreach(_.destroy()) } - debug("Shutdown complete.") + + info("Shutdown complete.") } + /** * Truncate the partition logs to the specified offsets and checkpoint the recovery point to this offset * @@ -227,14 +314,19 @@ class LogManager(val logDirs: Array[File], * 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)) + this.logDirs.foreach(checkpointLogsInDir) + } + + /** + * Make a checkpoint for all logs in provided directory. + */ + private def checkpointLogsInDir(dir: File): Unit = { + val recoveryPoints = this.logsByDir.get(dir.toString) + if (recoveryPoints.isDefined) { + this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint)) } } - + /** * Get the log if it exists, otherwise return None */ @@ -272,7 +364,7 @@ class LogManager(val logDirs: Array[File], .format(topicAndPartition.topic, topicAndPartition.partition, dataDir.getAbsolutePath, - {import JavaConversions._; config.toProps.mkString(", ")})) + {import JavaConversions._; config.originals.mkString(", ")})) log } } @@ -287,8 +379,10 @@ class LogManager(val logDirs: Array[File], } 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) + if (cleaner != null) { cleaner.abortCleaning(topicAndPartition) + cleaner.updateCheckpoints(removedLog.dir.getParentFile) + } removedLog.delete() info("Deleted log for partition [%s,%d] in %s." .format(topicAndPartition.topic, @@ -321,6 +415,8 @@ class LogManager(val logDirs: Array[File], * Runs through the log removing segments older than a certain age */ private def cleanupExpiredSegments(log: Log): Int = { + if (log.config.retentionMs < 0) + return 0 val startMs = time.milliseconds log.deleteOldSegments(startMs - _.lastModified > log.config.retentionMs) } @@ -351,7 +447,7 @@ class LogManager(val logDirs: Array[File], debug("Beginning log cleanup...") var total = 0 val startMs = time.milliseconds - for(log <- allLogs; if !log.config.dedupe) { + for(log <- allLogs; if !log.config.compact) { debug("Garbage collecting '" + log.name + "'") total += cleanupExpiredSegments(log) + cleanupSegmentsToMaintainSize(log) } @@ -363,12 +459,21 @@ class LogManager(val logDirs: Array[File], * Get all the partition logs */ def allLogs(): Iterable[Log] = logs.values - + /** * Get a map of TopicAndPartition => Log */ def logsByTopicPartition = logs.toMap + /** + * Map of log dir to logs by topic and partitions in that dir + */ + private def logsByDir = { + this.logsByTopicPartition.groupBy { + case (_, log) => log.dir.getParent + } + } + /** * Flush any log which has exceeded its flush interval and has unwritten messages. */ diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala old mode 100644 new mode 100755 index 0d6926ea105a9..1377e8f322a3f --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -14,15 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.log +package kafka.log -import scala.math._ -import java.io.File import kafka.message._ import kafka.common._ import kafka.utils._ +import kafka.server.{LogOffsetMetadata, FetchDataInfo} -/** +import scala.math._ +import java.io.File + + + /** * A segment of the log. Each segment has two components: a log and an index. The log is a FileMessageSet containing * the actual messages. The index is an OffsetIndex that maps from logical offsets to physical file positions. Each * segment has a base offset which is an offset <= the least offset of any message in this segment and > any offset in @@ -41,18 +44,20 @@ class LogSegment(val log: FileMessageSet, val index: OffsetIndex, val baseOffset: Long, val indexIntervalBytes: Int, + val rollJitterMs: Long, time: Time) extends Logging { var created = time.milliseconds - + /* the number of bytes since we last added an entry in the offset index */ private var bytesSinceLastIndexEntry = 0 - def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, time: Time) = - this(new FileMessageSet(file = Log.logFilename(dir, startOffset)), + def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) = + this(new FileMessageSet(file = Log.logFilename(dir, startOffset), fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate), new OffsetIndex(file = Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), startOffset, indexIntervalBytes, + rollJitterMs, time) /* Return the size in bytes of this log segment */ @@ -86,7 +91,7 @@ class LogSegment(val log: FileMessageSet, * 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. + * in the file higher than the greatest-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 @@ -99,7 +104,7 @@ class LogSegment(val log: FileMessageSet, val mapping = index.lookup(offset) log.searchFor(offset, max(mapping.position, startingFilePosition)) } - + /** * 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. @@ -108,22 +113,27 @@ class LogSegment(val log: FileMessageSet, * @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. + * @return The fetched data and the offset metadata of the first message whose offset is >= startOffset, + * or null if the startOffset is larger than the largest offset in this log */ @threadsafe - def read(startOffset: Long, maxOffset: Option[Long], maxSize: Int): MessageSet = { + def read(startOffset: Long, maxOffset: Option[Long], maxSize: Int): FetchDataInfo = { if(maxSize < 0) throw new IllegalArgumentException("Invalid max size for log read (%d)".format(maxSize)) - if(maxSize == 0) - return MessageSet.Empty - + 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 null if(startPosition == null) return null - + + val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position) + + // if the size is zero, still return a log segment but with zero size + if(maxSize == 0) + return FetchDataInfo(offsetMetadata, MessageSet.Empty) + // calculate the length of the message set to read based on whether or not they gave us a maxOffset val length = maxOffset match { @@ -143,7 +153,7 @@ class LogSegment(val log: FileMessageSet, min(endPosition - startPosition.position, maxSize) } } - log.read(startPosition.position, length) + FetchDataInfo(offsetMetadata, log.read(startPosition.position, length)) } /** @@ -172,7 +182,7 @@ class LogSegment(val log: FileMessageSet, case NoCompressionCodec => entry.offset case _ => - ByteBufferMessageSet.decompress(entry.message).head.offset + ByteBufferMessageSet.deepIterator(entry.message).next().offset } index.append(startOffset, validBytes) lastIndexEntry = validBytes @@ -222,7 +232,7 @@ class LogSegment(val log: FileMessageSet, if(ms == null) { baseOffset } else { - ms.lastOption match { + ms.messageSet.lastOption match { case None => baseOffset case Some(last) => last.nextOffset } @@ -244,10 +254,10 @@ class LogSegment(val log: FileMessageSet, * 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))) + val logRenamed = log.renameTo(new File(CoreUtils.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))) + val indexRenamed = index.renameTo(new File(CoreUtils.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)) } @@ -256,8 +266,8 @@ class LogSegment(val log: FileMessageSet, * Close this log segment */ def close() { - Utils.swallow(index.close) - Utils.swallow(log.close) + CoreUtils.swallow(index.close) + CoreUtils.swallow(log.close) } /** diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala old mode 100644 new mode 100755 index 8a62dfad9bc48..332d5e25bd648 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -24,7 +24,7 @@ import java.nio.channels._ import java.util.concurrent.locks._ import java.util.concurrent.atomic._ import kafka.utils._ -import kafka.utils.Utils.inLock +import kafka.utils.CoreUtils.inLock import kafka.common.InvalidOffsetException /** @@ -81,7 +81,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi idx.position(roundToExactMultiple(idx.limit, 8)) idx } finally { - Utils.swallow(raf.close()) + CoreUtils.swallow(raf.close()) } } @@ -274,7 +274,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ def resize(newSize: Int) { inLock(lock) { - val raf = new RandomAccessFile(file, "rws") + val raf = new RandomAccessFile(file, "rw") val roundedNewSize = roundToExactMultiple(newSize, 8) val position = this.mmap.position @@ -287,7 +287,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi this.maxEntries = this.mmap.limit / 8 this.mmap.position(position) } finally { - Utils.swallow(raf.close()) + CoreUtils.swallow(raf.close()) } } } @@ -318,6 +318,8 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ def delete(): Boolean = { info("Deleting index " + this.file.getAbsolutePath) + if(Os.isWindows) + CoreUtils.swallow(forceUnmap(this.mmap)) this.file.delete() } @@ -346,7 +348,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi /** * Do a basic sanity check on this index to detect obvious problems - * @throw IllegalArgumentException if any problems are found + * @throws IllegalArgumentException if any problems are found */ def sanityCheck() { require(entries == 0 || lastOffset > baseOffset, @@ -373,10 +375,10 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi if(Os.isWindows) lock.lock() try { - return fun + 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 old mode 100644 new mode 100755 index 42cdfbb6100b5..303aad5ba76bf --- a/core/src/main/scala/kafka/log/OffsetMap.scala +++ b/core/src/main/scala/kafka/log/OffsetMap.scala @@ -21,6 +21,7 @@ import java.util.Arrays import java.security.MessageDigest import java.nio.ByteBuffer import kafka.utils._ +import org.apache.kafka.common.utils.Utils trait OffsetMap { def slots: Int @@ -68,7 +69,7 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend /** * The maximum number of entries this map can contain */ - val slots: Int = (memory / bytesPerEntry).toInt + val slots: Int = memory / bytesPerEntry /** * Associate this offset to the given key. @@ -158,7 +159,7 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend * @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 probe = CoreUtils.readInt(hash, math.min(attempt, hashSize - 4)) + math.max(0, attempt - hashSize + 4) val slot = Utils.abs(probe) % slots this.probes += 1 slot * bytesPerEntry @@ -176,4 +177,4 @@ class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extend digest.digest(buffer, 0, hashSize) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/log/OffsetPosition.scala b/core/src/main/scala/kafka/log/OffsetPosition.scala index 6cefde4d90e28..24b6dcf0bb789 100644 --- a/core/src/main/scala/kafka/log/OffsetPosition.scala +++ b/core/src/main/scala/kafka/log/OffsetPosition.scala @@ -22,4 +22,4 @@ package kafka.log * in some log file of the beginning of the message set entry with the * given offset. */ -case class OffsetPosition(val offset: Long, val position: Int) \ No newline at end of file +case class OffsetPosition(offset: Long, position: Int) diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 73401c5ff34d0..5a32de83b6b0b 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.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 @@ -17,16 +17,16 @@ package kafka.message -import scala.reflect.BeanProperty -import kafka.utils.Logging +import kafka.utils.{IteratorTemplate, Logging} +import kafka.common.KafkaException + import java.nio.ByteBuffer import java.nio.channels._ -import java.io.{InputStream, ByteArrayOutputStream, DataOutputStream} +import java.io._ import java.util.concurrent.atomic.AtomicLong -import kafka.utils.IteratorTemplate object ByteBufferMessageSet { - + private def create(offsetCounter: AtomicLong, compressionCodec: CompressionCodec, messages: Message*): ByteBuffer = { if(messages.size == 0) { MessageSet.Empty.buffer @@ -37,52 +37,78 @@ object ByteBufferMessageSet { buffer.rewind() buffer } else { - val byteArrayStream = new ByteArrayOutputStream(MessageSet.messageSetSize(messages)) - val output = new DataOutputStream(CompressionFactory(compressionCodec, byteArrayStream)) var offset = -1L - try { - for(message <- messages) { - offset = offsetCounter.getAndIncrement - output.writeLong(offset) - output.writeInt(message.size) - output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit) + val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16)) + messageWriter.write(codec = compressionCodec) { outputStream => + val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream)) + try { + for (message <- messages) { + offset = offsetCounter.getAndIncrement + output.writeLong(offset) + output.writeInt(message.size) + output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit) + } + } finally { + output.close() } - } finally { - output.close() } - val bytes = byteArrayStream.toByteArray - val message = new Message(bytes, compressionCodec) - val buffer = ByteBuffer.allocate(message.size + MessageSet.LogOverhead) - writeMessage(buffer, message, offset) + val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead) + writeMessage(buffer, messageWriter, offset) buffer.rewind() buffer } } - - def decompress(message: Message): ByteBufferMessageSet = { - val outputStream: ByteArrayOutputStream = new ByteArrayOutputStream - val inputStream: InputStream = new ByteBufferBackedInputStream(message.payload) - val intermediateBuffer = new Array[Byte](1024) - val compressed = CompressionFactory(message.compressionCodec, inputStream) - try { - Stream.continually(compressed.read(intermediateBuffer)).takeWhile(_ > 0).foreach { dataRead => - outputStream.write(intermediateBuffer, 0, dataRead) + + /** Deep iterator that decompresses the message sets in-place. */ + def deepIterator(wrapperMessage: Message): Iterator[MessageAndOffset] = { + new IteratorTemplate[MessageAndOffset] { + + val inputStream: InputStream = new ByteBufferBackedInputStream(wrapperMessage.payload) + val compressed: DataInputStream = new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, inputStream)) + + override def makeNext(): MessageAndOffset = { + try { + // read the offset + val offset = compressed.readLong() + // read record size + val size = compressed.readInt() + + if (size < Message.MinHeaderSize) + throw new InvalidMessageException("Message found with corrupt size (" + size + ") in deep iterator") + + // read the record into an intermediate record buffer + // and hence has to do extra copy + val bufferArray = new Array[Byte](size) + compressed.readFully(bufferArray, 0, size) + val buffer = ByteBuffer.wrap(bufferArray) + + val newMessage = new Message(buffer) + + // the decompressed message should not be a wrapper message since we do not allow nested compression + new MessageAndOffset(newMessage, offset) + } catch { + case eofe: EOFException => + compressed.close() + allDone() + case ioe: IOException => + throw new KafkaException(ioe) + } } - } finally { - compressed.close() } - val outputBuffer = ByteBuffer.allocate(outputStream.size) - outputBuffer.put(outputStream.toByteArray) - outputBuffer.rewind - new ByteBufferMessageSet(outputBuffer) } - + private[kafka] def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) { buffer.putLong(offset) buffer.putInt(message.size) buffer.put(message.buffer) message.buffer.rewind() } + + private[kafka] def writeMessage(buffer: ByteBuffer, messageWriter: MessageWriter, offset: Long) { + buffer.putLong(offset) + buffer.putInt(messageWriter.size) + messageWriter.writeTo(buffer) + } } /** @@ -93,15 +119,15 @@ object ByteBufferMessageSet { * Option 1: From a ByteBuffer which already contains the serialized message set. Consumers will use this method. * * Option 2: Give it a list of messages along with instructions relating to serialization format. Producers will use this method. - * + * */ -class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends MessageSet with Logging { +class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging { private var shallowValidByteCount = -1 def this(compressionCodec: CompressionCodec, messages: Message*) { this(ByteBufferMessageSet.create(new AtomicLong(0), compressionCodec, messages:_*)) } - + def this(compressionCodec: CompressionCodec, offsetCounter: AtomicLong, messages: Message*) { this(ByteBufferMessageSet.create(offsetCounter, compressionCodec, messages:_*)) } @@ -110,6 +136,8 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message this(NoCompressionCodec, new AtomicLong(0), messages: _*) } + def getBuffer = buffer + private def shallowValidBytes: Int = { if(shallowValidByteCount < 0) { var bytes = 0 @@ -122,7 +150,7 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message } shallowValidByteCount } - + /** Write the messages in this set to the given channel */ def writeTo(channel: GatheringByteChannel, offset: Long, size: Int): Int = { // Ignore offset and size from input. We just want to write the whole buffer to the channel. @@ -146,7 +174,7 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message var topIter = buffer.slice() var innerIter: Iterator[MessageAndOffset] = null - def innerDone():Boolean = (innerIter == null || !innerIter.hasNext) + def innerDone(): Boolean = (innerIter == null || !innerIter.hasNext) def makeNextOuter: MessageAndOffset = { // if there isn't at least an offset and size, we are done @@ -155,12 +183,12 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message val offset = topIter.getLong() val size = topIter.getInt() if(size < Message.MinHeaderSize) - throw new InvalidMessageException("Message found with corrupt size (" + size + ")") - + throw new InvalidMessageException("Message found with corrupt size (" + size + ") in shallow iterator") + // we have an incomplete message if(topIter.remaining < size) return allDone() - + // read the current message and check correctness val message = topIter.slice() message.limit(size) @@ -175,7 +203,7 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message innerIter = null new MessageAndOffset(newMessage, offset) case _ => - innerIter = ByteBufferMessageSet.decompress(newMessage).internalIterator() + innerIter = ByteBufferMessageSet.deepIterator(newMessage) if(!innerIter.hasNext) innerIter = null makeNext() @@ -190,42 +218,62 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message if(innerDone()) makeNextOuter else - innerIter.next + innerIter.next() } } - + } } - + /** - * Update the offsets for this message set. This method attempts to do an in-place conversion - * if there is no compression, but otherwise recopies the messages + * Update the offsets for this message set and do further validation on messages. This method attempts to do an + * in-place conversion if there is no compression, but otherwise recopies the messages */ - private[kafka] def assignOffsets(offsetCounter: AtomicLong, codec: CompressionCodec): ByteBufferMessageSet = { - if(codec == NoCompressionCodec) { - // do an in-place conversion - var position = 0 + private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: AtomicLong, + sourceCodec: CompressionCodec, + targetCodec: CompressionCodec, + compactedTopic: Boolean = false): ByteBufferMessageSet = { + if(sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { + // do in-place validation and offset assignment + var messagePosition = 0 buffer.mark() - while(position < sizeInBytes - MessageSet.LogOverhead) { - buffer.position(position) + while(messagePosition < sizeInBytes - MessageSet.LogOverhead) { + buffer.position(messagePosition) buffer.putLong(offsetCounter.getAndIncrement()) - position += MessageSet.LogOverhead + buffer.getInt() + val messageSize = buffer.getInt() + val positionAfterKeySize = buffer.position + Message.KeySizeOffset + Message.KeySizeLength + if (compactedTopic && positionAfterKeySize < sizeInBytes) { + buffer.position(buffer.position() + Message.KeySizeOffset) + val keySize = buffer.getInt() + if (keySize <= 0) { + buffer.reset() + throw new InvalidMessageException("Compacted topic cannot accept message without key.") + } + } + messagePosition += MessageSet.LogOverhead + messageSize } buffer.reset() this } else { - // messages are compressed, crack open the messageset and recompress with correct offset - val messages = this.internalIterator(isShallow = false).map(_.message) - new ByteBufferMessageSet(compressionCodec = codec, offsetCounter = offsetCounter, messages = messages.toBuffer:_*) + // We need to deep-iterate over the message-set if any of these are true: + // (i) messages are compressed + // (ii) the topic is configured with a target compression codec so we need to recompress regardless of original codec + val messages = this.internalIterator(isShallow = false).map(messageAndOffset => { + if (compactedTopic && !messageAndOffset.message.hasKey) + throw new InvalidMessageException("Compacted topic cannot accept message without key.") + + messageAndOffset.message + }) + new ByteBufferMessageSet(compressionCodec = targetCodec, offsetCounter = offsetCounter, messages = messages.toBuffer:_*) } } - + /** * The total number of bytes in this message set, including any partial trailing messages */ def sizeInBytes: Int = buffer.limit - + /** * The total number of bytes in this message set not including any partial, trailing messages */ @@ -236,7 +284,7 @@ class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends Message */ override def equals(other: Any): Boolean = { other match { - case that: ByteBufferMessageSet => + case that: ByteBufferMessageSet => buffer.equals(that.buffer) case _ => false } diff --git a/core/src/main/scala/kafka/message/CompressionCodec.scala b/core/src/main/scala/kafka/message/CompressionCodec.scala index 8762a790af2ab..4d7ce1774f0e5 100644 --- a/core/src/main/scala/kafka/message/CompressionCodec.scala +++ b/core/src/main/scala/kafka/message/CompressionCodec.scala @@ -23,6 +23,7 @@ object CompressionCodec { case NoCompressionCodec.codec => NoCompressionCodec case GZIPCompressionCodec.codec => GZIPCompressionCodec case SnappyCompressionCodec.codec => SnappyCompressionCodec + case LZ4CompressionCodec.codec => LZ4CompressionCodec case _ => throw new kafka.common.UnknownCodecException("%d is an unknown compression codec".format(codec)) } } @@ -31,29 +32,66 @@ object CompressionCodec { case NoCompressionCodec.name => NoCompressionCodec case GZIPCompressionCodec.name => GZIPCompressionCodec case SnappyCompressionCodec.name => SnappyCompressionCodec + case LZ4CompressionCodec.name => LZ4CompressionCodec case _ => throw new kafka.common.UnknownCodecException("%s is an unknown compression codec".format(name)) } } } +object BrokerCompressionCodec { + + val brokerCompressionCodecs = List(UncompressedCodec, SnappyCompressionCodec, LZ4CompressionCodec, GZIPCompressionCodec, ProducerCompressionCodec) + val brokerCompressionOptions = brokerCompressionCodecs.map(codec => codec.name) + + def isValid(compressionType: String): Boolean = brokerCompressionOptions.contains(compressionType.toLowerCase()) + + def getCompressionCodec(compressionType: String): CompressionCodec = { + compressionType.toLowerCase match { + case UncompressedCodec.name => NoCompressionCodec + case _ => CompressionCodec.getCompressionCodec(compressionType) + } + } + + def getTargetCompressionCodec(compressionType: String, producerCompression: CompressionCodec): CompressionCodec = { + if (ProducerCompressionCodec.name.equals(compressionType)) + producerCompression + else + getCompressionCodec(compressionType) + } +} + sealed trait CompressionCodec { def codec: Int; def name: String } +sealed trait BrokerCompressionCodec { def name: String } -case object DefaultCompressionCodec extends CompressionCodec { +case object DefaultCompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = GZIPCompressionCodec.codec val name = GZIPCompressionCodec.name } -case object GZIPCompressionCodec extends CompressionCodec { +case object GZIPCompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = 1 val name = "gzip" } -case object SnappyCompressionCodec extends CompressionCodec { +case object SnappyCompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = 2 val name = "snappy" } -case object NoCompressionCodec extends CompressionCodec { +case object LZ4CompressionCodec extends CompressionCodec with BrokerCompressionCodec { + val codec = 3 + val name = "lz4" +} + +case object NoCompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = 0 val name = "none" } + +case object UncompressedCodec extends BrokerCompressionCodec { + val name = "uncompressed" +} + +case object ProducerCompressionCodec extends BrokerCompressionCodec { + val name = "producer" +} diff --git a/core/src/main/scala/kafka/message/CompressionFactory.scala b/core/src/main/scala/kafka/message/CompressionFactory.scala index ca833ee317975..b047f68fb7f56 100644 --- a/core/src/main/scala/kafka/message/CompressionFactory.scala +++ b/core/src/main/scala/kafka/message/CompressionFactory.scala @@ -22,6 +22,8 @@ import java.util.zip.GZIPOutputStream import java.util.zip.GZIPInputStream import java.io.InputStream +import org.apache.kafka.common.record.{KafkaLZ4BlockInputStream, KafkaLZ4BlockOutputStream} + object CompressionFactory { def apply(compressionCodec: CompressionCodec, stream: OutputStream): OutputStream = { @@ -31,6 +33,8 @@ object CompressionFactory { case SnappyCompressionCodec => import org.xerial.snappy.SnappyOutputStream new SnappyOutputStream(stream) + case LZ4CompressionCodec => + new KafkaLZ4BlockOutputStream(stream) case _ => throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) } @@ -43,6 +47,8 @@ object CompressionFactory { case SnappyCompressionCodec => import org.xerial.snappy.SnappyInputStream new SnappyInputStream(stream) + case LZ4CompressionCodec => + new KafkaLZ4BlockInputStream(stream) case _ => throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) } diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala old mode 100644 new mode 100755 index 52c082f5213ba..999b11500f9ff --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -20,6 +20,7 @@ package kafka.message import java.nio._ import scala.math._ import kafka.utils._ +import org.apache.kafka.common.utils.Utils /** * Constants related to messages @@ -54,10 +55,10 @@ object Message { val CurrentMagicValue: Byte = 0 /** - * Specifies the mask for the compression code. 2 bits to hold the compression codec. + * Specifies the mask for the compression code. 3 bits to hold the compression codec. * 0 is reserved to indicate no compression */ - val CompressionCodeMask: Int = 0x03 + val CompressionCodeMask: Int = 0x07 /** * Compression code for uncompressed messages @@ -70,7 +71,7 @@ object Message { * A message. The format of an N byte message is the following: * * 1. 4 byte CRC32 of the message - * 2. 1 byte "magic" identifier to allow format changes, value is 2 currently + * 2. 1 byte "magic" identifier to allow format changes, value is 0 currently * 3. 1 byte "attributes" identifier to allow annotations on the message independent of the version (e.g. compression enabled, type of codec used) * 4. 4 byte key length, containing length K * 5. K byte key @@ -86,7 +87,7 @@ class Message(val buffer: ByteBuffer) { /** * A constructor to create a Message * @param bytes The payload of the message - * @param compressionCodec The compression codec used on the contents of the message (if any) + * @param codec The compression codec used on the contents of the message (if any) * @param key The key of the message (null, if none) * @param payloadOffset The offset into the payload array used to extract payload * @param payloadSize The size of the payload to use @@ -146,7 +147,7 @@ class Message(val buffer: ByteBuffer) { * Compute the checksum of the message from the message contents */ def computeChecksum(): Long = - Utils.crc32(buffer.array, buffer.arrayOffset + MagicOffset, buffer.limit - MagicOffset) + CoreUtils.crc32(buffer.array, buffer.arrayOffset + MagicOffset, buffer.limit - MagicOffset) /** * Retrieve the previously computed CRC for this message diff --git a/core/src/main/scala/kafka/message/MessageAndMetadata.scala b/core/src/main/scala/kafka/message/MessageAndMetadata.scala old mode 100644 new mode 100755 index d693abc620aef..26b75c8432d7b --- a/core/src/main/scala/kafka/message/MessageAndMetadata.scala +++ b/core/src/main/scala/kafka/message/MessageAndMetadata.scala @@ -18,7 +18,7 @@ package kafka.message import kafka.serializer.Decoder -import kafka.utils.Utils +import org.apache.kafka.common.utils.Utils case class MessageAndMetadata[K, V](topic: String, partition: Int, private val rawMessage: Message, offset: Long, diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala index a1b5c63b4d9de..28b56e68cfdbb 100644 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ b/core/src/main/scala/kafka/message/MessageSet.scala @@ -43,7 +43,7 @@ object MessageSet { var size = 0 val iter = messages.iterator while(iter.hasNext) { - val message = iter.next.asInstanceOf[Message] + val message = iter.next size += entrySize(message) } size @@ -80,17 +80,7 @@ abstract class MessageSet extends Iterable[MessageAndOffset] { * Gives the total size of this message set in bytes */ def sizeInBytes: Int - - /** - * Validate the checksum of all the messages in the set. Throws an InvalidMessageException if the checksum doesn't - * match the payload for any message. - */ - def validate(): Unit = { - for(messageAndOffset <- this) - if(!messageAndOffset.message.isValid) - throw new InvalidMessageException - } - + /** * Print this message set's contents. If the message set has more than 100 messages, just * print the first 100. diff --git a/core/src/main/scala/kafka/message/MessageWriter.scala b/core/src/main/scala/kafka/message/MessageWriter.scala new file mode 100755 index 0000000000000..0c6040e9f5f27 --- /dev/null +++ b/core/src/main/scala/kafka/message/MessageWriter.scala @@ -0,0 +1,206 @@ +/** + * 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.message + +import java.io.{InputStream, OutputStream} +import java.nio.ByteBuffer + +import org.apache.kafka.common.utils.Crc32 + +class MessageWriter(segmentSize: Int) extends BufferingOutputStream(segmentSize) { + + import Message._ + + def write(key: Array[Byte] = null, codec: CompressionCodec)(writePayload: OutputStream => Unit): Unit = { + withCrc32Prefix { + write(CurrentMagicValue) + var attributes: Byte = 0 + if (codec.codec > 0) + attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte + write(attributes) + // write the key + if (key == null) { + writeInt(-1) + } else { + writeInt(key.length) + write(key, 0, key.length) + } + // write the payload with length prefix + withLengthPrefix { + writePayload(this) + } + } + } + + private def writeInt(value: Int): Unit = { + write(value >>> 24) + write(value >>> 16) + write(value >>> 8) + write(value) + } + + private def writeInt(out: ReservedOutput, value: Int): Unit = { + out.write(value >>> 24) + out.write(value >>> 16) + out.write(value >>> 8) + out.write(value) + } + + private def withCrc32Prefix(writeData: => Unit): Unit = { + // get a writer for CRC value + val crcWriter = reserve(CrcLength) + // save current position + var seg = currentSegment + val offset = currentSegment.written + // write data + writeData + // compute CRC32 + val crc = new Crc32() + if (offset < seg.written) crc.update(seg.bytes, offset, seg.written - offset) + seg = seg.next + while (seg != null) { + if (seg.written > 0) crc.update(seg.bytes, 0, seg.written) + seg = seg.next + } + // write CRC32 + writeInt(crcWriter, crc.getValue().toInt) + } + + private def withLengthPrefix(writeData: => Unit): Unit = { + // get a writer for length value + val lengthWriter = reserve(ValueSizeLength) + // save current size + val oldSize = size + // write data + writeData + // write length value + writeInt(lengthWriter, size - oldSize) + } + +} + +/* + * OutputStream that buffers incoming data in segmented byte arrays + * This does not copy data when expanding its capacity + * It has a ability to + * - write data directly to ByteBuffer + * - copy data from an input stream to interval segmented arrays directly + * - hold a place holder for an unknown value that can be filled in later + */ +class BufferingOutputStream(segmentSize: Int) extends OutputStream { + + protected final class Segment(size: Int) { + val bytes = new Array[Byte](size) + var written = 0 + var next: Segment = null + def freeSpace: Int = bytes.length - written + } + + protected class ReservedOutput(seg: Segment, offset: Int, length: Int) extends OutputStream { + private[this] var cur = seg + private[this] var off = offset + private[this] var len = length + + override def write(value: Int) = { + if (len <= 0) throw new IndexOutOfBoundsException() + if (cur.bytes.length <= off) { + cur = cur.next + off = 0 + } + cur.bytes(off) = value.toByte + off += 1 + len -= 1 + } + } + + protected var currentSegment = new Segment(segmentSize) + private[this] val headSegment = currentSegment + private[this] var filled = 0 + + def size(): Int = filled + currentSegment.written + + override def write(b: Int): Unit = { + if (currentSegment.freeSpace <= 0) addSegment() + currentSegment.bytes(currentSegment.written) = b.toByte + currentSegment.written += 1 + } + + override def write(b: Array[Byte], off: Int, len: Int) { + if (off >= 0 && off <= b.length && len >= 0 && off + len <= b.length) { + var remaining = len + var offset = off + while (remaining > 0) { + if (currentSegment.freeSpace <= 0) addSegment() + + val amount = math.min(currentSegment.freeSpace, remaining) + System.arraycopy(b, offset, currentSegment.bytes, currentSegment.written, amount) + currentSegment.written += amount + offset += amount + remaining -= amount + } + } else { + throw new IndexOutOfBoundsException() + } + } + + def write(in: InputStream): Unit = { + var amount = 0 + while (amount >= 0) { + currentSegment.written += amount + if (currentSegment.freeSpace <= 0) addSegment() + amount = in.read(currentSegment.bytes, currentSegment.written, currentSegment.freeSpace) + } + } + + private def addSegment() = { + filled += currentSegment.written + val newSeg = new Segment(segmentSize) + currentSegment.next = newSeg + currentSegment = newSeg + } + + private def skip(len: Int): Unit = { + if (len >= 0) { + var remaining = len + while (remaining > 0) { + if (currentSegment.freeSpace <= 0) addSegment() + + val amount = math.min(currentSegment.freeSpace, remaining) + currentSegment.written += amount + remaining -= amount + } + } else { + throw new IndexOutOfBoundsException() + } + } + + def reserve(len: Int): ReservedOutput = { + val out = new ReservedOutput(currentSegment, currentSegment.written, len) + skip(len) + out + } + + def writeTo(buffer: ByteBuffer): Unit = { + var seg = headSegment + while (seg != null) { + buffer.put(seg.bytes, 0, seg.written) + seg = seg.next + } + } + +} diff --git a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala old mode 100644 new mode 100755 index ea9559f822c11..cc0da9f7b89c0 --- a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala +++ b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala @@ -24,7 +24,7 @@ import com.yammer.metrics.Metrics import java.io.File import com.yammer.metrics.reporting.CsvReporter import java.util.concurrent.TimeUnit -import kafka.utils.{Utils, VerifiableProperties, Logging} +import kafka.utils.{CoreUtils, VerifiableProperties, Logging} private trait KafkaCSVMetricsReporterMBean extends KafkaMetricsReporterMBean @@ -48,7 +48,7 @@ private class KafkaCSVMetricsReporter extends KafkaMetricsReporter if (!initialized) { val metricsConfig = new KafkaMetricsConfig(props) csvDir = new File(props.getString("kafka.csv.metrics.dir", "kafka_metrics")) - Utils.rm(csvDir) + CoreUtils.rm(csvDir) csvDir.mkdirs() underlying = new CsvReporter(Metrics.defaultRegistry(), csvDir) if (props.getBoolean("kafka.csv.metrics.reporter.enabled", default = false)) { diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala old mode 100644 new mode 100755 index 84f6208610e09..ad9eb20f3e72d --- a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala @@ -20,7 +20,7 @@ package kafka.metrics -import kafka.utils.{VerifiableProperties, Utils} +import kafka.utils.{VerifiableProperties, CoreUtils} class KafkaMetricsConfig(props: VerifiableProperties) { @@ -28,7 +28,7 @@ class KafkaMetricsConfig(props: VerifiableProperties) { * Comma-separated list of reporter types. These classes should be on the * classpath and will be instantiated at run-time. */ - val reporters = Utils.parseCsvList(props.getString("kafka.metrics.reporters", "")) + val reporters = CoreUtils.parseCsvList(props.getString("kafka.metrics.reporters", "")) /** * The metrics polling interval (in seconds). diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala index a20ab90165cc7..9e31184515697 100644 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala @@ -6,7 +6,7 @@ * (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 + * 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, @@ -18,10 +18,15 @@ package kafka.metrics -import com.yammer.metrics.core.{Gauge, MetricName} -import kafka.utils.Logging import java.util.concurrent.TimeUnit + import com.yammer.metrics.Metrics +import com.yammer.metrics.core.{Gauge, MetricName} +import kafka.consumer.{ConsumerTopicStatsRegistry, FetchRequestAndResponseStatsRegistry} +import kafka.producer.{ProducerRequestStatsRegistry, ProducerStatsRegistry, ProducerTopicStatsRegistry} +import kafka.utils.Logging + +import scala.collection.immutable trait KafkaMetricsGroup extends Logging { @@ -30,25 +35,188 @@ trait KafkaMetricsGroup extends Logging { * Creates a new MetricName object for gauges, meters, etc. created for this * metrics group. * @param name Descriptive name of the metric. + * @param tags Additional attributes which mBean will have. * @return Sanitized metric name object. */ - private def metricName(name: String) = { + private def metricName(name: String, tags: scala.collection.Map[String, String] = Map.empty) = { val klass = this.getClass val pkg = if (klass.getPackage == null) "" else klass.getPackage.getName val simpleName = klass.getSimpleName.replaceAll("\\$$", "") - new MetricName(pkg, simpleName, name) + + explicitMetricName(pkg, simpleName, name, tags) } - def newGauge[T](name: String, metric: Gauge[T]) = - Metrics.defaultRegistry().newGauge(metricName(name), metric) - def newMeter(name: String, eventType: String, timeUnit: TimeUnit) = - Metrics.defaultRegistry().newMeter(metricName(name), eventType, timeUnit) + private def explicitMetricName(group: String, typeName: String, name: String, tags: scala.collection.Map[String, String] = Map.empty) = { + val nameBuilder: StringBuilder = new StringBuilder + + nameBuilder.append(group) + + nameBuilder.append(":type=") - def newHistogram(name: String, biased: Boolean = true) = - Metrics.defaultRegistry().newHistogram(metricName(name), biased) + nameBuilder.append(typeName) - def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit) = - Metrics.defaultRegistry().newTimer(metricName(name), durationUnit, rateUnit) + if (name.length > 0) { + nameBuilder.append(",name=") + nameBuilder.append(name) + } + + val scope: String = KafkaMetricsGroup.toScope(tags).getOrElse(null) + val tagsName = KafkaMetricsGroup.toMBeanName(tags) + tagsName match { + case Some(tn) => + nameBuilder.append(",").append(tn) + case None => + } + + new MetricName(group, typeName, name, scope, nameBuilder.toString()) + } + + def newGauge[T](name: String, metric: Gauge[T], tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().newGauge(metricName(name, tags), metric) + + def newMeter(name: String, eventType: String, timeUnit: TimeUnit, tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().newMeter(metricName(name, tags), eventType, timeUnit) + + def newHistogram(name: String, biased: Boolean = true, tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().newHistogram(metricName(name, tags), biased) + + def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit, tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().newTimer(metricName(name, tags), durationUnit, rateUnit) + + def removeMetric(name: String, tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().removeMetric(metricName(name, tags)) + + +} + +object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { + /** + * To make sure all the metrics be de-registered after consumer/producer close, the metric names should be + * put into the metric name set. + */ + private val consumerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName]( + // kafka.consumer.ZookeeperConsumerConnector + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "FetchQueueSize"), + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "KafkaCommitsPerSec"), + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "ZooKeeperCommitsPerSec"), + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "RebalanceRateAndTime"), + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "OwnedPartitionsCount"), + // kafka.consumer.ConsumerFetcherManager + new MetricName("kafka.consumer", "ConsumerFetcherManager", "MaxLag"), + new MetricName("kafka.consumer", "ConsumerFetcherManager", "MinFetchRate"), + + // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread + new MetricName("kafka.server", "FetcherLagMetrics", "ConsumerLag"), + + // kafka.consumer.ConsumerTopicStats <-- kafka.consumer.{ConsumerIterator, PartitionTopicInfo} + new MetricName("kafka.consumer", "ConsumerTopicMetrics", "MessagesPerSec"), + + // kafka.consumer.ConsumerTopicStats + new MetricName("kafka.consumer", "ConsumerTopicMetrics", "BytesPerSec"), + + // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread + new MetricName("kafka.server", "FetcherStats", "BytesPerSec"), + new MetricName("kafka.server", "FetcherStats", "RequestsPerSec"), + + // kafka.consumer.FetchRequestAndResponseStats <-- kafka.consumer.SimpleConsumer + new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchResponseSize"), + new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchRequestRateAndTimeMs"), + + /** + * ProducerRequestStats <-- SyncProducer + * metric for SyncProducer in fetchTopicMetaData() needs to be removed when consumer is closed. + */ + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"), + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize") + ) + + private val producerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName]( + // kafka.producer.ProducerStats <-- DefaultEventHandler <-- Producer + new MetricName("kafka.producer", "ProducerStats", "SerializationErrorsPerSec"), + new MetricName("kafka.producer", "ProducerStats", "ResendsPerSec"), + new MetricName("kafka.producer", "ProducerStats", "FailedSendsPerSec"), + + // kafka.producer.ProducerSendThread + new MetricName("kafka.producer.async", "ProducerSendThread", "ProducerQueueSize"), + + // kafka.producer.ProducerTopicStats <-- kafka.producer.{Producer, async.DefaultEventHandler} + new MetricName("kafka.producer", "ProducerTopicMetrics", "MessagesPerSec"), + new MetricName("kafka.producer", "ProducerTopicMetrics", "DroppedMessagesPerSec"), + new MetricName("kafka.producer", "ProducerTopicMetrics", "BytesPerSec"), + + // kafka.producer.ProducerRequestStats <-- SyncProducer + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"), + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize") + ) + + private def toMBeanName(tags: collection.Map[String, String]): Option[String] = { + val filteredTags = tags + .filter { case (tagKey, tagValue) => tagValue != ""} + if (filteredTags.nonEmpty) { + val tagsString = filteredTags + .map { case (key, value) => "%s=%s".format(key, value)} + .mkString(",") + + Some(tagsString) + } + else { + None + } + } + + private def toScope(tags: collection.Map[String, String]): Option[String] = { + val filteredTags = tags + .filter { case (tagKey, tagValue) => tagValue != ""} + if (filteredTags.nonEmpty) { + // convert dot to _ since reporters like Graphite typically use dot to represent hierarchy + val tagsString = filteredTags + .toList.sortWith((t1, t2) => t1._1 < t2._1) + .map { case (key, value) => "%s.%s".format(key, value.replaceAll("\\.", "_"))} + .mkString(".") + + Some(tagsString) + } + else { + None + } + } + + def removeAllConsumerMetrics(clientId: String) { + FetchRequestAndResponseStatsRegistry.removeConsumerFetchRequestAndResponseStats(clientId) + ConsumerTopicStatsRegistry.removeConsumerTopicStat(clientId) + ProducerRequestStatsRegistry.removeProducerRequestStats(clientId) + removeAllMetricsInList(KafkaMetricsGroup.consumerMetricNameList, clientId) + } + + def removeAllProducerMetrics(clientId: String) { + ProducerRequestStatsRegistry.removeProducerRequestStats(clientId) + ProducerTopicStatsRegistry.removeProducerTopicStats(clientId) + ProducerStatsRegistry.removeProducerStats(clientId) + removeAllMetricsInList(KafkaMetricsGroup.producerMetricNameList, clientId) + } + + private def removeAllMetricsInList(metricNameList: immutable.List[MetricName], clientId: String) { + metricNameList.foreach(metric => { + val pattern = (".*clientId=" + clientId + ".*").r + val registeredMetrics = scala.collection.JavaConversions.asScalaSet(Metrics.defaultRegistry().allMetrics().keySet()) + for (registeredMetric <- registeredMetrics) { + if (registeredMetric.getGroup == metric.getGroup && + registeredMetric.getName == metric.getName && + registeredMetric.getType == metric.getType) { + pattern.findFirstIn(registeredMetric.getMBeanName) match { + case Some(_) => { + val beforeRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size + Metrics.defaultRegistry().removeMetric(registeredMetric) + val afterRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size + trace("Removing metric %s. Metrics registry size reduced from %d to %d".format( + registeredMetric, beforeRemovalSize, afterRemovalSize)) + } + case _ => + } + } + } + }) + } } diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsReporter.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsReporter.scala old mode 100644 new mode 100755 index 14e46244ee27c..0d6da3466fd59 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsReporter.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsReporter.scala @@ -20,7 +20,7 @@ package kafka.metrics -import kafka.utils.{Utils, VerifiableProperties} +import kafka.utils.{CoreUtils, VerifiableProperties} import java.util.concurrent.atomic.AtomicBoolean @@ -52,14 +52,14 @@ object KafkaMetricsReporter { def startReporters (verifiableProps: VerifiableProperties) { ReporterStarted synchronized { - if (ReporterStarted.get() == false) { + if (!ReporterStarted.get()) { val metricsConfig = new KafkaMetricsConfig(verifiableProps) if(metricsConfig.reporters.size > 0) { metricsConfig.reporters.foreach(reporterType => { - val reporter = Utils.createObject[KafkaMetricsReporter](reporterType) + val reporter = CoreUtils.createObject[KafkaMetricsReporter](reporterType) reporter.init(verifiableProps) if (reporter.isInstanceOf[KafkaMetricsReporterMBean]) - Utils.registerMBean(reporter, reporter.asInstanceOf[KafkaMetricsReporterMBean].getMBeanName) + CoreUtils.registerMBean(reporter, reporter.asInstanceOf[KafkaMetricsReporterMBean].getMBeanName) }) ReporterStarted.set(true) } diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala index ab04b3fe0dc67..1197259cd9f59 100644 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ b/core/src/main/scala/kafka/network/BlockingChannel.scala @@ -19,8 +19,10 @@ package kafka.network import java.net.InetSocketAddress import java.nio.channels._ -import kafka.utils.{nonthreadsafe, Logging} + import kafka.api.RequestOrResponse +import kafka.utils.{Logging, nonthreadsafe} +import org.apache.kafka.common.network.NetworkReceive object BlockingChannel{ @@ -42,64 +44,84 @@ class BlockingChannel( val host: String, private var readChannel: ReadableByteChannel = null private var writeChannel: GatheringByteChannel = null private val lock = new Object() - + private val connectTimeoutMs = readTimeoutMs + private var connectionId: String = "" + def connect() = lock synchronized { if(!connected) { - channel = SocketChannel.open() - if(readBufferSize > 0) - channel.socket.setReceiveBufferSize(readBufferSize) - if(writeBufferSize > 0) - channel.socket.setSendBufferSize(writeBufferSize) - channel.configureBlocking(true) - channel.socket.setSoTimeout(readTimeoutMs) - channel.socket.setKeepAlive(true) - channel.socket.setTcpNoDelay(true) - channel.connect(new InetSocketAddress(host, port)) + try { + channel = SocketChannel.open() + if(readBufferSize > 0) + channel.socket.setReceiveBufferSize(readBufferSize) + if(writeBufferSize > 0) + channel.socket.setSendBufferSize(writeBufferSize) + channel.configureBlocking(true) + channel.socket.setSoTimeout(readTimeoutMs) + channel.socket.setKeepAlive(true) + channel.socket.setTcpNoDelay(true) + channel.socket.connect(new InetSocketAddress(host, port), connectTimeoutMs) + + writeChannel = channel + // Need to create a new ReadableByteChannel from input stream because SocketChannel doesn't implement read with timeout + // See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work + readChannel = Channels.newChannel(channel.socket().getInputStream) + connected = true + val localHost = channel.socket.getLocalAddress.getHostAddress + val localPort = channel.socket.getLocalPort + val remoteHost = channel.socket.getInetAddress.getHostAddress + val remotePort = channel.socket.getPort + connectionId = localHost + ":" + localPort + "-" + remoteHost + ":" + remotePort + // settings may not match what we requested above + val msg = "Created socket with SO_TIMEOUT = %d (requested %d), SO_RCVBUF = %d (requested %d), SO_SNDBUF = %d (requested %d), connectTimeoutMs = %d." + debug(msg.format(channel.socket.getSoTimeout, + readTimeoutMs, + channel.socket.getReceiveBufferSize, + readBufferSize, + channel.socket.getSendBufferSize, + writeBufferSize, + connectTimeoutMs)) - writeChannel = channel - readChannel = Channels.newChannel(channel.socket().getInputStream) - connected = true - // settings may not match what we requested above - val msg = "Created socket with SO_TIMEOUT = %d (requested %d), SO_RCVBUF = %d (requested %d), SO_SNDBUF = %d (requested %d)." - debug(msg.format(channel.socket.getSoTimeout, - readTimeoutMs, - channel.socket.getReceiveBufferSize, - readBufferSize, - channel.socket.getSendBufferSize, - writeBufferSize)) + } catch { + case e: Throwable => disconnect() + } } } def disconnect() = lock synchronized { - if(connected || channel != null) { - // closing the main socket channel *should* close the read channel - // but let's do it to be sure. + if(channel != null) { swallow(channel.close()) swallow(channel.socket.close()) - if(readChannel != null) swallow(readChannel.close()) - channel = null; readChannel = null; writeChannel = null - connected = false + channel = null + writeChannel = null } + // closing the main socket channel *should* close the read channel + // but let's do it to be sure. + if(readChannel != null) { + swallow(readChannel.close()) + readChannel = null + } + connected = false } def isConnected = connected - - def send(request: RequestOrResponse):Int = { + + def send(request: RequestOrResponse): Long = { if(!connected) throw new ClosedChannelException() - val send = new BoundedByteBufferSend(request) + val send = new RequestOrResponseSend(connectionId, request) send.writeCompletely(writeChannel) } - def receive(): Receive = { + def receive(): NetworkReceive = { if(!connected) throw new ClosedChannelException() - val response = new BoundedByteBufferReceive() + val response = new NetworkReceive() response.readCompletely(readChannel) + response.payload().rewind() response } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/network/BoundedByteBufferReceive.scala b/core/src/main/scala/kafka/network/BoundedByteBufferReceive.scala deleted file mode 100644 index a44254564531a..0000000000000 --- a/core/src/main/scala/kafka/network/BoundedByteBufferReceive.scala +++ /dev/null @@ -1,90 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.network - -import java.nio._ -import java.nio.channels._ -import kafka.utils._ - -/** - * Represents a communication between the client and server - * - */ -@nonthreadsafe -private[kafka] class BoundedByteBufferReceive(val maxSize: Int) extends Receive with Logging { - - private val sizeBuffer = ByteBuffer.allocate(4) - private var contentBuffer: ByteBuffer = null - - def this() = this(Int.MaxValue) - - var complete: Boolean = false - - /** - * Get the content buffer for this transmission - */ - def buffer: ByteBuffer = { - expectComplete() - contentBuffer - } - - /** - * Read the bytes in this response from the given channel - */ - def readFrom(channel: ReadableByteChannel): Int = { - expectIncomplete() - var read = 0 - // have we read the request size yet? - if(sizeBuffer.remaining > 0) - read += Utils.read(channel, sizeBuffer) - // have we allocated the request buffer yet? - if(contentBuffer == null && !sizeBuffer.hasRemaining) { - sizeBuffer.rewind() - val size = sizeBuffer.getInt() - if(size <= 0) - throw new InvalidRequestException("%d is not a valid request size.".format(size)) - if(size > maxSize) - throw new InvalidRequestException("Request of length %d is not valid, it is larger than the maximum size of %d bytes.".format(size, maxSize)) - contentBuffer = byteBufferAllocate(size) - } - // if we have a buffer read some stuff into it - if(contentBuffer != null) { - read = Utils.read(channel, contentBuffer) - // did we get everything? - if(!contentBuffer.hasRemaining) { - contentBuffer.rewind() - complete = true - } - } - read - } - - private def byteBufferAllocate(size: Int): ByteBuffer = { - var buffer: ByteBuffer = null - try { - buffer = ByteBuffer.allocate(size) - } catch { - case e: OutOfMemoryError => - error("OOME with size " + size, e) - throw e - case e2: Throwable => - throw e2 - } - buffer - } -} diff --git a/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala b/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala deleted file mode 100644 index a624359fb2059..0000000000000 --- a/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala +++ /dev/null @@ -1,63 +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.network - -import java.nio._ -import java.nio.channels._ -import kafka.utils._ -import kafka.api.RequestOrResponse - -@nonthreadsafe -private[kafka] class BoundedByteBufferSend(val buffer: ByteBuffer) extends Send { - - private var sizeBuffer = ByteBuffer.allocate(4) - - // Avoid possibility of overflow for 2GB-4 byte buffer - if(buffer.remaining > Int.MaxValue - sizeBuffer.limit) - throw new IllegalStateException("Attempt to create a bounded buffer of " + buffer.remaining + " bytes, but the maximum " + - "allowable size for a bounded buffer is " + (Int.MaxValue - sizeBuffer.limit) + ".") - sizeBuffer.putInt(buffer.limit) - sizeBuffer.rewind() - - var complete: Boolean = false - - def this(size: Int) = this(ByteBuffer.allocate(size)) - - def this(request: RequestOrResponse) = { - this(request.sizeInBytes + (if(request.requestId != None) 2 else 0)) - request.requestId match { - case Some(requestId) => - buffer.putShort(requestId) - case None => - } - - request.writeTo(buffer) - buffer.rewind() - } - - - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - var written = channel.write(Array(sizeBuffer, buffer)) - // if we are done, mark it off - if(!buffer.hasRemaining) - complete = true - written.asInstanceOf[Int] - } - -} diff --git a/core/src/main/scala/kafka/network/Handler.scala b/core/src/main/scala/kafka/network/Handler.scala index a0300336b8cb5..1a7d56e5978ef 100644 --- a/core/src/main/scala/kafka/network/Handler.scala +++ b/core/src/main/scala/kafka/network/Handler.scala @@ -17,17 +17,19 @@ package kafka.network +import org.apache.kafka.common.network.{NetworkReceive, Send} + private[kafka] object Handler { /** * A request handler is a function that turns an incoming * transmission into an outgoing transmission */ - type Handler = Receive => Option[Send] + type Handler = NetworkReceive => Option[Send] /** * A handler mapping finds the right Handler function for a given request */ - type HandlerMapping = (Short, Receive) => Handler + type HandlerMapping = (Short, NetworkReceive) => Handler } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index a6ec970d72fc8..20741281dcaa7 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -17,20 +17,23 @@ package kafka.network +import java.nio.ByteBuffer import java.util.concurrent._ -import kafka.metrics.KafkaMetricsGroup + import com.yammer.metrics.core.Gauge -import java.nio.ByteBuffer import kafka.api._ import kafka.common.TopicAndPartition -import kafka.utils.{Logging, SystemTime} import kafka.message.ByteBufferMessageSet -import java.net._ +import kafka.metrics.KafkaMetricsGroup +import kafka.utils.{Logging, SystemTime} +import org.apache.kafka.common.network.Send +import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} +import org.apache.kafka.common.requests.{AbstractRequest, RequestHeader} import org.apache.log4j.Logger object RequestChannel extends Logging { - val AllDone = new Request(1, 2, getShutdownReceive(), 0) + val AllDone = new Request(processor = 1, connectionId = "2", buffer = getShutdownReceive(), startTimeMs = 0, securityProtocol = SecurityProtocol.PLAINTEXT) def getShutdownReceive() = { val emptyProducerRequest = new ProducerRequest(0, 0, "", 0, 0, collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]()) @@ -41,16 +44,43 @@ object RequestChannel extends Logging { byteBuffer } - case class Request(processor: Int, requestKey: Any, private var buffer: ByteBuffer, startTimeMs: Long, remoteAddress: SocketAddress = new InetSocketAddress(0)) { + case class Request(processor: Int, connectionId: String, private var buffer: ByteBuffer, startTimeMs: Long, securityProtocol: SecurityProtocol) { @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) + // for server-side request / response format + // TODO: this will be removed once we migrated to client-side format + val requestObj = + if ( RequestKeys.keyToNameAndDeserializerMap.contains(requestId)) + RequestKeys.deserializerForKey(requestId)(buffer) + else + null + // for client-side request / response format + val header: RequestHeader = + if (requestObj == null) { + buffer.rewind + RequestHeader.parse(buffer) + } else + null + val body: AbstractRequest = + if (requestObj == null) + AbstractRequest.getRequest(header.apiKey, header.apiVersion, buffer) + else + null + buffer = null private val requestLogger = Logger.getLogger("kafka.request.logger") - trace("Processor %d received request : %s".format(processor, requestObj)) + + private def requestDesc: String = { + if (requestObj != null) + requestObj.describe(false) + else + header.toString + " -- " + body.toString + } + + trace("Processor %d received request : %s".format(processor, requestDesc)) def updateRequestMetrics() { val endTimeMs = SystemTime.milliseconds @@ -64,7 +94,7 @@ object RequestChannel extends Logging { val responseQueueTime = (responseDequeueTimeMs - responseCompleteTimeMs).max(0L) val responseSendTime = (endTimeMs - responseDequeueTimeMs).max(0L) val totalTime = endTimeMs - startTimeMs - var metricsList = List(RequestMetrics.metricsMap(RequestKeys.nameForKey(requestId))) + var metricsList = List(RequestMetrics.metricsMap(ApiKeys.forId(requestId).name)) if (requestId == RequestKeys.FetchKey) { val isFromFollower = requestObj.asInstanceOf[FetchRequest].isFromFollower metricsList ::= ( if (isFromFollower) @@ -81,13 +111,13 @@ object RequestChannel extends Logging { m.responseSendTimeHist.update(responseSendTime) m.totalTimeHist.update(totalTime) } + if(requestLogger.isTraceEnabled) - 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)) - } + requestLogger.trace("Completed request:%s from connection %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" + .format(requestDesc, connectionId, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) + else if(requestLogger.isDebugEnabled) + requestLogger.debug("Completed request:%s from connection %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d" + .format(requestDesc, connectionId, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime)) } } @@ -121,12 +151,16 @@ class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMe } ) - for(i <- 0 until numProcessors) { - newGauge( - "Processor-" + i + "-ResponseQueueSize", + newGauge("ResponseQueueSize", new Gauge[Int]{ + def value = responseQueues.foldLeft(0) {(total, q) => total + q.size()} + }) + + for (i <- 0 until numProcessors) { + newGauge("ResponseQueueSize", new Gauge[Int] { def value = responseQueues(i).size() - } + }, + Map("processor" -> i.toString) ) } @@ -156,6 +190,10 @@ class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMe onResponse(processor) } + /** Get the next request or block until specified time has elapsed */ + def receiveRequest(timeout: Long): RequestChannel.Request = + requestQueue.poll(timeout, TimeUnit.MILLISECONDS) + /** Get the next request or block until there is one */ def receiveRequest(): RequestChannel.Request = requestQueue.take() @@ -179,24 +217,25 @@ class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMe object RequestMetrics { val metricsMap = new scala.collection.mutable.HashMap[String, RequestMetrics] - val consumerFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "-Consumer" - val followFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "-Follower" - (RequestKeys.keyToNameAndDeserializerMap.values.map(e => e._1) + val consumerFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "Consumer" + val followFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "Follower" + (ApiKeys.values().toList.map(e => e.name) ++ List(consumerFetchMetricName, followFetchMetricName)).foreach(name => metricsMap.put(name, new RequestMetrics(name))) } class RequestMetrics(name: String) extends KafkaMetricsGroup { - val requestRate = newMeter(name + "-RequestsPerSec", "requests", TimeUnit.SECONDS) + val tags = Map("request" -> name) + val requestRate = newMeter("RequestsPerSec", "requests", TimeUnit.SECONDS, tags) // time a request spent in a request queue - val requestQueueTimeHist = newHistogram(name + "-RequestQueueTimeMs") + val requestQueueTimeHist = newHistogram("RequestQueueTimeMs", biased = true, tags) // time a request takes to be processed at the local broker - val localTimeHist = newHistogram(name + "-LocalTimeMs") + val localTimeHist = newHistogram("LocalTimeMs", biased = true, tags) // time a request takes to wait on remote brokers (only relevant to fetch and produce requests) - val remoteTimeHist = newHistogram(name + "-RemoteTimeMs") + val remoteTimeHist = newHistogram("RemoteTimeMs", biased = true, tags) // time a response spent in a response queue - val responseQueueTimeHist = newHistogram(name + "-ResponseQueueTimeMs") + val responseQueueTimeHist = newHistogram("ResponseQueueTimeMs", biased = true, tags) // time to send the response to the requester - val responseSendTimeHist = newHistogram(name + "-ResponseSendTimeMs") - val totalTimeHist = newHistogram(name + "-TotalTimeMs") + val responseSendTimeHist = newHistogram("ResponseSendTimeMs", biased = true, tags) + val totalTimeHist = newHistogram("TotalTimeMs", biased = true, tags) } diff --git a/core/src/main/scala/kafka/network/RequestOrResponseSend.scala b/core/src/main/scala/kafka/network/RequestOrResponseSend.scala new file mode 100644 index 0000000000000..364f24bdc5625 --- /dev/null +++ b/core/src/main/scala/kafka/network/RequestOrResponseSend.scala @@ -0,0 +1,57 @@ +/* + * 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.network + +import java.nio.ByteBuffer +import java.nio.channels.GatheringByteChannel + +import kafka.api.RequestOrResponse +import kafka.utils.Logging +import org.apache.kafka.common.network.NetworkSend + +object RequestOrResponseSend { + def serialize(request: RequestOrResponse): ByteBuffer = { + val buffer = ByteBuffer.allocate(request.sizeInBytes + (if(request.requestId != None) 2 else 0)) + request.requestId match { + case Some(requestId) => + buffer.putShort(requestId) + case None => + } + request.writeTo(buffer) + buffer.rewind() + buffer + } +} + +class RequestOrResponseSend(val dest: String, val buffer: ByteBuffer) extends NetworkSend(dest, buffer) with Logging { + + def this(dest: String, request: RequestOrResponse) { + this(dest, RequestOrResponseSend.serialize(request)) + } + + def writeCompletely(channel: GatheringByteChannel): Long = { + var totalWritten = 0L + while(!completed()) { + val written = writeTo(channel) + trace(written + " bytes written.") + totalWritten += written + } + totalWritten + } + +} diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 216245deb119c..91319fa010b14 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -17,14 +17,26 @@ package kafka.network -import java.util.concurrent._ -import java.util.concurrent.atomic._ +import java.io.IOException import java.net._ -import java.io._ import java.nio.channels._ +import java.util +import java.util.concurrent._ +import java.util.concurrent.atomic._ +import com.yammer.metrics.core.Gauge +import kafka.cluster.EndPoint import kafka.common.KafkaException +import kafka.metrics.KafkaMetricsGroup import kafka.utils._ +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.network.InvalidReceiveException +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.protocol.types.SchemaException +import org.apache.kafka.common.utils.{SystemTime, Time, Utils} + +import scala.collection._ /** * An NIO socket server. The threading model is @@ -33,35 +45,84 @@ import kafka.utils._ * M Handler threads that handle requests and produce responses back to the processor threads for writing. */ class SocketServer(val brokerId: Int, - val host: String, - val port: Int, + val endpoints: Map[SecurityProtocol, EndPoint], val numProcessorThreads: Int, val maxQueuedRequests: Int, val sendBufferSize: Int, val recvBufferSize: Int, - val maxRequestSize: Int = Int.MaxValue) extends Logging { + val maxRequestSize: Int = Int.MaxValue, + val maxConnectionsPerIp: Int = Int.MaxValue, + val connectionsMaxIdleMs: Long, + val maxConnectionsPerIpOverrides: Map[String, Int], + val time: Time, + val metrics: Metrics) extends Logging with KafkaMetricsGroup { this.logIdent = "[Socket Server on Broker " + brokerId + "], " - private val time = SystemTime + private val processors = new Array[Processor](numProcessorThreads) - @volatile private var acceptor: Acceptor = null + private[network] var acceptors = mutable.Map[EndPoint,Acceptor]() val requestChannel = new RequestChannel(numProcessorThreads, maxQueuedRequests) + private val allMetricNames = (0 until numProcessorThreads).map { i => + val tags = new util.HashMap[String, String]() + tags.put("networkProcessor", i.toString) + new MetricName("io-wait-ratio", "socket-server-metrics", tags) + } + + /* I'm pushing the mapping of port-to-protocol to the processor level, + so the processor can put the correct protocol in the request channel. + we'll probably have a more elegant way of doing this once we patch the request channel + to include more information about security and authentication. + TODO: re-consider this code when working on KAFKA-1683 + */ + private val portToProtocol: ConcurrentHashMap[Int, SecurityProtocol] = new ConcurrentHashMap[Int, SecurityProtocol]() + /** * Start the socket server */ def startup() { - for(i <- 0 until numProcessorThreads) { - processors(i) = new Processor(i, time, maxRequestSize, requestChannel) - Utils.newThread("kafka-processor-%d-%d".format(port, i), processors(i), false).start() + val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides) + + newGauge("NetworkProcessorAvgIdlePercent", + new Gauge[Double] { + def value = allMetricNames.map( metricName => + metrics.metrics().get(metricName).value()).sum / numProcessorThreads + } + ) + + + this.synchronized { + for (i <- 0 until numProcessorThreads) { + processors(i) = new Processor(i, + time, + maxRequestSize, + numProcessorThreads, + requestChannel, + quotas, + connectionsMaxIdleMs, + portToProtocol, + metrics + ) + Utils.newThread("kafka-network-thread-%d-%d".format(brokerId, i), processors(i), false).start() + } } + // register the processor threads for notification of responses requestChannel.addResponseListener((id:Int) => processors(id).wakeup()) // start accepting connections - this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize) - Utils.newThread("kafka-acceptor", acceptor, false).start() - acceptor.awaitStartup - info("Started") + // right now we will use the same processors for all ports, since we didn't implement different protocols + // in the future, we may implement different processors for SSL and Kerberos + + this.synchronized { + endpoints.values.foreach(endpoint => { + val acceptor = new Acceptor(endpoint.host, endpoint.port, processors, sendBufferSize, recvBufferSize, quotas, endpoint.protocolType, portToProtocol) + acceptors.put(endpoint, acceptor) + Utils.newThread("kafka-socket-acceptor-%s-%d".format(endpoint.protocolType.toString, endpoint.port), acceptor, false).start() + acceptor.awaitStartup + }) + } + + info("Started " + acceptors.size + " acceptor threads") } /** @@ -69,31 +130,40 @@ class SocketServer(val brokerId: Int, */ def shutdown() = { info("Shutting down") - if(acceptor != null) - acceptor.shutdown() - for(processor <- processors) - processor.shutdown() + this.synchronized { + acceptors.values.foreach(_.shutdown) + processors.foreach(_.shutdown) + } info("Shutdown completed") } + + def boundPort(protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Int = { + try { + acceptors(endpoints(protocol)).serverChannel.socket().getLocalPort + } catch { + case e: Exception => throw new KafkaException("Tried to check server's port before server was started or checked for port of non-existing protocol", e) + } + } } /** * A base class with some helper variables and methods */ -private[kafka] abstract class AbstractServerThread extends Runnable with Logging { +private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQuotas) extends Runnable with Logging { - protected val selector = Selector.open(); private val startupLatch = new CountDownLatch(1) private val shutdownLatch = new CountDownLatch(1) - private val alive = new AtomicBoolean(false) + private val alive = new AtomicBoolean(true) + + def wakeup() /** * Initiates a graceful shutdown by signaling to stop and waiting for the shutdown to complete */ def shutdown(): Unit = { alive.set(false) - selector.wakeup() - shutdownLatch.await + wakeup() + shutdownLatch.await() } /** @@ -105,14 +175,13 @@ private[kafka] abstract class AbstractServerThread extends Runnable with Logging * Record that the thread startup is complete */ protected def startupComplete() = { - alive.set(true) - startupLatch.countDown + startupLatch.countDown() } /** * Record that the thread shutdown is complete */ - protected def shutdownComplete() = shutdownLatch.countDown + protected def shutdownComplete() = shutdownLatch.countDown() /** * Is the server still running? @@ -120,30 +189,52 @@ private[kafka] abstract class AbstractServerThread extends Runnable with Logging protected def isRunning = alive.get /** - * Wakeup the thread for selection. + * Close the given key and associated socket */ - def wakeup() = selector.wakeup() + def close(key: SelectionKey) { + if(key != null) { + key.attach(null) + close(key.channel.asInstanceOf[SocketChannel]) + swallowError(key.cancel()) + } + } + def close(channel: SocketChannel) { + if(channel != null) { + debug("Closing connection from " + channel.socket.getRemoteSocketAddress()) + connectionQuotas.dec(channel.socket.getInetAddress) + swallowError(channel.socket().close()) + swallowError(channel.close()) + } + } } /** * Thread that accepts and configures new connections. There is only need for one of these */ -private[kafka] class Acceptor(val host: String, val port: Int, private val processors: Array[Processor], - val sendBufferSize: Int, val recvBufferSize: Int) extends AbstractServerThread { +private[kafka] class Acceptor(val host: String, + private val port: Int, + private val processors: Array[Processor], + val sendBufferSize: Int, + val recvBufferSize: Int, + connectionQuotas: ConnectionQuotas, + protocol: SecurityProtocol, + portToProtocol: ConcurrentHashMap[Int, SecurityProtocol]) extends AbstractServerThread(connectionQuotas) { + val nioSelector = java.nio.channels.Selector.open() val serverChannel = openServerSocket(host, port) + portToProtocol.put(serverChannel.socket().getLocalPort, protocol) /** * Accept loop that checks for new connection attempts */ def run() { - serverChannel.register(selector, SelectionKey.OP_ACCEPT); + serverChannel.register(nioSelector, SelectionKey.OP_ACCEPT); startupComplete() var currentProcessor = 0 while(isRunning) { - val ready = selector.select(500) + val ready = nioSelector.select(500) if(ready > 0) { - val keys = selector.selectedKeys() + val keys = nioSelector.selectedKeys() val iter = keys.iterator() while(iter.hasNext && isRunning) { var key: SelectionKey = null @@ -151,21 +242,21 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce key = iter.next iter.remove() if(key.isAcceptable) - accept(key, processors(currentProcessor)) - else - throw new IllegalStateException("Unrecognized key state for acceptor thread.") + accept(key, processors(currentProcessor)) + else + throw new IllegalStateException("Unrecognized key state for acceptor thread.") - // round robin to the next processor thread - currentProcessor = (currentProcessor + 1) % processors.length + // round robin to the next processor thread + currentProcessor = (currentProcessor + 1) % processors.length } catch { - case e: Throwable => error("Error in acceptor", e) + case e: Throwable => error("Error while accepting connection", e) } } } } debug("Closing server socket and selector.") swallowError(serverChannel.close()) - swallowError(selector.close()) + swallowError(nioSelector.close()) shutdownComplete() } @@ -180,9 +271,10 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce new InetSocketAddress(host, port) val serverChannel = ServerSocketChannel.open() serverChannel.configureBlocking(false) + serverChannel.socket().setReceiveBufferSize(recvBufferSize) try { serverChannel.socket.bind(socketAddress) - info("Awaiting socket connections on %s:%d.".format(socketAddress.getHostName, port)) + info("Awaiting socket connections on %s:%d.".format(socketAddress.getHostName, serverChannel.socket.getLocalPort)) } catch { case e: SocketException => throw new KafkaException("Socket server failed to bind to %s:%d: %s.".format(socketAddress.getHostName, port, e.getMessage), e) @@ -195,21 +287,33 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce */ def accept(key: SelectionKey, processor: Processor) { val serverSocketChannel = key.channel().asInstanceOf[ServerSocketChannel] - serverSocketChannel.socket().setReceiveBufferSize(recvBufferSize) - val socketChannel = serverSocketChannel.accept() - socketChannel.configureBlocking(false) - socketChannel.socket().setTcpNoDelay(true) - socketChannel.socket().setSendBufferSize(sendBufferSize) + try { + connectionQuotas.inc(socketChannel.socket().getInetAddress) + socketChannel.configureBlocking(false) + socketChannel.socket().setTcpNoDelay(true) + socketChannel.socket().setKeepAlive(true) + socketChannel.socket().setSendBufferSize(sendBufferSize) - debug("Accepted connection from %s on %s. sendBufferSize [actual|requested]: [%d|%d] recvBufferSize [actual|requested]: [%d|%d]" - .format(socketChannel.socket.getInetAddress, socketChannel.socket.getLocalSocketAddress, + debug("Accepted connection from %s on %s. sendBufferSize [actual|requested]: [%d|%d] recvBufferSize [actual|requested]: [%d|%d]" + .format(socketChannel.socket.getInetAddress, socketChannel.socket.getLocalSocketAddress, socketChannel.socket.getSendBufferSize, sendBufferSize, socketChannel.socket.getReceiveBufferSize, recvBufferSize)) - processor.accept(socketChannel) + processor.accept(socketChannel) + } catch { + case e: TooManyConnectionsException => + info("Rejected connection from %s, address already has the configured maximum of %d connections.".format(e.ip, e.count)) + close(socketChannel) + } } + /** + * Wakeup the thread for selection. + */ + @Override + def wakeup = nioSelector.wakeup() + } /** @@ -219,9 +323,36 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce private[kafka] class Processor(val id: Int, val time: Time, val maxRequestSize: Int, - val requestChannel: RequestChannel) extends AbstractServerThread { - - private val newConnections = new ConcurrentLinkedQueue[SocketChannel](); + val totalProcessorThreads: Int, + val requestChannel: RequestChannel, + connectionQuotas: ConnectionQuotas, + val connectionsMaxIdleMs: Long, + val portToProtocol: ConcurrentHashMap[Int,SecurityProtocol], + val metrics: Metrics) extends AbstractServerThread(connectionQuotas) with KafkaMetricsGroup { + + private val newConnections = new ConcurrentLinkedQueue[SocketChannel]() + private val inflightResponses = mutable.Map[String, RequestChannel.Response]() + + private val metricTags = new util.HashMap[String, String]() + metricTags.put("networkProcessor", id.toString) + + newGauge("IdlePercent", + new Gauge[Double] { + def value = { + metrics.metrics().get(new MetricName("io-wait-ratio", "socket-server-metrics", metricTags)).value() + } + }, + JavaConversions.mapAsScalaMap(metricTags) + ) + + private val selector = new org.apache.kafka.common.network.Selector( + maxRequestSize, + connectionsMaxIdleMs, + metrics, + time, + "socket-server", + metricTags, + false) override def run() { startupComplete() @@ -230,49 +361,51 @@ private[kafka] class Processor(val id: Int, configureNewConnections() // register any new responses for writing processNewResponses() - val startSelectTime = SystemTime.milliseconds - val ready = selector.select(300) - trace("Processor id " + id + " selection time = " + (SystemTime.milliseconds - startSelectTime) + " ms") - if(ready > 0) { - val keys = selector.selectedKeys() - val iter = keys.iterator() - while(iter.hasNext && isRunning) { - var key: SelectionKey = null - try { - key = iter.next - iter.remove() - if(key.isReadable) - read(key) - else if(key.isWritable) - write(key) - else if(!key.isValid) - close(key) - else - throw new IllegalStateException("Unrecognized key state for processor thread.") - } catch { - case e: EOFException => { - info("Closing socket connection to %s.".format(channelFor(key).socket.getInetAddress)) - close(key) - } case e: InvalidRequestException => { - info("Closing socket connection to %s due to invalid request: %s".format(channelFor(key).socket.getInetAddress, e.getMessage)) - close(key) - } case e: Throwable => { - error("Closing socket for " + channelFor(key).socket.getInetAddress + " because of error", e) - close(key) - } - } + + try { + selector.poll(300) + } catch { + case e @ (_: IllegalStateException | _: IOException) => { + error("Closing processor %s due to illegal state or IO exception".format(id)) + swallow(closeAll()) + shutdownComplete() + throw e } + case e: InvalidReceiveException => + // Log warning and continue since Selector already closed the connection + warn("Connection was closed due to invalid receive. Processor will continue handling other connections") } + collection.JavaConversions.collectionAsScalaIterable(selector.completedReceives).foreach( receive => { + try { + val req = RequestChannel.Request(processor = id, connectionId = receive.source, buffer = receive.payload, startTimeMs = time.milliseconds, securityProtocol = SecurityProtocol.PLAINTEXT) + requestChannel.sendRequest(req) + } catch { + case e @ (_: InvalidRequestException | _: SchemaException) => { + // note that even though we got an exception, we can assume that receive.source is valid. Issues with constructing a valid receive object were handled earlier + error("Closing socket for " + receive.source + " because of error", e) + selector.close(receive.source) + } + } + selector.mute(receive.source) + }) + + collection.JavaConversions.iterableAsScalaIterable(selector.completedSends()).foreach( send => { + val resp = inflightResponses.remove(send.destination()).get + resp.request.updateRequestMetrics() + selector.unmute(send.destination()) + }) } - debug("Closing selector.") - swallowError(selector.close()) + + + + debug("Closing selector - processor " + id) + closeAll() shutdownComplete() } private def processNewResponses() { var curr = requestChannel.receiveResponse(id) while(curr != null) { - val key = curr.request.requestKey.asInstanceOf[SelectionKey] try { curr.responseAction match { case RequestChannel.NoOpAction => { @@ -280,40 +413,26 @@ private[kafka] class Processor(val id: Int, // that are sitting in the server's socket buffer curr.request.updateRequestMetrics trace("Socket server received empty response to send, registering for read: " + curr) - key.interestOps(SelectionKey.OP_READ) - key.attach(null) + selector.unmute(curr.request.connectionId) } case RequestChannel.SendAction => { - trace("Socket server received response to send, registering for write: " + curr) - key.interestOps(SelectionKey.OP_WRITE) - key.attach(curr) + trace("Socket server received response to send, registering for write and sending data: " + curr) + selector.send(curr.responseSend) + inflightResponses += (curr.request.connectionId -> curr) } case RequestChannel.CloseConnectionAction => { curr.request.updateRequestMetrics trace("Closing socket connection actively according to the response code.") - close(key) + selector.close(curr.request.connectionId) } - case responseCode => throw new KafkaException("No mapping found for response code " + responseCode) - } - } catch { - case e: CancelledKeyException => { - debug("Ignoring response for closed socket.") - close(key) } + + } finally { curr = requestChannel.receiveResponse(id) } } } - - private def close(key: SelectionKey) { - val channel = key.channel.asInstanceOf[SocketChannel] - debug("Closing connection from " + channel.socket.getRemoteSocketAddress()) - swallowError(channel.socket().close()) - swallowError(channel.close()) - key.attach(null) - swallowError(key.cancel()) - } /** * Queue up a new connection for reading @@ -327,65 +446,57 @@ private[kafka] class Processor(val id: Int, * Register any new connections that have been queued up */ private def configureNewConnections() { - while(newConnections.size() > 0) { + while(!newConnections.isEmpty) { val channel = newConnections.poll() debug("Processor " + id + " listening to new connection from " + channel.socket.getRemoteSocketAddress) - channel.register(selector, SelectionKey.OP_READ) + val localHost = channel.socket().getLocalAddress.getHostAddress + val localPort = channel.socket().getLocalPort + val remoteHost = channel.socket().getInetAddress.getHostAddress + val remotePort = channel.socket().getPort + val connectionId = localHost + ":" + localPort + "-" + remoteHost + ":" + remotePort + selector.register(connectionId, channel) } } - /* - * Process reads from ready sockets + /** + * Close all open connections */ - def read(key: SelectionKey) { - val socketChannel = channelFor(key) - var receive = key.attachment.asInstanceOf[Receive] - if(key.attachment == null) { - receive = new BoundedByteBufferReceive(maxRequestSize) - key.attach(receive) - } - val read = receive.readFrom(socketChannel) - val address = socketChannel.socket.getRemoteSocketAddress(); - trace(read + " bytes read from " + address) - if(read < 0) { - close(key) - } else if(receive.complete) { - val req = RequestChannel.Request(processor = id, requestKey = key, buffer = receive.buffer, startTimeMs = time.milliseconds, remoteAddress = address) - requestChannel.sendRequest(req) - key.attach(null) - // explicitly reset interest ops to not READ, no need to wake up the selector just yet - key.interestOps(key.interestOps & (~SelectionKey.OP_READ)) - } else { - // more reading to be done - trace("Did not finish reading, registering for read again on connection " + socketChannel.socket.getRemoteSocketAddress()) - key.interestOps(SelectionKey.OP_READ) - wakeup() - } + def closeAll() { + selector.close() } - /* - * Process writes to ready sockets + /** + * Wakeup the thread for selection. */ - def write(key: SelectionKey) { - val socketChannel = channelFor(key) - val response = key.attachment().asInstanceOf[RequestChannel.Response] - val responseSend = response.responseSend - if(responseSend == null) - throw new IllegalStateException("Registered for write interest but no response attached to key.") - val written = responseSend.writeTo(socketChannel) - trace(written + " bytes written to " + socketChannel.socket.getRemoteSocketAddress() + " using key " + key) - if(responseSend.complete) { - response.request.updateRequestMetrics() - key.attach(null) - trace("Finished writing, registering for read on connection " + socketChannel.socket.getRemoteSocketAddress()) - key.interestOps(SelectionKey.OP_READ) - } else { - trace("Did not finish writing, registering for write again on connection " + socketChannel.socket.getRemoteSocketAddress()) - key.interestOps(SelectionKey.OP_WRITE) - wakeup() - } - } + @Override + def wakeup = selector.wakeup() - private def channelFor(key: SelectionKey) = key.channel().asInstanceOf[SocketChannel] +} +class ConnectionQuotas(val defaultMax: Int, overrideQuotas: Map[String, Int]) { + private val overrides = overrideQuotas.map(entry => (InetAddress.getByName(entry._1), entry._2)) + private val counts = mutable.Map[InetAddress, Int]() + + def inc(addr: InetAddress) { + counts synchronized { + val count = counts.getOrElse(addr, 0) + counts.put(addr, count + 1) + val max = overrides.getOrElse(addr, defaultMax) + if(count >= max) + throw new TooManyConnectionsException(addr, max) + } + } + + def dec(addr: InetAddress) { + counts synchronized { + val count = counts.get(addr).get + if(count == 1) + counts.remove(addr) + else + counts.put(addr, count - 1) + } + } + } + +class TooManyConnectionsException(val ip: InetAddress, val count: Int) extends KafkaException("Too many connections from %s (maximum = %d)".format(ip, count)) diff --git a/core/src/main/scala/kafka/network/Transmission.scala b/core/src/main/scala/kafka/network/Transmission.scala deleted file mode 100644 index 2827103d7e577..0000000000000 --- a/core/src/main/scala/kafka/network/Transmission.scala +++ /dev/null @@ -1,122 +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.network - -import java.nio._ -import java.nio.channels._ -import kafka.utils.Logging -import kafka.common.KafkaException - -/** - * Represents a stateful transfer of data to or from the network - */ -private[network] trait Transmission extends Logging { - - def complete: Boolean - - protected def expectIncomplete(): Unit = { - if(complete) - throw new KafkaException("This operation cannot be completed on a complete request.") - } - - protected def expectComplete(): Unit = { - if(!complete) - throw new KafkaException("This operation cannot be completed on an incomplete request.") - } - -} - -/** - * A transmission that is being received from a channel - */ -trait Receive extends Transmission { - - def buffer: ByteBuffer - - def readFrom(channel: ReadableByteChannel): Int - - def readCompletely(channel: ReadableByteChannel): Int = { - var totalRead = 0 - while(!complete) { - val read = readFrom(channel) - trace(read + " bytes read.") - totalRead += read - } - totalRead - } - -} - -/** - * A transmission that is being sent out to the channel - */ -trait Send extends Transmission { - - def writeTo(channel: GatheringByteChannel): Int - - def writeCompletely(channel: GatheringByteChannel): Int = { - var totalWritten = 0 - while(!complete) { - val written = writeTo(channel) - trace(written + " bytes written.") - totalWritten += written - } - totalWritten - } - -} - -/** - * A set of composite sends, sent one after another - */ -abstract class MultiSend[S <: Send](val sends: List[S]) extends Send { - val expectedBytesToWrite: Int - private var current = sends - var totalWritten = 0 - - /** - * This method continues to write to the socket buffer till an incomplete - * write happens. On an incomplete write, it returns to the caller to give it - * a chance to schedule other work till the buffered write completes. - */ - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete - var totalWrittenPerCall = 0 - var sendComplete: Boolean = false - do { - val written = current.head.writeTo(channel) - totalWritten += written - totalWrittenPerCall += written - sendComplete = current.head.complete - if(sendComplete) - current = current.tail - } while (!complete && sendComplete) - trace("Bytes written as part of multisend call : " + totalWrittenPerCall + "Total bytes written so far : " + totalWritten + "Expected bytes to write : " + expectedBytesToWrite) - totalWrittenPerCall - } - - def complete: Boolean = { - if (current == Nil) { - if (totalWritten != expectedBytesToWrite) - error("mismatch in sending bytes over socket; expected: " + expectedBytesToWrite + " actual: " + totalWritten) - true - } else { - false - } - } -} diff --git a/core/src/main/scala/kafka/producer/BaseProducer.scala b/core/src/main/scala/kafka/producer/BaseProducer.scala new file mode 100644 index 0000000000000..8e007130b2825 --- /dev/null +++ b/core/src/main/scala/kafka/producer/BaseProducer.scala @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.producer + +import java.util.Properties + +// A base producer used whenever we need to have options for both old and new producers; +// this class will be removed once we fully rolled out 0.9 +trait BaseProducer { + def send(topic: String, key: Array[Byte], value: Array[Byte]) + def close() +} + +class NewShinyProducer(producerProps: Properties) extends BaseProducer { + import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} + import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback + + // decide whether to send synchronously based on producer properties + val sync = producerProps.getProperty("producer.type", "async").equals("sync") + + val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + + override def send(topic: String, key: Array[Byte], value: Array[Byte]) { + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic, key, value) + if(sync) { + this.producer.send(record).get() + } else { + this.producer.send(record, + new ErrorLoggingCallback(topic, key, value, false)) + } + } + + override def close() { + this.producer.close() + } +} + +class OldProducer(producerProps: Properties) extends BaseProducer { + import kafka.producer.{KeyedMessage, ProducerConfig} + + // default to byte array partitioner + if (producerProps.getProperty("partitioner.class") == null) + producerProps.setProperty("partitioner.class", classOf[kafka.producer.ByteArrayPartitioner].getName) + val producer = new kafka.producer.Producer[Array[Byte], Array[Byte]](new ProducerConfig(producerProps)) + + override def send(topic: String, key: Array[Byte], value: Array[Byte]) { + this.producer.send(new KeyedMessage[Array[Byte], Array[Byte]](topic, key, value)) + } + + override def close() { + this.producer.close() + } +} + diff --git a/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala b/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala old mode 100644 new mode 100755 index 988e4374d8c9b..e6b100eb0f5de --- a/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala +++ b/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala @@ -19,8 +19,9 @@ package kafka.producer import kafka.utils._ +import org.apache.kafka.common.utils.Utils -private class ByteArrayPartitioner(props: VerifiableProperties = null) extends Partitioner { +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 deleted file mode 100644 index 28de57304db6a..0000000000000 --- a/core/src/main/scala/kafka/producer/ConsoleProducer.scala +++ /dev/null @@ -1,238 +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.producer - -import scala.collection.JavaConversions._ -import joptsimple._ -import java.util.Properties -import java.io._ -import kafka.common._ -import kafka.message._ -import kafka.serializer._ - -object ConsoleProducer { - - def main(args: Array[String]) { - val parser = new OptionParser - val topicOpt = parser.accepts("topic", "REQUIRED: The topic id to produce messages to.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.") - .withRequiredArg - .describedAs("broker-list") - .ofType(classOf[String]) - val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") - val compressOpt = parser.accepts("compress", "If set, messages batches are sent compressed") - val batchSizeOpt = parser.accepts("batch-size", "Number of messages to send in a single batch if they are not being sent synchronously.") - .withRequiredArg - .describedAs("size") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(200) - val messageSendMaxRetriesOpt = parser.accepts("message-send-max-retries", "Brokers can fail receiving the message for multiple reasons, and being unavailable transiently is just one of them. This property specifies the number of retires before the producer give up and drop this message.") - .withRequiredArg - .ofType(classOf[java.lang.Integer]) - .defaultsTo(3) - val retryBackoffMsOpt = parser.accepts("retry-backoff-ms", "Before each retry, the producer refreshes the metadata of relevant topics. Since leader election takes a bit of time, this property specifies the amount of time that the producer waits before refreshing the metadata.") - .withRequiredArg - .ofType(classOf[java.lang.Long]) - .defaultsTo(100) - val sendTimeoutOpt = parser.accepts("timeout", "If set and the producer is running in asynchronous mode, this gives the maximum amount of time" + - " a message will queue awaiting suffient batch size. The value is given in ms.") - .withRequiredArg - .describedAs("timeout_ms") - .ofType(classOf[java.lang.Long]) - .defaultsTo(1000) - val queueSizeOpt = parser.accepts("queue-size", "If set and the producer is running in asynchronous mode, this gives the maximum amount of " + - " messages will queue awaiting suffient batch size.") - .withRequiredArg - .describedAs("queue_size") - .ofType(classOf[java.lang.Long]) - .defaultsTo(10000) - val queueEnqueueTimeoutMsOpt = parser.accepts("queue-enqueuetimeout-ms", "Timeout for event enqueue") - .withRequiredArg - .describedAs("queue enqueuetimeout ms") - .ofType(classOf[java.lang.Long]) - .defaultsTo(Int.MaxValue) - val requestRequiredAcksOpt = parser.accepts("request-required-acks", "The required acks of the producer requests") - .withRequiredArg - .describedAs("request required acks") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) - val requestTimeoutMsOpt = parser.accepts("request-timeout-ms", "The ack timeout of the producer requests. Value must be non-negative and non-zero") - .withRequiredArg - .describedAs("request timeout ms") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1500) - val valueEncoderOpt = parser.accepts("value-serializer", "The class name of the message encoder implementation to use for serializing values.") - .withRequiredArg - .describedAs("encoder_class") - .ofType(classOf[java.lang.String]) - .defaultsTo(classOf[StringEncoder].getName) - val keyEncoderOpt = parser.accepts("key-serializer", "The class name of the message encoder implementation to use for serializing keys.") - .withRequiredArg - .describedAs("encoder_class") - .ofType(classOf[java.lang.String]) - .defaultsTo(classOf[StringEncoder].getName) - val messageReaderOpt = parser.accepts("line-reader", "The class name of the class to use for reading lines from standard in. " + - "By default each line is read as a separate message.") - .withRequiredArg - .describedAs("reader_class") - .ofType(classOf[java.lang.String]) - .defaultsTo(classOf[LineMessageReader].getName) - val socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.") - .withRequiredArg - .describedAs("size") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1024*100) - val propertyOpt = parser.accepts("property", "A mechanism to pass user-defined properties in the form key=value to the message reader. " + - "This allows custom configuration for a user-defined message reader.") - .withRequiredArg - .describedAs("prop") - .ofType(classOf[String]) - - - val options = parser.parse(args : _*) - for(arg <- List(topicOpt, brokerListOpt)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } - - val topic = options.valueOf(topicOpt) - val brokerList = options.valueOf(brokerListOpt) - val sync = options.has(syncOpt) - val compress = options.has(compressOpt) - val batchSize = options.valueOf(batchSizeOpt) - val sendTimeout = options.valueOf(sendTimeoutOpt) - val queueSize = options.valueOf(queueSizeOpt) - val queueEnqueueTimeoutMs = options.valueOf(queueEnqueueTimeoutMsOpt) - val requestRequiredAcks = options.valueOf(requestRequiredAcksOpt) - val requestTimeoutMs = options.valueOf(requestTimeoutMsOpt) - val keyEncoderClass = options.valueOf(keyEncoderOpt) - val valueEncoderClass = options.valueOf(valueEncoderOpt) - val readerClass = options.valueOf(messageReaderOpt) - val socketBuffer = options.valueOf(socketBufferSizeOpt) - val cmdLineProps = parseLineReaderArgs(options.valuesOf(propertyOpt)) - cmdLineProps.put("topic", topic) - - val props = new Properties() - props.put("metadata.broker.list", brokerList) - val codec = if(compress) DefaultCompressionCodec.codec else NoCompressionCodec.codec - props.put("compression.codec", codec.toString) - props.put("producer.type", if(sync) "sync" else "async") - if(options.has(batchSizeOpt)) - props.put("batch.num.messages", batchSize.toString) - - props.put("message.send.max.retries", options.valueOf(messageSendMaxRetriesOpt).toString) - props.put("retry.backoff.ms", options.valueOf(retryBackoffMsOpt).toString) - props.put("queue.buffering.max.ms", sendTimeout.toString) - props.put("queue.buffering.max.messages", queueSize.toString) - props.put("queue.enqueue.timeout.ms", queueEnqueueTimeoutMs.toString) - props.put("request.required.acks", requestRequiredAcks.toString) - props.put("request.timeout.ms", requestTimeoutMs.toString) - props.put("key.serializer.class", keyEncoderClass) - props.put("serializer.class", valueEncoderClass) - props.put("send.buffer.bytes", socketBuffer.toString) - val reader = Class.forName(readerClass).newInstance().asInstanceOf[MessageReader[AnyRef, AnyRef]] - reader.init(System.in, cmdLineProps) - - try { - val producer = new Producer[AnyRef, AnyRef](new ProducerConfig(props)) - - Runtime.getRuntime.addShutdownHook(new Thread() { - override def run() { - producer.close() - } - }) - - var message: KeyedMessage[AnyRef, AnyRef] = null - do { - message = reader.readMessage() - if(message != null) - producer.send(message) - } while(message != null) - } catch { - case e: Exception => - e.printStackTrace - System.exit(1) - } - System.exit(0) - } - - def parseLineReaderArgs(args: Iterable[String]): Properties = { - val splits = args.map(_ split "=").filterNot(_ == null).filterNot(_.length == 0) - if(!splits.forall(_.length == 2)) { - System.err.println("Invalid line reader properties: " + args.mkString(" ")) - System.exit(1) - } - val props = new Properties - for(a <- splits) - props.put(a(0), a(1)) - props - } - - trait MessageReader[K,V] { - def init(inputStream: InputStream, props: Properties) {} - def readMessage(): KeyedMessage[K,V] - def close() {} - } - - class LineMessageReader extends MessageReader[String, String] { - var topic: String = null - var reader: BufferedReader = null - var parseKey = false - var keySeparator = "\t" - var ignoreError = false - var lineNumber = 0 - - override def init(inputStream: InputStream, props: Properties) { - topic = props.getProperty("topic") - if(props.containsKey("parse.key")) - parseKey = props.getProperty("parse.key").trim.toLowerCase.equals("true") - if(props.containsKey("key.separator")) - keySeparator = props.getProperty("key.separator") - if(props.containsKey("ignore.error")) - ignoreError = props.getProperty("ignore.error").trim.toLowerCase.equals("true") - reader = new BufferedReader(new InputStreamReader(inputStream)) - } - - override def readMessage() = { - lineNumber += 1 - (reader.readLine(), parseKey) match { - case (null, _) => null - case (line, true) => - line.indexOf(keySeparator) match { - case -1 => - if(ignoreError) - new KeyedMessage(topic, line) - else - throw new KafkaException("No key found on line " + lineNumber + ": " + line) - case n => - new KeyedMessage(topic, - line.substring(0, n), - if(n + keySeparator.size > line.size) "" else line.substring(n + keySeparator.size)) - } - case (line, false) => - new KeyedMessage(topic, line) - } - } - } -} diff --git a/core/src/main/scala/kafka/producer/DefaultPartitioner.scala b/core/src/main/scala/kafka/producer/DefaultPartitioner.scala old mode 100644 new mode 100755 index 3afb22eeb4e3b..1141ed16769b8 --- a/core/src/main/scala/kafka/producer/DefaultPartitioner.scala +++ b/core/src/main/scala/kafka/producer/DefaultPartitioner.scala @@ -19,6 +19,7 @@ package kafka.producer import kafka.utils._ +import org.apache.kafka.common.utils.Utils class DefaultPartitioner(props: VerifiableProperties = null) extends Partitioner { private val random = new java.util.Random diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala deleted file mode 100644 index 42239b27ab267..0000000000000 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ /dev/null @@ -1,111 +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.producer - -import async.MissingConfigException -import org.apache.log4j.spi.LoggingEvent -import org.apache.log4j.AppenderSkeleton -import org.apache.log4j.helpers.LogLog -import kafka.utils.Logging -import java.util.{Properties, Date} - -class KafkaLog4jAppender extends AppenderSkeleton with Logging { - var topic:String = null - var serializerClass:String = null - var brokerList:String = null - var producerType:String = null - var compressionCodec:String = null - var enqueueTimeout:String = null - var queueSize:String = null - var requiredNumAcks: Int = Int.MaxValue - - private var producer: Producer[String, String] = null - - def getTopic:String = topic - def setTopic(topic: String) { this.topic = topic } - - def getBrokerList:String = brokerList - def setBrokerList(brokerList: String) { this.brokerList = brokerList } - - def getSerializerClass:String = serializerClass - def setSerializerClass(serializerClass:String) { this.serializerClass = serializerClass } - - def getProducerType:String = producerType - def setProducerType(producerType:String) { this.producerType = producerType } - - def getCompressionCodec:String = compressionCodec - def setCompressionCodec(compressionCodec:String) { this.compressionCodec = compressionCodec } - - def getEnqueueTimeout:String = enqueueTimeout - def setEnqueueTimeout(enqueueTimeout:String) { this.enqueueTimeout = enqueueTimeout } - - def getQueueSize:String = queueSize - def setQueueSize(queueSize:String) { this.queueSize = queueSize } - - def getRequiredNumAcks:Int = requiredNumAcks - def setRequiredNumAcks(requiredNumAcks:Int) { this.requiredNumAcks = requiredNumAcks } - - override def activateOptions() { - // check for config parameter validity - val props = new Properties() - if(brokerList != null) - props.put("metadata.broker.list", brokerList) - if(props.isEmpty) - throw new MissingConfigException("The metadata.broker.list property should be specified") - if(topic == null) - throw new MissingConfigException("topic must be specified by the Kafka log4j appender") - if(serializerClass == null) { - serializerClass = "kafka.serializer.StringEncoder" - LogLog.debug("Using default encoder - kafka.serializer.StringEncoder") - } - props.put("serializer.class", serializerClass) - //These have default values in ProducerConfig and AsyncProducerConfig. We don't care if they're not specified - if(producerType != null) props.put("producer.type", producerType) - if(compressionCodec != null) props.put("compression.codec", compressionCodec) - if(enqueueTimeout != null) props.put("queue.enqueue.timeout.ms", enqueueTimeout) - if(queueSize != null) props.put("queue.buffering.max.messages", queueSize) - if(requiredNumAcks != Int.MaxValue) props.put("request.required.acks", requiredNumAcks.toString) - val config : ProducerConfig = new ProducerConfig(props) - producer = new Producer[String, String](config) - LogLog.debug("Kafka producer connected to " + config.brokerList) - LogLog.debug("Logging for topic: " + topic) - } - - override def append(event: LoggingEvent) { - 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 - producer.close() - } - } - - override def requiresLayout: Boolean = false -} diff --git a/core/src/main/scala/kafka/producer/KeyedMessage.scala b/core/src/main/scala/kafka/producer/KeyedMessage.scala index 388bc9bbd9a69..dbcf29515bb86 100644 --- a/core/src/main/scala/kafka/producer/KeyedMessage.scala +++ b/core/src/main/scala/kafka/producer/KeyedMessage.scala @@ -21,7 +21,7 @@ package kafka.producer * 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 partKey: Any, val message: V) { +case class KeyedMessage[K, V](topic: String, key: K, partKey: Any, message: V) { if(topic == null) throw new IllegalArgumentException("Topic cannot be null.") @@ -39,4 +39,4 @@ case class KeyedMessage[K, V](val topic: String, val key: K, val partKey: Any, v } def hasKey = key != null -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/producer/Producer.scala b/core/src/main/scala/kafka/producer/Producer.scala old mode 100644 new mode 100755 index 4798481d573bb..4be06c8d3fc7f --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -16,14 +16,14 @@ */ package kafka.producer -import async.{DefaultEventHandler, ProducerSendThread, EventHandler} -import kafka.utils._ -import java.util.Random -import java.util.concurrent.{TimeUnit, LinkedBlockingQueue} -import kafka.serializer.Encoder import java.util.concurrent.atomic.AtomicBoolean -import kafka.common.QueueFullException +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} + +import kafka.common.{AppInfo, QueueFullException} import kafka.metrics._ +import kafka.producer.async.{DefaultEventHandler, EventHandler, ProducerSendThread} +import kafka.serializer.Encoder +import kafka.utils._ class Producer[K,V](val config: ProducerConfig, @@ -53,13 +53,14 @@ class Producer[K,V](val config: ProducerConfig, private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(config.clientId) KafkaMetricsReporter.startReporters(config.props) + AppInfo.registerInfo() def this(config: ProducerConfig) = this(config, new DefaultEventHandler[K,V](config, - Utils.createObject[Partitioner](config.partitionerClass, config.props), - Utils.createObject[Encoder[V]](config.serializerClass, config.props), - Utils.createObject[Encoder[K]](config.keySerializerClass, config.props), + CoreUtils.createObject[Partitioner](config.partitionerClass, config.props), + CoreUtils.createObject[Encoder[V]](config.serializerClass, config.props), + CoreUtils.createObject[Encoder[K]](config.keySerializerClass, config.props), new ProducerPool(config))) /** @@ -126,9 +127,12 @@ class Producer[K,V](val config: ProducerConfig, val canShutdown = hasShutdown.compareAndSet(false, true) if(canShutdown) { info("Shutting down producer") + val startTime = System.nanoTime() + KafkaMetricsGroup.removeAllProducerMetrics(config.clientId) if (producerSendThread != null) producerSendThread.shutdown eventHandler.close + info("Producer shutdown completed in " + (System.nanoTime() - startTime) / 1000000 + " ms") } } } diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala old mode 100644 new mode 100755 index 7947b18aceb29..08a4e5146678d --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -19,8 +19,8 @@ package kafka.producer import async.AsyncProducerConfig import java.util.Properties -import kafka.utils.{Utils, VerifiableProperties} -import kafka.message.{CompressionCodec, NoCompressionCodec} +import kafka.utils.{CoreUtils, VerifiableProperties} +import kafka.message.NoCompressionCodec import kafka.common.{InvalidConfigException, Config} object ProducerConfig extends Config { @@ -77,16 +77,7 @@ class ProducerConfig private (val props: VerifiableProperties) * This parameter allows you to specify the compression codec for all data generated * * by this producer. The default is NoCompressionCodec */ - val compressionCodec = { - val prop = props.getString("compression.codec", NoCompressionCodec.name) - try { - CompressionCodec.getCompressionCodec(prop.toInt) - } - catch { - case nfe: NumberFormatException => - CompressionCodec.getCompressionCodec(prop) - } - } + val compressionCodec = props.getCompressionCodec("compression.codec", NoCompressionCodec) /** This parameter allows you to set whether compression should be turned * * on for particular topics @@ -99,7 +90,7 @@ class ProducerConfig private (val props: VerifiableProperties) * * If the compression codec is NoCompressionCodec, compression is disabled for all topics */ - val compressedTopics = Utils.parseCsvList(props.getString("compressed.topics", null)) + val compressedTopics = CoreUtils.parseCsvList(props.getString("compressed.topics", null)) /** The leader may be unavailable transiently, which can fail the sending of a message. * This property specifies the number of retries when such failures occur. diff --git a/core/src/main/scala/kafka/producer/ProducerPool.scala b/core/src/main/scala/kafka/producer/ProducerPool.scala index 43df70bb461dd..5ad68129403dd 100644 --- a/core/src/main/scala/kafka/producer/ProducerPool.scala +++ b/core/src/main/scala/kafka/producer/ProducerPool.scala @@ -17,20 +17,21 @@ package kafka.producer -import kafka.cluster.Broker import java.util.Properties -import collection.mutable.HashMap -import java.lang.Object -import kafka.utils.Logging + import kafka.api.TopicMetadata +import kafka.cluster.BrokerEndPoint import kafka.common.UnavailableProducerException +import kafka.utils.Logging + +import scala.collection.mutable.HashMap object ProducerPool { /** * Used in ProducerPool to initiate a SyncProducer connection with a broker. */ - def createSyncProducer(config: ProducerConfig, broker: Broker): SyncProducer = { + def createSyncProducer(config: ProducerConfig, broker: BrokerEndPoint): SyncProducer = { val props = new Properties() props.put("host", broker.host) props.put("port", broker.port.toString) @@ -44,11 +45,12 @@ class ProducerPool(val config: ProducerConfig) extends Logging { private val lock = new Object() def updateProducer(topicMetadata: Seq[TopicMetadata]) { - val newBrokers = new collection.mutable.HashSet[Broker] + val newBrokers = new collection.mutable.HashSet[BrokerEndPoint] topicMetadata.foreach(tmd => { tmd.partitionsMetadata.foreach(pmd => { - if(pmd.leader.isDefined) - newBrokers+=(pmd.leader.get) + if(pmd.leader.isDefined) { + newBrokers += pmd.leader.get + } }) }) lock synchronized { diff --git a/core/src/main/scala/kafka/producer/ProducerRequestStats.scala b/core/src/main/scala/kafka/producer/ProducerRequestStats.scala index 96942205a6a46..026e93a2f1dcc 100644 --- a/core/src/main/scala/kafka/producer/ProducerRequestStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerRequestStats.scala @@ -19,11 +19,16 @@ package kafka.producer import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} import java.util.concurrent.TimeUnit import kafka.utils.Pool -import kafka.common.ClientIdAndBroker +import kafka.common.{ClientIdAllBrokers, ClientIdBroker, ClientIdAndBroker} -class ProducerRequestMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGroup { - val requestTimer = new KafkaTimer(newTimer(metricId + "ProducerRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) - val requestSizeHist = newHistogram(metricId + "ProducerRequestSize") +class ProducerRequestMetrics(metricId: ClientIdBroker) extends KafkaMetricsGroup { + val tags = metricId match { + case ClientIdAndBroker(clientId, brokerHost, brokerPort) => Map("clientId" -> clientId, "brokerHost" -> brokerHost, "brokerPort" -> brokerPort.toString) + case ClientIdAllBrokers(clientId) => Map("clientId" -> clientId) + } + + val requestTimer = new KafkaTimer(newTimer("ProducerRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, tags)) + val requestSizeHist = newHistogram("ProducerRequestSize", biased = true, tags) } /** @@ -31,14 +36,14 @@ class ProducerRequestMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGr * @param clientId ClientId of the given producer */ class ProducerRequestStats(clientId: String) { - private val valueFactory = (k: ClientIdAndBroker) => new ProducerRequestMetrics(k) - private val stats = new Pool[ClientIdAndBroker, ProducerRequestMetrics](Some(valueFactory)) - private val allBrokersStats = new ProducerRequestMetrics(new ClientIdAndBroker(clientId, "AllBrokers")) + private val valueFactory = (k: ClientIdBroker) => new ProducerRequestMetrics(k) + private val stats = new Pool[ClientIdBroker, ProducerRequestMetrics](Some(valueFactory)) + private val allBrokersStats = new ProducerRequestMetrics(new ClientIdAllBrokers(clientId)) def getProducerRequestAllBrokersStats(): ProducerRequestMetrics = allBrokersStats - def getProducerRequestStats(brokerInfo: String): ProducerRequestMetrics = { - stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo + "-")) + def getProducerRequestStats(brokerHost: String, brokerPort: Int): ProducerRequestMetrics = { + stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerHost, brokerPort)) } } @@ -52,5 +57,9 @@ object ProducerRequestStatsRegistry { def getProducerRequestStats(clientId: String) = { globalStats.getAndMaybePut(clientId) } + + def removeProducerRequestStats(clientId: String) { + globalStats.remove(clientId) + } } diff --git a/core/src/main/scala/kafka/producer/ProducerStats.scala b/core/src/main/scala/kafka/producer/ProducerStats.scala index e1610d3c602fb..1d0fa888c99a5 100644 --- a/core/src/main/scala/kafka/producer/ProducerStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerStats.scala @@ -21,9 +21,10 @@ import java.util.concurrent.TimeUnit import kafka.utils.Pool class ProducerStats(clientId: String) extends KafkaMetricsGroup { - val serializationErrorRate = newMeter(clientId + "-SerializationErrorsPerSec", "errors", TimeUnit.SECONDS) - val resendRate = newMeter(clientId + "-ResendsPerSec", "resends", TimeUnit.SECONDS) - val failedSendRate = newMeter(clientId + "-FailedSendsPerSec", "failed sends", TimeUnit.SECONDS) + val tags: Map[String, String] = Map("clientId" -> clientId) + val serializationErrorRate = newMeter("SerializationErrorsPerSec", "errors", TimeUnit.SECONDS, tags) + val resendRate = newMeter("ResendsPerSec", "resends", TimeUnit.SECONDS, tags) + val failedSendRate = newMeter("FailedSendsPerSec", "failed sends", TimeUnit.SECONDS, tags) } /** @@ -36,4 +37,8 @@ object ProducerStatsRegistry { def getProducerStats(clientId: String) = { statsRegistry.getAndMaybePut(clientId) } + + def removeProducerStats(clientId: String) { + statsRegistry.remove(clientId) + } } diff --git a/core/src/main/scala/kafka/producer/ProducerTopicStats.scala b/core/src/main/scala/kafka/producer/ProducerTopicStats.scala index ed209f4773ded..97594c8313672 100644 --- a/core/src/main/scala/kafka/producer/ProducerTopicStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerTopicStats.scala @@ -17,16 +17,21 @@ package kafka.producer import kafka.metrics.KafkaMetricsGroup -import kafka.common.ClientIdAndTopic +import kafka.common.{ClientIdTopic, ClientIdAllTopics, ClientIdAndTopic} import kafka.utils.{Pool, threadsafe} import java.util.concurrent.TimeUnit @threadsafe -class ProducerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup { - val messageRate = newMeter(metricId + "MessagesPerSec", "messages", TimeUnit.SECONDS) - val byteRate = newMeter(metricId + "BytesPerSec", "bytes", TimeUnit.SECONDS) - val droppedMessageRate = newMeter(metricId + "DroppedMessagesPerSec", "drops", TimeUnit.SECONDS) +class ProducerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup { + val tags = metricId match { + case ClientIdAndTopic(clientId, topic) => Map("clientId" -> clientId, "topic" -> topic) + case ClientIdAllTopics(clientId) => Map("clientId" -> clientId) + } + + val messageRate = newMeter("MessagesPerSec", "messages", TimeUnit.SECONDS, tags) + val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, tags) + val droppedMessageRate = newMeter("DroppedMessagesPerSec", "drops", TimeUnit.SECONDS, tags) } /** @@ -34,14 +39,14 @@ class ProducerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup * @param clientId The clientId of the given producer client. */ class ProducerTopicStats(clientId: String) { - private val valueFactory = (k: ClientIdAndTopic) => new ProducerTopicMetrics(k) - private val stats = new Pool[ClientIdAndTopic, ProducerTopicMetrics](Some(valueFactory)) - private val allTopicsStats = new ProducerTopicMetrics(new ClientIdAndTopic(clientId, "AllTopics")) // to differentiate from a topic named AllTopics + private val valueFactory = (k: ClientIdTopic) => new ProducerTopicMetrics(k) + private val stats = new Pool[ClientIdTopic, ProducerTopicMetrics](Some(valueFactory)) + private val allTopicsStats = new ProducerTopicMetrics(new ClientIdAllTopics(clientId)) // to differentiate from a topic named AllTopics def getProducerAllTopicsStats(): ProducerTopicMetrics = allTopicsStats def getProducerTopicStats(topic: String): ProducerTopicMetrics = { - stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic + "-")) + stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic)) } } @@ -55,4 +60,8 @@ object ProducerTopicStatsRegistry { def getProducerTopicStats(clientId: String) = { globalStats.getAndMaybePut(clientId) } + + def removeProducerTopicStats(clientId: String) { + globalStats.remove(clientId) + } } diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index 041cfa59c18fa..dcee50113b1b1 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.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 @@ -17,10 +17,13 @@ package kafka.producer +import java.util.Random + import kafka.api._ -import kafka.network.{BlockingChannel, BoundedByteBufferSend, Receive} +import kafka.network.{RequestOrResponseSend, BlockingChannel} import kafka.utils._ -import java.util.Random +import org.apache.kafka.common.network.NetworkReceive +import org.apache.kafka.common.utils.Utils._ object SyncProducer { val RequestKey: Short = 0 @@ -37,10 +40,9 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { @volatile private var shutdown: Boolean = false private val blockingChannel = new BlockingChannel(config.host, config.port, BlockingChannel.UseDefaultBufferSize, config.sendBufferBytes, config.requestTimeoutMs) - val brokerInfo = "host_%s-port_%s".format(config.host, config.port) val producerRequestStats = ProducerRequestStatsRegistry.getProducerRequestStats(config.clientId) - trace("Instantiating Scala Sync Producer") + trace("Instantiating Scala Sync Producer with properties: %s".format(config.props)) private def verifyRequest(request: RequestOrResponse) = { /** @@ -49,7 +51,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { * data. So, leaving the rest of the logging at TRACE, while errors should be logged at ERROR level */ if (logger.isDebugEnabled) { - val buffer = new BoundedByteBufferSend(request).buffer + val buffer = new RequestOrResponseSend("", request).buffer trace("verifying sendbuffer of size " + buffer.limit) val requestTypeId = buffer.getShort() if(requestTypeId == RequestKeys.ProduceKey) { @@ -62,12 +64,12 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { /** * Common functionality for the public send methods */ - private def doSend(request: RequestOrResponse, readResponse: Boolean = true): Receive = { + private def doSend(request: RequestOrResponse, readResponse: Boolean = true): NetworkReceive = { lock synchronized { verifyRequest(request) getOrMakeConnection() - var response: Receive = null + var response: NetworkReceive = null try { blockingChannel.send(request) if(readResponse) @@ -91,11 +93,11 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { */ def send(producerRequest: ProducerRequest): ProducerResponse = { val requestSize = producerRequest.sizeInBytes - producerRequestStats.getProducerRequestStats(brokerInfo).requestSizeHist.update(requestSize) + producerRequestStats.getProducerRequestStats(config.host, config.port).requestSizeHist.update(requestSize) producerRequestStats.getProducerRequestAllBrokersStats.requestSizeHist.update(requestSize) - var response: Receive = null - val specificTimer = producerRequestStats.getProducerRequestStats(brokerInfo).requestTimer + var response: NetworkReceive = null + val specificTimer = producerRequestStats.getProducerRequestStats(config.host, config.port).requestTimer val aggregateTimer = producerRequestStats.getProducerRequestAllBrokersStats.requestTimer aggregateTimer.time { specificTimer.time { @@ -103,14 +105,14 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { } } if(producerRequest.requiredAcks != 0) - ProducerResponse.readFrom(response.buffer) + ProducerResponse.readFrom(response.payload) else null } def send(request: TopicMetadataRequest): TopicMetadataResponse = { val response = doSend(request) - TopicMetadataResponse.readFrom(response.buffer) + TopicMetadataResponse.readFrom(response.payload) } def close() = { @@ -126,24 +128,22 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { */ private def disconnect() { try { - if(blockingChannel.isConnected) { - info("Disconnecting from " + config.host + ":" + config.port) - blockingChannel.disconnect() - } + info("Disconnecting from " + formatAddress(config.host, config.port)) + blockingChannel.disconnect() } catch { case e: Exception => error("Error on disconnect: ", e) } } - + private def connect(): BlockingChannel = { if (!blockingChannel.isConnected && !shutdown) { try { blockingChannel.connect() - info("Connected to " + config.host + ":" + config.port + " for producing") + info("Connected to " + formatAddress(config.host, config.port) + " for producing") } catch { case e: Exception => { disconnect() - error("Producer connection to " + config.host + ":" + config.port + " unsuccessful", e) + error("Producer connection to " + formatAddress(config.host, config.port) + " unsuccessful", e) throw e } } @@ -156,5 +156,4 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { connect() } } -} - +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala index 69b2d0c11bb14..a08ce00a0aae7 100644 --- a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala @@ -42,11 +42,15 @@ trait SyncProducerConfigShared { val clientId = props.getString("client.id", SyncProducerConfig.DefaultClientId) /* - * The required acks of the producer requests - negative value means ack - * after the replicas in ISR have caught up to the leader's offset - * corresponding to this produce request. + * The number of acknowledgments the producer requires the leader to have received before considering a request complete. + * This controls the durability of the messages sent by the producer. + * + * request.required.acks = 0 - means the producer will not wait for any acknowledgement from the leader. + * request.required.acks = 1 - means the leader will write the message to its local log and immediately acknowledge + * request.required.acks = -1 - means the leader will wait for acknowledgement from all in-sync replicas before acknowledging the write */ - val requestRequiredAcks = props.getShort("request.required.acks", SyncProducerConfig.DefaultRequiredAcks) + + val requestRequiredAcks = props.getShortInRange("request.required.acks", SyncProducerConfig.DefaultRequiredAcks,(-1,1)) /* * The ack timeout of the producer requests. Value must be non-negative and non-zero @@ -59,4 +63,4 @@ object SyncProducerConfig { val DefaultClientId = "" val DefaultRequiredAcks : Short = 0 val DefaultAckTimeoutMs = 10000 -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala old mode 100644 new mode 100755 index d8ac915de31a2..a6179a902f7fa --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -21,12 +21,13 @@ import kafka.common._ import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} import kafka.producer._ import kafka.serializer.Encoder -import kafka.utils.{Utils, Logging, SystemTime} +import kafka.utils.{CoreUtils, Logging, SystemTime} import scala.util.Random import scala.collection.{Seq, Map} import scala.collection.mutable.{ArrayBuffer, HashMap, Set} import java.util.concurrent.atomic._ import kafka.api.{TopicMetadata, ProducerRequest} +import org.apache.kafka.common.utils.Utils class DefaultEventHandler[K,V](config: ProducerConfig, private val partitioner: Partitioner, @@ -64,7 +65,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, topicMetadataToRefresh ++= outstandingProduceRequests.map(_.topic) if (topicMetadataRefreshInterval >= 0 && SystemTime.milliseconds - lastTopicMetadataRefreshTime > topicMetadataRefreshInterval) { - Utils.swallowError(brokerPartitionInfo.updateInfo(topicMetadataToRefresh.toSet, correlationId.getAndIncrement)) + CoreUtils.swallowError(brokerPartitionInfo.updateInfo(topicMetadataToRefresh.toSet, correlationId.getAndIncrement)) sendPartitionPerTopicCache.clear() topicMetadataToRefresh.clear lastTopicMetadataRefreshTime = SystemTime.milliseconds @@ -75,7 +76,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig, // back off and update the topic metadata cache before attempting another send operation Thread.sleep(config.retryBackoffMs) // get topics of the outstanding produce requests and refresh metadata for those - Utils.swallowError(brokerPartitionInfo.updateInfo(outstandingProduceRequests.map(_.topic).toSet, correlationId.getAndIncrement)) + CoreUtils.swallowError(brokerPartitionInfo.updateInfo(outstandingProduceRequests.map(_.topic).toSet, correlationId.getAndIncrement)) sendPartitionPerTopicCache.clear() remainingRetries -= 1 producerStats.resendRate.mark() @@ -95,34 +96,35 @@ class DefaultEventHandler[K,V](config: ProducerConfig, val partitionedDataOpt = partitionAndCollate(messages) partitionedDataOpt match { case Some(partitionedData) => - val failedProduceRequests = new ArrayBuffer[KeyedMessage[K,Message]] - try { - for ((brokerid, messagesPerBrokerMap) <- partitionedData) { - if (logger.isTraceEnabled) - messagesPerBrokerMap.foreach(partitionAndEvent => - trace("Handling event for Topic: %s, Broker: %d, Partitions: %s".format(partitionAndEvent._1, brokerid, partitionAndEvent._2))) - val messageSetPerBroker = groupMessagesToSet(messagesPerBrokerMap) - - val failedTopicPartitions = send(brokerid, messageSetPerBroker) - failedTopicPartitions.foreach(topicPartition => { - messagesPerBrokerMap.get(topicPartition) match { - case Some(data) => failedProduceRequests.appendAll(data) - case None => // nothing - } - }) + val failedProduceRequests = new ArrayBuffer[KeyedMessage[K, Message]] + for ((brokerid, messagesPerBrokerMap) <- partitionedData) { + if (logger.isTraceEnabled) { + messagesPerBrokerMap.foreach(partitionAndEvent => + trace("Handling event for Topic: %s, Broker: %d, Partitions: %s".format(partitionAndEvent._1, brokerid, partitionAndEvent._2))) + } + val messageSetPerBrokerOpt = groupMessagesToSet(messagesPerBrokerMap) + messageSetPerBrokerOpt match { + case Some(messageSetPerBroker) => + val failedTopicPartitions = send(brokerid, messageSetPerBroker) + failedTopicPartitions.foreach(topicPartition => { + messagesPerBrokerMap.get(topicPartition) match { + case Some(data) => failedProduceRequests.appendAll(data) + case None => // nothing + } + }) + case None => // failed to group messages + messagesPerBrokerMap.values.foreach(m => failedProduceRequests.appendAll(m)) } - } catch { - case t: Throwable => error("Failed to send messages", t) } failedProduceRequests - case None => // all produce requests failed + case None => // failed to collate messages messages } } def serialize(events: Seq[KeyedMessage[K,V]]): Seq[KeyedMessage[K,Message]] = { val serializedMessages = new ArrayBuffer[KeyedMessage[K,Message]](events.size) - events.map{e => + events.foreach{e => try { if(e.hasKey) 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))) @@ -261,7 +263,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(if(message.message.isNull) null else Utils.readString(message.message.payload))))) + trace("Successfully sent message: %s".format(if(message.message.isNull) null else message.message.toString())))) } val failedPartitionsAndStatus = response.status.filter(_._2.error != ErrorMapping.NoError).toSeq failedTopicPartitions = failedPartitionsAndStatus.map(partitionStatus => partitionStatus._1) @@ -290,43 +292,46 @@ class DefaultEventHandler[K,V](config: ProducerConfig, } } - private def groupMessagesToSet(messagesPerTopicAndPartition: collection.mutable.Map[TopicAndPartition, Seq[KeyedMessage[K,Message]]]) = { + private def groupMessagesToSet(messagesPerTopicAndPartition: collection.mutable.Map[TopicAndPartition, Seq[KeyedMessage[K, Message]]]) = { /** enforce the compressed.topics config here. - * If the compression codec is anything other than NoCompressionCodec, - * Enable compression only for specified topics if any - * If the list of compressed topics is empty, then enable the specified compression codec for all topics - * If the compression codec is NoCompressionCodec, compression is disabled for all topics + * If the compression codec is anything other than NoCompressionCodec, + * Enable compression only for specified topics if any + * If the list of compressed topics is empty, then enable the specified compression codec for all topics + * If the compression codec is NoCompressionCodec, compression is disabled for all topics */ - - val messagesPerTopicPartition = messagesPerTopicAndPartition.map { case (topicAndPartition, messages) => - val rawMessages = messages.map(_.message) - ( topicAndPartition, - config.compressionCodec match { - case NoCompressionCodec => - debug("Sending %d messages with no compression to %s".format(messages.size, topicAndPartition)) - new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*) - case _ => - config.compressedTopics.size match { - case 0 => - debug("Sending %d messages with compression codec %d to %s" - .format(messages.size, config.compressionCodec.codec, topicAndPartition)) - new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*) - case _ => - if(config.compressedTopics.contains(topicAndPartition.topic)) { + try { + val messagesPerTopicPartition = messagesPerTopicAndPartition.map { case (topicAndPartition, messages) => + val rawMessages = messages.map(_.message) + (topicAndPartition, + config.compressionCodec match { + case NoCompressionCodec => + debug("Sending %d messages with no compression to %s".format(messages.size, topicAndPartition)) + new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*) + case _ => + config.compressedTopics.size match { + case 0 => debug("Sending %d messages with compression codec %d to %s" .format(messages.size, config.compressionCodec.codec, topicAndPartition)) new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*) - } - else { - debug("Sending %d messages to %s with no compression as it is not in compressed.topics - %s" - .format(messages.size, topicAndPartition, config.compressedTopics.toString)) - new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*) - } - } - } - ) + case _ => + if (config.compressedTopics.contains(topicAndPartition.topic)) { + debug("Sending %d messages with compression codec %d to %s" + .format(messages.size, config.compressionCodec.codec, topicAndPartition)) + new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*) + } + else { + debug("Sending %d messages to %s with no compression as it is not in compressed.topics - %s" + .format(messages.size, topicAndPartition, config.compressedTopics.toString)) + new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*) + } + } + } + ) + } + Some(messagesPerTopicPartition) + } catch { + case t: Throwable => error("Failed to group messages", t); None } - messagesPerTopicPartition } def close() { diff --git a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala index 42e9c741c2dce..2ccf82a6f1ed6 100644 --- a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala +++ b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala @@ -34,10 +34,11 @@ class ProducerSendThread[K,V](val threadName: String, private val shutdownLatch = new CountDownLatch(1) private val shutdownCommand = new KeyedMessage[K,V]("shutdown", null.asInstanceOf[K], null.asInstanceOf[V]) - newGauge(clientId + "-ProducerQueueSize", + newGauge("ProducerQueueSize", new Gauge[Int] { def value = queue.size - }) + }, + Map("clientId" -> clientId)) override def run { try { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala old mode 100644 new mode 100755 index 9390edf37dae9..ec405169bfbc2 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -20,13 +20,14 @@ package kafka.server import scala.collection.mutable import scala.collection.Set import scala.collection.Map -import kafka.utils.{Utils, Logging} -import kafka.cluster.Broker +import kafka.utils.Logging +import kafka.cluster.BrokerEndPoint import kafka.metrics.KafkaMetricsGroup import kafka.common.TopicAndPartition import com.yammer.metrics.core.Gauge +import org.apache.kafka.common.utils.Utils -abstract class AbstractFetcherManager(protected val name: String, metricPrefix: String, numFetchers: Int = 1) +abstract class AbstractFetcherManager(protected val name: String, clientId: String, numFetchers: Int = 1) extends Logging with KafkaMetricsGroup { // map of (source broker_id, fetcher_id per source broker) => fetcher private val fetcherThreadMap = new mutable.HashMap[BrokerAndFetcherId, AbstractFetcherThread] @@ -34,7 +35,7 @@ abstract class AbstractFetcherManager(protected val name: String, metricPrefix: this.logIdent = "[" + name + "] " newGauge( - metricPrefix + "-MaxLag", + "MaxLag", new Gauge[Long] { // current max lag across all fetchers/topics/partitions def value = fetcherThreadMap.foldLeft(0L)((curMaxAll, fetcherThreadMapEntry) => { @@ -42,24 +43,25 @@ abstract class AbstractFetcherManager(protected val name: String, metricPrefix: curMaxThread.max(fetcherLagStatsEntry._2.lag) }).max(curMaxAll) }) - } + }, + Map("clientId" -> clientId) ) newGauge( - metricPrefix + "-MinFetchRate", - { - new Gauge[Double] { - // current min fetch rate across all fetchers/topics/partitions - def value = { - val headRate: Double = - fetcherThreadMap.headOption.map(_._2.fetcherStats.requestRate.oneMinuteRate).getOrElse(0) - - fetcherThreadMap.foldLeft(headRate)((curMinAll, fetcherThreadMapEntry) => { - fetcherThreadMapEntry._2.fetcherStats.requestRate.oneMinuteRate.min(curMinAll) - }) - } + "MinFetchRate", { + new Gauge[Double] { + // current min fetch rate across all fetchers/topics/partitions + def value = { + val headRate: Double = + fetcherThreadMap.headOption.map(_._2.fetcherStats.requestRate.oneMinuteRate).getOrElse(0) + + fetcherThreadMap.foldLeft(headRate)((curMinAll, fetcherThreadMapEntry) => { + fetcherThreadMapEntry._2.fetcherStats.requestRate.oneMinuteRate.min(curMinAll) + }) } } + }, + Map("clientId" -> clientId) ) private def getFetcherId(topic: String, partitionId: Int) : Int = { @@ -67,7 +69,7 @@ abstract class AbstractFetcherManager(protected val name: String, metricPrefix: } // to be defined in subclass to create a specific fetcher - def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread + def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread def addFetcherForPartitions(partitionAndOffsets: Map[TopicAndPartition, BrokerAndInitialOffset]) { mapLock synchronized { @@ -125,6 +127,6 @@ abstract class AbstractFetcherManager(protected val name: String, metricPrefix: } } -case class BrokerAndFetcherId(broker: Broker, fetcherId: Int) +case class BrokerAndFetcherId(broker: BrokerEndPoint, fetcherId: Int) -case class BrokerAndInitialOffset(broker: Broker, initOffset: Long) \ No newline at end of file +case class BrokerAndInitialOffset(broker: BrokerEndPoint, initOffset: Long) diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala old mode 100644 new mode 100755 index db7017b8a8ff0..f84306143c430 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -17,36 +17,35 @@ 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 kafka.cluster.BrokerEndPoint import kafka.utils.{Pool, ShutdownableThread} 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 kafka.utils.DelayedItem +import kafka.utils.CoreUtils.inLock +import kafka.message.{InvalidMessageException, ByteBufferMessageSet, MessageAndOffset} +import kafka.metrics.KafkaMetricsGroup + +import scala.collection.{mutable, Set, Map} import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.atomic.AtomicLong +import com.yammer.metrics.core.Gauge /** * Abstract class for fetching data from multiple partitions from the same broker. */ -abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: Broker, socketTimeout: Int, socketBufferSize: Int, - fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1, +abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: BrokerEndPoint, socketTimeout: Int, socketBufferSize: Int, + fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1, fetchBackOffMs: Int = 0, isInterruptible: Boolean = true) extends ShutdownableThread(name, isInterruptible) { - private val partitionMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> offset map + private val partitionMap = new mutable.HashMap[TopicAndPartition, PartitionFetchState] // a (topic, partition) -> partitionFetchState map private val partitionMapLock = new ReentrantLock private val partitionMapCond = partitionMapLock.newCondition() val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, clientId) - private val brokerInfo = "host_%s-port_%s".format(sourceBroker.host, sourceBroker.port) - private val metricId = new ClientIdAndBroker(clientId, brokerInfo) + private val metricId = new ClientIdAndBroker(clientId, sourceBroker.host, sourceBroker.port) val fetcherStats = new FetcherStats(metricId) val fetcherLagStats = new FetcherLagStats(metricId) val fetchRequestBuilder = new FetchRequestBuilder(). @@ -68,23 +67,35 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) override def shutdown(){ - super.shutdown() + val justShutdown = initiateShutdown() + if (justShutdown && isInterruptible) + simpleConsumer.disconnectToHandleJavaIOBug() + inLock(partitionMapLock) { + partitionMapCond.signalAll() + } + awaitShutdown() simpleConsumer.close() } override def doWork() { + var fetchRequest: FetchRequest = null + inLock(partitionMapLock) { - if (partitionMap.isEmpty) - partitionMapCond.await(200L, TimeUnit.MILLISECONDS) partitionMap.foreach { - case((topicAndPartition, offset)) => - fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition, - offset, fetchSize) + case((topicAndPartition, partitionFetchState)) => + if(partitionFetchState.isActive) + fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition, + partitionFetchState.offset, fetchSize) + } + + fetchRequest = fetchRequestBuilder.build() + if (fetchRequest.requestInfo.isEmpty) { + trace("There are no active partitions. Back off for %d ms before sending a fetch request".format(fetchBackOffMs)) + partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS) } } - val fetchRequest = fetchRequestBuilder.build() - if (!fetchRequest.requestInfo.isEmpty) + if(!fetchRequest.requestInfo.isEmpty) processFetchRequest(fetchRequest) } @@ -92,14 +103,16 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke val partitionsWithError = new mutable.HashSet[TopicAndPartition] var response: FetchResponse = null try { - trace("issuing to broker %d of fetch request %s".format(sourceBroker.id, fetchRequest)) + trace("Issuing to broker %d of fetch request %s".format(sourceBroker.id, fetchRequest)) response = simpleConsumer.fetch(fetchRequest) } catch { case t: Throwable => if (isRunning.get) { - error("Error in fetch %s".format(fetchRequest), t) - partitionMapLock synchronized { + warn("Error in fetch %s. Possible cause: %s".format(fetchRequest, t.toString)) + inLock(partitionMapLock) { partitionsWithError ++= partitionMap.keys + // there is an error occurred while fetching partitions, sleep a while + partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS) } } } @@ -111,53 +124,53 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke response.data.foreach { case(topicAndPartition, partitionData) => val (topic, partitionId) = topicAndPartition.asTuple - val currentOffset = partitionMap.get(topicAndPartition) - // we append to the log if the current offset is defined and it is the same as the offset requested during fetch - if (currentOffset.isDefined && fetchRequest.requestInfo(topicAndPartition).offset == currentOffset.get) { - partitionData.error match { - case ErrorMapping.NoError => - try { - val messages = partitionData.messages.asInstanceOf[ByteBufferMessageSet] - val validBytes = messages.validBytes - val newOffset = messages.shallowIterator.toSeq.lastOption match { - case Some(m: MessageAndOffset) => m.nextOffset - case None => currentOffset.get + partitionMap.get(topicAndPartition).foreach(currentPartitionFetchState => + // we append to the log if the current offset is defined and it is the same as the offset requested during fetch + if (fetchRequest.requestInfo(topicAndPartition).offset == currentPartitionFetchState.offset) { + partitionData.error match { + case ErrorMapping.NoError => + try { + val messages = partitionData.messages.asInstanceOf[ByteBufferMessageSet] + val validBytes = messages.validBytes + val newOffset = messages.shallowIterator.toSeq.lastOption match { + case Some(m: MessageAndOffset) => m.nextOffset + case None => currentPartitionFetchState.offset + } + partitionMap.put(topicAndPartition, new PartitionFetchState(newOffset)) + fetcherLagStats.getFetcherLagStats(topic, partitionId).lag = partitionData.hw - newOffset + fetcherStats.byteRate.mark(validBytes) + // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread + processPartitionData(topicAndPartition, currentPartitionFetchState.offset, partitionData) + } catch { + case ime: InvalidMessageException => + // we log the error and continue. This ensures two things + // 1. If there is a corrupt message in a topic partition, it does not bring the fetcher thread down and cause other topic partition to also lag + // 2. If the message is corrupt due to a transient state in the log (truncation, partial writes can cause this), we simply continue and + // should get fixed in the subsequent fetches + logger.error("Found invalid messages during fetch for partition [" + topic + "," + partitionId + "] offset " + currentPartitionFetchState.offset + " error " + ime.getMessage) + case e: Throwable => + throw new KafkaException("error processing data for partition [%s,%d] offset %d" + .format(topic, partitionId, currentPartitionFetchState.offset), e) + } + case ErrorMapping.OffsetOutOfRangeCode => + try { + val newOffset = handleOffsetOutOfRange(topicAndPartition) + partitionMap.put(topicAndPartition, new PartitionFetchState(newOffset)) + error("Current offset %d for partition [%s,%d] out of range; reset offset to %d" + .format(currentPartitionFetchState.offset, topic, partitionId, newOffset)) + } catch { + case e: Throwable => + error("Error getting offset for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), e) + partitionsWithError += topicAndPartition } - partitionMap.put(topicAndPartition, newOffset) - fetcherLagStats.getFetcherLagStats(topic, partitionId).lag = partitionData.hw - newOffset - fetcherStats.byteRate.mark(validBytes) - // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread - processPartitionData(topicAndPartition, currentOffset.get, partitionData) - } catch { - case ime: InvalidMessageException => - // we log the error and continue. This ensures two things - // 1. If there is a corrupt message in a topic partition, it does not bring the fetcher thread down and cause other topic partition to also lag - // 2. If the message is corrupt due to a transient state in the log (truncation, partial writes can cause this), we simply continue and - // should get fixed in the subsequent fetches - logger.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) - } - case ErrorMapping.OffsetOutOfRangeCode => - try { - val newOffset = handleOffsetOutOfRange(topicAndPartition) - partitionMap.put(topicAndPartition, newOffset) - 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 => - error("Error getting offset for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), e) + case _ => + if (isRunning.get) { + error("Error for partition [%s,%d] to broker %d:%s".format(topic, partitionId, sourceBroker.id, + ErrorMapping.exceptionFor(partitionData.error).getClass)) partitionsWithError += topicAndPartition - } - case _ => - if (isRunning.get) { - error("Error for partition [%s,%d] to broker %d:%s".format(topic, partitionId, sourceBroker.id, - ErrorMapping.exceptionFor(partitionData.error).getClass)) - partitionsWithError += topicAndPartition - } - } - } + } + } + }) } } } @@ -176,7 +189,23 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke if (!partitionMap.contains(topicAndPartition)) partitionMap.put( topicAndPartition, - if (PartitionTopicInfo.isOffsetInvalid(offset)) handleOffsetOutOfRange(topicAndPartition) else offset) + if (PartitionTopicInfo.isOffsetInvalid(offset)) new PartitionFetchState(handleOffsetOutOfRange(topicAndPartition)) + else new PartitionFetchState(offset) + )} + partitionMapCond.signalAll() + } finally { + partitionMapLock.unlock() + } + } + + def delayPartitions(partitions: Iterable[TopicAndPartition], delay: Long) { + partitionMapLock.lockInterruptibly() + try { + for (partition <- partitions) { + partitionMap.get(partition).foreach (currentPartitionFetchState => + if(currentPartitionFetchState.isActive) + partitionMap.put(partition, new PartitionFetchState(currentPartitionFetchState.offset, new DelayedItem(delay))) + ) } partitionMapCond.signalAll() } finally { @@ -203,13 +232,15 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke } } -class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMetricsGroup { +class FetcherLagMetrics(metricId: ClientIdTopicPartition) extends KafkaMetricsGroup { private[this] val lagVal = new AtomicLong(-1L) - newGauge( - metricId + "-ConsumerLag", + newGauge("ConsumerLag", new Gauge[Long] { def value = lagVal.get - } + }, + Map("clientId" -> metricId.clientId, + "topic" -> metricId.topic, + "partition" -> metricId.partitionId.toString) ) def lag_=(newLag: Long) { @@ -220,20 +251,36 @@ class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMet } class FetcherLagStats(metricId: ClientIdAndBroker) { - private val valueFactory = (k: ClientIdBrokerTopicPartition) => new FetcherLagMetrics(k) - val stats = new Pool[ClientIdBrokerTopicPartition, FetcherLagMetrics](Some(valueFactory)) + private val valueFactory = (k: ClientIdTopicPartition) => new FetcherLagMetrics(k) + val stats = new Pool[ClientIdTopicPartition, FetcherLagMetrics](Some(valueFactory)) def getFetcherLagStats(topic: String, partitionId: Int): FetcherLagMetrics = { - stats.getAndMaybePut(new ClientIdBrokerTopicPartition(metricId.clientId, metricId.brokerInfo, topic, partitionId)) + stats.getAndMaybePut(new ClientIdTopicPartition(metricId.clientId, topic, partitionId)) } } class FetcherStats(metricId: ClientIdAndBroker) extends KafkaMetricsGroup { - val requestRate = newMeter(metricId + "-RequestsPerSec", "requests", TimeUnit.SECONDS) - val byteRate = newMeter(metricId + "-BytesPerSec", "bytes", TimeUnit.SECONDS) + val tags = Map("clientId" -> metricId.clientId, + "brokerHost" -> metricId.brokerHost, + "brokerPort" -> metricId.brokerPort.toString) + + val requestRate = newMeter("RequestsPerSec", "requests", TimeUnit.SECONDS, tags) + + val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, tags) } -case class ClientIdBrokerTopicPartition(clientId: String, brokerInfo: String, topic: String, partitionId: Int) { - override def toString = "%s-%s-%s-%d".format(clientId, brokerInfo, topic, partitionId) +case class ClientIdTopicPartition(clientId: String, topic: String, partitionId: Int) { + override def toString = "%s-%s-%d".format(clientId, topic, partitionId) } +/** + * case class to keep partition offset and its state(active , inactive) + */ +case class PartitionFetchState(offset: Long, delay: DelayedItem) { + + def this(offset: Long) = this(offset, new DelayedItem(0)) + + def isActive: Boolean = { delay.getDelay(TimeUnit.MILLISECONDS) == 0 } + + override def toString = "%d-%b".format(offset, isActive) +} diff --git a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala new file mode 100755 index 0000000000000..6e8d68dfae20d --- /dev/null +++ b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala @@ -0,0 +1,83 @@ +/** + * 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._ +import java.util.Properties +import kafka.utils._ +import org.apache.kafka.common.utils.Utils + +case class BrokerMetadata(brokerId: Int) + +/** + * This class saves broker's metadata to a file + */ +class BrokerMetadataCheckpoint(val file: File) extends Logging { + private val lock = new Object() + new File(file + ".tmp").delete() // try to delete any existing temp files for cleanliness + + def write(brokerMetadata: BrokerMetadata) = { + lock synchronized { + try { + val brokerMetaProps = new Properties() + brokerMetaProps.setProperty("version", 0.toString) + brokerMetaProps.setProperty("broker.id", brokerMetadata.brokerId.toString) + val temp = new File(file.getAbsolutePath + ".tmp") + val fileOutputStream = new FileOutputStream(temp) + brokerMetaProps.store(fileOutputStream,"") + fileOutputStream.flush() + fileOutputStream.getFD().sync() + fileOutputStream.close() + // swap new BrokerMetadata file with previous one + if(!temp.renameTo(file)) { + // renameTo() fails on windows if destination file exists. + file.delete() + if(!temp.renameTo(file)) + throw new IOException("File rename from %s to %s failed.".format(temp.getAbsolutePath(), file.getAbsolutePath())) + } + } catch { + case ie: IOException => + error("Failed to write meta.properties due to ",ie) + throw ie + } + } + } + + def read(): Option[BrokerMetadata] = { + lock synchronized { + try { + val brokerMetaProps = new VerifiableProperties(Utils.loadProps(file.getAbsolutePath())) + val version = brokerMetaProps.getIntInRange("version", (0, Int.MaxValue)) + version match { + case 0 => + val brokerId = brokerMetaProps.getIntInRange("broker.id", (0, Int.MaxValue)) + return Some(BrokerMetadata(brokerId)) + case _ => + throw new IOException("Unrecognized version of the server meta.properties file: " + version) + } + } catch { + case e: FileNotFoundException => + warn("No meta.properties file under dir %s".format(file.getAbsolutePath())) + None + case e1: Exception => + error("Failed to read meta.properties file under dir %s due to %s".format(file.getAbsolutePath(), e1.getMessage)) + throw e1 + } + } + } +} diff --git a/core/src/main/scala/kafka/server/BrokerStates.scala b/core/src/main/scala/kafka/server/BrokerStates.scala new file mode 100644 index 0000000000000..e6ee77ebde5c2 --- /dev/null +++ b/core/src/main/scala/kafka/server/BrokerStates.scala @@ -0,0 +1,80 @@ +/** + * 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 + +/** + * Broker states are the possible state that a kafka broker can be in. + * A broker should be only in one state at a time. + * The expected state transition with the following defined states is: + * + * +-----------+ + * |Not Running| + * +-----+-----+ + * | + * v + * +-----+-----+ + * |Starting +--+ + * +-----+-----+ | +----+------------+ + * | +>+RecoveringFrom | + * v |UncleanShutdown | + * +----------+ +-----+-----+ +-------+---------+ + * |RunningAs | |RunningAs | | + * |Controller+<--->+Broker +<-----------+ + * +----------+ +-----+-----+ + * | | + * | v + * | +-----+------------+ + * |-----> |PendingControlled | + * |Shutdown | + * +-----+------------+ + * | + * v + * +-----+----------+ + * |BrokerShutting | + * |Down | + * +-----+----------+ + * | + * v + * +-----+-----+ + * |Not Running| + * +-----------+ + * + * Custom states is also allowed for cases where there are custom kafka states for different scenarios. + */ +sealed trait BrokerStates { def state: Byte } +case object NotRunning extends BrokerStates { val state: Byte = 0 } +case object Starting extends BrokerStates { val state: Byte = 1 } +case object RecoveringFromUncleanShutdown extends BrokerStates { val state: Byte = 2 } +case object RunningAsBroker extends BrokerStates { val state: Byte = 3 } +case object RunningAsController extends BrokerStates { val state: Byte = 4 } +case object PendingControlledShutdown extends BrokerStates { val state: Byte = 6 } +case object BrokerShuttingDown extends BrokerStates { val state: Byte = 7 } + + +case class BrokerState() { + @volatile var currentState: Byte = NotRunning.state + + def newState(newState: BrokerStates) { + this.newState(newState.state) + } + + // Allowing undefined custom state + def newState(newState: Byte) { + currentState = newState + } +} diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala new file mode 100644 index 0000000000000..de6cf5bdaa0e7 --- /dev/null +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -0,0 +1,143 @@ +/** + * 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.concurrent.TimeUnit + +import kafka.api.FetchResponsePartitionData +import kafka.api.PartitionFetchInfo +import kafka.common.UnknownTopicOrPartitionException +import kafka.common.NotLeaderForPartitionException +import kafka.common.TopicAndPartition +import kafka.metrics.KafkaMetricsGroup + +import scala.collection._ + +case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInfo: PartitionFetchInfo) { + + override def toString = "[startOffsetMetadata: " + startOffsetMetadata + ", " + + "fetchInfo: " + fetchInfo + "]" +} + +/** + * The fetch metadata maintained by the delayed fetch operation + */ +case class FetchMetadata(fetchMinBytes: Int, + fetchOnlyLeader: Boolean, + fetchOnlyCommitted: Boolean, + isFromFollower: Boolean, + fetchPartitionStatus: Map[TopicAndPartition, FetchPartitionStatus]) { + + override def toString = "[minBytes: " + fetchMinBytes + ", " + + "onlyLeader:" + fetchOnlyLeader + ", " + "onlyCommitted: " + fetchOnlyCommitted + ", " + "partitionStatus: " + fetchPartitionStatus + "]" +} +/** + * A delayed fetch operation that can be created by the replica manager and watched + * in the fetch operation purgatory + */ +class DelayedFetch(delayMs: Long, + fetchMetadata: FetchMetadata, + replicaManager: ReplicaManager, + responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) + extends DelayedOperation(delayMs) { + + /** + * The operation can be completed if: + * + * Case A: This broker is no longer the leader for some partitions it tries to fetch + * Case B: This broker does not know of some partitions it tries to fetch + * Case C: The fetch offset locates not on the last segment of the log + * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes + * + * Upon completion, should return whatever data is available for each valid partition + */ + override def tryComplete() : Boolean = { + var accumulatedSize = 0 + fetchMetadata.fetchPartitionStatus.foreach { + case (topicAndPartition, fetchStatus) => + val fetchOffset = fetchStatus.startOffsetMetadata + try { + if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) { + val replica = replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) + val endOffset = + if (fetchMetadata.fetchOnlyCommitted) + replica.highWatermark + else + replica.logEndOffset + + if (endOffset.offsetOnOlderSegment(fetchOffset)) { + // Case C, this can happen when the new fetch operation is on a truncated leader + debug("Satisfying fetch %s since it is fetching later segments of partition %s.".format(fetchMetadata, topicAndPartition)) + return forceComplete() + } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { + // Case C, this can happen when the fetch operation is falling behind the current segment + // or the partition has just rolled a new segment + debug("Satisfying fetch %s immediately since it is fetching older segments.".format(fetchMetadata)) + return forceComplete() + } else if (fetchOffset.precedes(endOffset)) { + // we need take the partition fetch size as upper bound when accumulating the bytes + accumulatedSize += math.min(endOffset.positionDiff(fetchOffset), fetchStatus.fetchInfo.fetchSize) + } + } + } catch { + case utpe: UnknownTopicOrPartitionException => // Case B + debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) + return forceComplete() + case nle: NotLeaderForPartitionException => // Case A + debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) + return forceComplete() + } + } + + // Case D + if (accumulatedSize >= fetchMetadata.fetchMinBytes) + forceComplete() + else + false + } + + override def onExpiration() { + if (fetchMetadata.isFromFollower) + DelayedFetchMetrics.followerExpiredRequestMeter.mark() + else + DelayedFetchMetrics.consumerExpiredRequestMeter.mark() + } + + /** + * Upon completion, read whatever data is available and pass to the complete callback + */ + override def onComplete() { + val logReadResults = replicaManager.readFromLocalLog(fetchMetadata.fetchOnlyLeader, + fetchMetadata.fetchOnlyCommitted, + fetchMetadata.fetchPartitionStatus.mapValues(status => status.fetchInfo)) + + val fetchPartitionData = logReadResults.mapValues(result => + FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + + responseCallback(fetchPartitionData) + } +} + +object DelayedFetchMetrics extends KafkaMetricsGroup { + private val FetcherTypeKey = "fetcherType" + val followerExpiredRequestMeter = newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS, tags = Map(FetcherTypeKey -> "follower")) + val consumerExpiredRequestMeter = newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS, tags = Map(FetcherTypeKey -> "consumer")) +} + diff --git a/core/src/main/scala/kafka/server/DelayedOperation.scala b/core/src/main/scala/kafka/server/DelayedOperation.scala new file mode 100644 index 0000000000000..0b535329364e2 --- /dev/null +++ b/core/src/main/scala/kafka/server/DelayedOperation.scala @@ -0,0 +1,365 @@ +/** + * 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._ +import kafka.utils.timer._ +import kafka.utils.CoreUtils.{inReadLock, inWriteLock} +import kafka.metrics.KafkaMetricsGroup + +import java.util.LinkedList +import java.util.concurrent._ +import java.util.concurrent.atomic._ +import java.util.concurrent.locks.ReentrantReadWriteLock + +import org.apache.kafka.common.utils.Utils + +import scala.collection._ + +import com.yammer.metrics.core.Gauge + + +/** + * An operation whose processing needs to be delayed for at most the given delayMs. For example + * a delayed produce operation could be waiting for specified number of acks; or + * a delayed fetch operation could be waiting for a given number of bytes to accumulate. + * + * The logic upon completing a delayed operation is defined in onComplete() and will be called exactly once. + * Once an operation is completed, isCompleted() will return true. onComplete() can be triggered by either + * forceComplete(), which forces calling onComplete() after delayMs if the operation is not yet completed, + * or tryComplete(), which first checks if the operation can be completed or not now, and if yes calls + * forceComplete(). + * + * A subclass of DelayedOperation needs to provide an implementation of both onComplete() and tryComplete(). + */ +abstract class DelayedOperation(delayMs: Long) extends TimerTask with Logging { + + override val expirationMs = delayMs + System.currentTimeMillis() + + private val completed = new AtomicBoolean(false) + + /* + * Force completing the delayed operation, if not already completed. + * This function can be triggered when + * + * 1. The operation has been verified to be completable inside tryComplete() + * 2. The operation has expired and hence needs to be completed right now + * + * Return true iff the operation is completed by the caller: note that + * concurrent threads can try to complete the same operation, but only + * the first thread will succeed in completing the operation and return + * true, others will still return false + */ + def forceComplete(): Boolean = { + if (completed.compareAndSet(false, true)) { + // cancel the timeout timer + cancel() + onComplete() + true + } else { + false + } + } + + /** + * Check if the delayed operation is already completed + */ + def isCompleted(): Boolean = completed.get() + + /** + * Call-back to execute when a delayed operation gets expired and hence forced to complete. + */ + def onExpiration(): Unit + + /** + * Process for completing an operation; This function needs to be defined + * in subclasses and will be called exactly once in forceComplete() + */ + def onComplete(): Unit + + /* + * Try to complete the delayed operation by first checking if the operation + * can be completed by now. If yes execute the completion logic by calling + * forceComplete() and return true iff forceComplete returns true; otherwise return false + * + * This function needs to be defined in subclasses + */ + def tryComplete(): Boolean + + /* + * run() method defines a task that is executed on timeout + */ + override def run(): Unit = { + if (forceComplete()) + onExpiration() + } +} + +/** + * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations. + */ +class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, brokerId: Int = 0, purgeInterval: Int = 1000) + extends Logging with KafkaMetricsGroup { + + // timeout timer + private[this] val executor = Executors.newFixedThreadPool(1, new ThreadFactory() { + def newThread(runnable: Runnable): Thread = + Utils.newThread("executor-"+purgatoryName, runnable, false) + }) + private[this] val timeoutTimer = new Timer(executor) + + /* a list of operation watching keys */ + private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers(key))) + + private val removeWatchersLock = new ReentrantReadWriteLock() + + // the number of estimated total operations in the purgatory + private[this] val estimatedTotalOperations = new AtomicInteger(0) + + /* background thread expiring operations that have timed out */ + private val expirationReaper = new ExpiredOperationReaper() + + private val metricsTags = Map("delayedOperation" -> purgatoryName) + + newGauge( + "PurgatorySize", + new Gauge[Int] { + def value = watched() + }, + metricsTags + ) + + newGauge( + "NumDelayedOperations", + new Gauge[Int] { + def value = delayed() + }, + metricsTags + ) + + expirationReaper.start() + + /** + * Check if the operation can be completed, if not watch it based on the given watch keys + * + * Note that a delayed operation can be watched on multiple keys. It is possible that + * an operation is completed after it has been added to the watch list for some, but + * not all of the keys. In this case, the operation is considered completed and won't + * be added to the watch list of the remaining keys. The expiration reaper thread will + * remove this operation from any watcher list in which the operation exists. + * + * @param operation the delayed operation to be checked + * @param watchKeys keys for bookkeeping the operation + * @return true iff the delayed operations can be completed by the caller + */ + def tryCompleteElseWatch(operation: T, watchKeys: Seq[Any]): Boolean = { + assert(watchKeys.size > 0, "The watch key list can't be empty") + + // The cost of tryComplete() is typically proportional to the number of keys. Calling + // tryComplete() for each key is going to be expensive if there are many keys. Instead, + // we do the check in the following way. Call tryComplete(). If the operation is not completed, + // we just add the operation to all keys. Then we call tryComplete() again. At this time, if + // the operation is still not completed, we are guaranteed that it won't miss any future triggering + // event since the operation is already on the watcher list for all keys. This does mean that + // if the operation is completed (by another thread) between the two tryComplete() calls, the + // operation is unnecessarily added for watch. However, this is a less severe issue since the + // expire reaper will clean it up periodically. + + var isCompletedByMe = operation synchronized operation.tryComplete() + if (isCompletedByMe) + return true + + var watchCreated = false + for(key <- watchKeys) { + // If the operation is already completed, stop adding it to the rest of the watcher list. + if (operation.isCompleted()) + return false + watchForOperation(key, operation) + + if (!watchCreated) { + watchCreated = true + estimatedTotalOperations.incrementAndGet() + } + } + + isCompletedByMe = operation synchronized operation.tryComplete() + if (isCompletedByMe) + return true + + // if it cannot be completed by now and hence is watched, add to the expire queue also + if (! operation.isCompleted()) { + timeoutTimer.add(operation) + if (operation.isCompleted()) { + // cancel the timer task + operation.cancel() + } + } + + false + } + + /** + * Check if some some delayed operations can be completed with the given watch key, + * and if yes complete them. + * + * @return the number of completed operations during this process + */ + def checkAndComplete(key: Any): Int = { + val watchers = inReadLock(removeWatchersLock) { watchersForKey.get(key) } + if(watchers == null) + 0 + else + watchers.tryCompleteWatched() + } + + /** + * Return the total size of watch lists the purgatory. Since an operation may be watched + * on multiple lists, and some of its watched entries may still be in the watch lists + * even when it has been completed, this number may be larger than the number of real operations watched + */ + def watched() = allWatchers.map(_.watched).sum + + /** + * Return the number of delayed operations in the expiry queue + */ + def delayed() = timeoutTimer.size + + /* + * Return all the current watcher lists, + * note that the returned watchers may be removed from the list by other threads + */ + private def allWatchers = inReadLock(removeWatchersLock) { watchersForKey.values } + + /* + * Return the watch list of the given key, note that we need to + * grab the removeWatchersLock to avoid the operation being added to a removed watcher list + */ + private def watchForOperation(key: Any, operation: T) { + inReadLock(removeWatchersLock) { + val watcher = watchersForKey.getAndMaybePut(key) + watcher.watch(operation) + } + } + + /* + * Remove the key from watcher lists if its list is empty + */ + private def removeKeyIfEmpty(key: Any, watchers: Watchers) { + inWriteLock(removeWatchersLock) { + // if the current key is no longer correlated to the watchers to remove, skip + if (watchersForKey.get(key) != watchers) + return + + if (watchers != null && watchers.watched == 0) { + watchersForKey.remove(key) + } + } + } + + /** + * Shutdown the expire reaper thread + */ + def shutdown() { + expirationReaper.shutdown() + executor.shutdown() + } + + /** + * A linked list of watched delayed operations based on some key + */ + private class Watchers(val key: Any) { + + private[this] val operations = new LinkedList[T]() + + def watched: Int = operations synchronized operations.size + + // add the element to watch + def watch(t: T) { + operations synchronized operations.add(t) + } + + // traverse the list and try to complete some watched elements + def tryCompleteWatched(): Int = { + + var completed = 0 + operations synchronized { + val iter = operations.iterator() + while (iter.hasNext) { + val curr = iter.next() + if (curr.isCompleted) { + // another thread has completed this operation, just remove it + iter.remove() + } else if (curr synchronized curr.tryComplete()) { + completed += 1 + iter.remove() + } + } + } + + if (operations.size == 0) + removeKeyIfEmpty(key, this) + + completed + } + + // traverse the list and purge elements that are already completed by others + def purgeCompleted(): Int = { + var purged = 0 + operations synchronized { + val iter = operations.iterator() + while (iter.hasNext) { + val curr = iter.next() + if (curr.isCompleted) { + iter.remove() + purged += 1 + } + } + } + + if (operations.size == 0) + removeKeyIfEmpty(key, this) + + purged + } + } + + /** + * A background reaper to expire delayed operations that have timed out + */ + private class ExpiredOperationReaper extends ShutdownableThread( + "ExpirationReaper-%d".format(brokerId), + false) { + + override def doWork() { + timeoutTimer.advanceClock(200L) + + // Trigger a purge if the number of completed but still being watched operations is larger than + // the purge threshold. That number is computed by the difference btw the estimated total number of + // operations and the number of pending delayed operations. + if (estimatedTotalOperations.get - delayed > purgeInterval) { + // now set estimatedTotalOperations to delayed (the number of pending operations) since we are going to + // clean up watchers. Note that, if more operations are completed during the clean up, we may end up with + // a little overestimated total number of operations. + estimatedTotalOperations.getAndSet(delayed) + debug("Begin purging watch lists") + val purged = allWatchers.map(_.purgeCompleted()).sum + debug("Purged %d elements from watch lists.".format(purged)) + } + } + } +} diff --git a/core/src/main/scala/kafka/server/DelayedOperationKey.scala b/core/src/main/scala/kafka/server/DelayedOperationKey.scala new file mode 100644 index 0000000000000..c122bdebc6b90 --- /dev/null +++ b/core/src/main/scala/kafka/server/DelayedOperationKey.scala @@ -0,0 +1,51 @@ +/** + * 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.common.TopicAndPartition + +/** + * Keys used for delayed operation metrics recording + */ +trait DelayedOperationKey { + def keyLabel: String +} + +object DelayedOperationKey { + val globalLabel = "All" +} + +/* used by delayed-produce and delayed-fetch operations */ +case class TopicPartitionOperationKey(topic: String, partition: Int) extends DelayedOperationKey { + + def this(topicAndPartition: TopicAndPartition) = this(topicAndPartition.topic, topicAndPartition.partition) + + override def keyLabel = "%s-%d".format(topic, partition) +} + +/* used by delayed-join-group operations */ +case class ConsumerKey(groupId: String, consumerId: String) extends DelayedOperationKey { + + override def keyLabel = "%s-%s".format(groupId, consumerId) +} + +/* used by delayed-rebalance operations */ +case class ConsumerGroupKey(groupId: String) extends DelayedOperationKey { + + override def keyLabel = groupId +} diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala new file mode 100644 index 0000000000000..05078b24ef28f --- /dev/null +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -0,0 +1,149 @@ +/** + * 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.concurrent.TimeUnit + +import com.yammer.metrics.core.Meter +import kafka.api.ProducerResponseStatus +import kafka.common.ErrorMapping +import kafka.common.TopicAndPartition +import kafka.metrics.KafkaMetricsGroup +import kafka.utils.Pool + +import scala.collection._ + +case class ProducePartitionStatus(requiredOffset: Long, responseStatus: ProducerResponseStatus) { + @volatile var acksPending = false + + override def toString = "[acksPending: %b, error: %d, startOffset: %d, requiredOffset: %d]" + .format(acksPending, responseStatus.error, responseStatus.offset, requiredOffset) +} + +/** + * The produce metadata maintained by the delayed produce operation + */ +case class ProduceMetadata(produceRequiredAcks: Short, + produceStatus: Map[TopicAndPartition, ProducePartitionStatus]) { + + override def toString = "[requiredAcks: %d, partitionStatus: %s]" + .format(produceRequiredAcks, produceStatus) +} + +/** + * A delayed produce operation that can be created by the replica manager and watched + * in the produce operation purgatory + */ +class DelayedProduce(delayMs: Long, + produceMetadata: ProduceMetadata, + replicaManager: ReplicaManager, + responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) + extends DelayedOperation(delayMs) { + + // first update the acks pending variable according to the error code + produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => + if (status.responseStatus.error == ErrorMapping.NoError) { + // Timeout error state will be cleared when required acks are received + status.acksPending = true + status.responseStatus.error = ErrorMapping.RequestTimedOutCode + } else { + status.acksPending = false + } + + trace("Initial partition status for %s is %s".format(topicAndPartition, status)) + } + + /** + * The delayed produce operation can be completed if every partition + * it produces to is satisfied by one of the following: + * + * Case A: This broker is no longer the leader: set an error in response + * Case B: This broker is the leader: + * B.1 - If there was a local error thrown while checking if at least requiredAcks + * replicas have caught up to this operation: set an error in response + * B.2 - Otherwise, set the response with no error. + */ + override def tryComplete(): Boolean = { + // check for each partition if it still has pending acks + produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => + trace("Checking produce satisfaction for %s, current status %s" + .format(topicAndPartition, status)) + // skip those partitions that have already been satisfied + if (status.acksPending) { + val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) + val (hasEnough, errorCode) = partitionOpt match { + case Some(partition) => + partition.checkEnoughReplicasReachOffset(status.requiredOffset) + case None => + // Case A + (false, ErrorMapping.UnknownTopicOrPartitionCode) + } + if (errorCode != ErrorMapping.NoError) { + // Case B.1 + status.acksPending = false + status.responseStatus.error = errorCode + } else if (hasEnough) { + // Case B.2 + status.acksPending = false + status.responseStatus.error = ErrorMapping.NoError + } + } + } + + // check if each partition has satisfied at lease one of case A and case B + if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) + forceComplete() + else + false + } + + override def onExpiration() { + produceMetadata.produceStatus.foreach { case (topicPartition, status) => + if (status.acksPending) { + DelayedProduceMetrics.recordExpiration(topicPartition) + } + } + } + + /** + * Upon completion, return the current response status along with the error code per partition + */ + override def onComplete() { + val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) + responseCallback(responseStatus) + } +} + +object DelayedProduceMetrics extends KafkaMetricsGroup { + + private val aggregateExpirationMeter = newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS) + + private val partitionExpirationMeterFactory = (key: TopicAndPartition) => + newMeter("ExpiresPerSec", + "requests", + TimeUnit.SECONDS, + tags = Map("topic" -> key.topic, "partition" -> key.partition.toString)) + private val partitionExpirationMeters = new Pool[TopicAndPartition, Meter](valueFactory = Some(partitionExpirationMeterFactory)) + + def recordExpiration(partition: TopicAndPartition) { + aggregateExpirationMeter.mark() + partitionExpirationMeters.getAndMaybePut(partition).mark() + } +} + diff --git a/core/src/main/scala/kafka/network/ByteBufferSend.scala b/core/src/main/scala/kafka/server/FetchDataInfo.scala similarity index 62% rename from core/src/main/scala/kafka/network/ByteBufferSend.scala rename to core/src/main/scala/kafka/server/FetchDataInfo.scala index af30042a4c713..1a8a60494100e 100644 --- a/core/src/main/scala/kafka/network/ByteBufferSend.scala +++ b/core/src/main/scala/kafka/server/FetchDataInfo.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,26 +15,8 @@ * limitations under the License. */ -package kafka.network - -import java.nio._ -import java.nio.channels._ -import kafka.utils._ +package kafka.server -@nonthreadsafe -private[kafka] class ByteBufferSend(val buffer: ByteBuffer) extends Send { - - var complete: Boolean = false +import kafka.message.MessageSet - def this(size: Int) = this(ByteBuffer.allocate(size)) - - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - var written = 0 - written += channel.write(buffer) - if(!buffer.hasRemaining) - complete = true - written - } - -} +case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, messageSet: MessageSet) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c56ad503d83e3..18f5b5b895af1 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,45 +17,33 @@ package kafka.server -import kafka.admin.AdminUtils +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.TopicPartition import kafka.api._ -import kafka.message._ -import kafka.network._ +import kafka.admin.AdminUtils +import kafka.common._ +import kafka.controller.KafkaController +import kafka.coordinator.ConsumerCoordinator import kafka.log._ -import kafka.utils.ZKGroupTopicDirs +import kafka.network._ +import kafka.network.RequestChannel.Response +import org.apache.kafka.common.requests.{JoinGroupRequest, JoinGroupResponse, HeartbeatRequest, HeartbeatResponse, ResponseHeader, ResponseSend} +import kafka.utils.{ZkUtils, ZKGroupTopicDirs, SystemTime, Logging} import scala.collection._ -import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic._ -import kafka.metrics.KafkaMetricsGroup import org.I0Itec.zkclient.ZkClient -import kafka.common._ -import kafka.utils.{ZkUtils, Pool, SystemTime, Logging} -import kafka.network.RequestChannel.Response -import kafka.cluster.Broker -import kafka.controller.KafkaController - /** * Logic to handle the various Kafka requests */ class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, + val coordinator: ConsumerCoordinator, + val controller: KafkaController, val zkClient: ZkClient, val brokerId: Int, val config: KafkaConfig, - val controller: KafkaController) extends Logging { - - private val producerRequestPurgatory = - new ProducerRequestPurgatory(replicaManager.config.producerPurgatoryPurgeIntervalRequests) - private val fetchRequestPurgatory = - new FetchRequestPurgatory(requestChannel, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) - private val delayedRequestMetrics = new DelayedRequestMetrics - /* following 3 data structures are updated by the update metadata request - * and is queried by the topic metadata request. */ - var metadataCache: mutable.Map[TopicAndPartition, PartitionStateInfo] = - new mutable.HashMap[TopicAndPartition, PartitionStateInfo]() - private val aliveBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]() - private val partitionMetadataLock = new Object + val metadataCache: MetadataCache) extends Logging { + this.logIdent = "[KafkaApi-%d] ".format(brokerId) /** @@ -63,7 +51,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handle(request: RequestChannel.Request) { try{ - trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress) + trace("Handling request: " + request.requestObj + " from connection: " + request.connectionId) request.requestId match { case RequestKeys.ProduceKey => handleProducerRequest(request) case RequestKeys.FetchKey => handleFetchRequest(request) @@ -75,31 +63,55 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request) case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request) case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request) + case RequestKeys.ConsumerMetadataKey => handleConsumerMetadataRequest(request) + case RequestKeys.JoinGroupKey => handleJoinGroupRequest(request) + case RequestKeys.HeartbeatKey => handleHeartbeatRequest(request) case requestId => throw new KafkaException("Unknown api code " + requestId) } } catch { case e: Throwable => - request.requestObj.handleError(e, requestChannel, request) + if ( request.requestObj != null) + request.requestObj.handleError(e, requestChannel, request) + else { + val response = request.body.getErrorResponse(request.header.apiVersion, e) + val respHeader = new ResponseHeader(request.header.correlationId) + + /* If request doesn't have a default error response, we just close the connection. + For example, when produce request has acks set to 0 */ + if (response == null) + requestChannel.closeConnection(request.processor, request) + else + requestChannel.sendResponse(new Response(request, new ResponseSend(request.connectionId, respHeader, response))) + } error("error when handling request %s".format(request.requestObj), e) } finally request.apiLocalCompleteTimeMs = SystemTime.milliseconds } - // ensureTopicExists is only for client facing requests - private def ensureTopicExists(topic: String) = { - if(!metadataCache.exists { case(topicAndPartition, partitionStateInfo) => topicAndPartition.topic.equals(topic)} ) - throw new UnknownTopicOrPartitionException("Topic " + topic + " either doesn't exist or is in the process of being deleted") - } - def handleLeaderAndIsrRequest(request: RequestChannel.Request) { // ensureTopicExists is only for client facing requests // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest] try { - val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest) - val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, response, error) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(leaderAndIsrResponse))) + // call replica manager to handle updating partitions to become leader or follower + val result = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest) + val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, result.responseMap, result.errorCode) + // for each new leader or follower, call coordinator to handle + // consumer group migration + result.updatedLeaders.foreach { case partition => + if (partition.topic == ConsumerCoordinator.OffsetsTopicName) + coordinator.handleGroupImmigration(partition.partitionId) + } + result.updatedFollowers.foreach { case partition => + partition.leaderReplicaIdOpt.foreach { leaderReplica => + if (partition.topic == ConsumerCoordinator.OffsetsTopicName && + leaderReplica == brokerId) + coordinator.handleGroupEmigration(partition.partitionId) + } + } + + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, leaderAndIsrResponse))) } catch { case e: KafkaStorageException => fatal("Disk error during leadership change.", e) @@ -114,55 +126,16 @@ class KafkaApis(val requestChannel: RequestChannel, val stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest] val (response, error) = replicaManager.stopReplicas(stopReplicaRequest) val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.correlationId, response.toMap, error) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(stopReplicaResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, stopReplicaResponse))) replicaManager.replicaFetcherManager.shutdownIdleFetcherThreads() } def handleUpdateMetadataRequest(request: RequestChannel.Request) { val updateMetadataRequest = request.requestObj.asInstanceOf[UpdateMetadataRequest] - // ensureTopicExists is only for client facing requests - // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they - // stop serving data to clients for the topic being deleted - val stateChangeLogger = replicaManager.stateChangeLogger - if(updateMetadataRequest.controllerEpoch < replicaManager.controllerEpoch) { - val stateControllerEpochErrorMessage = ("Broker %d received update metadata request with correlation id %d from an " + - "old controller %d with epoch %d. Latest known controller epoch is %d").format(brokerId, - updateMetadataRequest.correlationId, updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, - replicaManager.controllerEpoch) - stateChangeLogger.warn(stateControllerEpochErrorMessage) - throw new ControllerMovedException(stateControllerEpochErrorMessage) - } - partitionMetadataLock synchronized { - replicaManager.controllerEpoch = updateMetadataRequest.controllerEpoch - // cache the list of alive brokers in the cluster - updateMetadataRequest.aliveBrokers.foreach(b => aliveBrokers.put(b.id, b)) - updateMetadataRequest.partitionStateInfos.foreach { partitionState => - metadataCache.put(partitionState._1, partitionState._2) - if(stateChangeLogger.isTraceEnabled) - stateChangeLogger.trace(("Broker %d cached leader info %s for partition %s in response to UpdateMetadata request " + - "sent by controller %d epoch %d with correlation id %d").format(brokerId, partitionState._2, partitionState._1, - updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId)) - } - // remove the topics that don't exist in the UpdateMetadata request since those are the topics that are - // currently being deleted by the controller - val topicsKnownToThisBroker = metadataCache.map{ - case(topicAndPartition, partitionStateInfo) => topicAndPartition.topic }.toSet - val topicsKnownToTheController = updateMetadataRequest.partitionStateInfos.map { - case(topicAndPartition, partitionStateInfo) => topicAndPartition.topic }.toSet - val deletedTopics = topicsKnownToThisBroker -- topicsKnownToTheController - val partitionsToBeDeleted = metadataCache.filter { - case(topicAndPartition, partitionStateInfo) => deletedTopics.contains(topicAndPartition.topic) - }.keySet - partitionsToBeDeleted.foreach { partition => - metadataCache.remove(partition) - if(stateChangeLogger.isTraceEnabled) - stateChangeLogger.trace(("Broker %d deleted partition %s from metadata cache in response to UpdateMetadata request " + - "sent by controller %d epoch %d with correlation id %d").format(brokerId, partition, - updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId)) - } - } + replicaManager.maybeUpdateMetadataCache(updateMetadataRequest, metadataCache) + val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(updateMetadataResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, updateMetadataResponse))) } def handleControlledShutdownRequest(request: RequestChannel.Request) { @@ -173,155 +146,158 @@ class KafkaApis(val requestChannel: RequestChannel, val partitionsRemaining = controller.shutdownBroker(controlledShutdownRequest.brokerId) val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId, ErrorMapping.NoError, partitionsRemaining) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse))) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, controlledShutdownResponse))) } + /** - * Check if a partitionData from a produce request can unblock any - * DelayedFetch requests. + * Handle an offset commit request */ - def maybeUnblockDelayedFetchRequests(topic: String, partition: Int, messageSizeInBytes: Int) { - val satisfied = fetchRequestPurgatory.update(RequestKey(topic, partition), messageSizeInBytes) - trace("Producer request to (%s-%d) unblocked %d fetch requests.".format(topic, partition, satisfied.size)) - - // send any newly unblocked responses - for(fetchReq <- satisfied) { - val topicData = readMessageSets(fetchReq.fetch) - val response = FetchResponse(fetchReq.fetch.correlationId, topicData) - requestChannel.sendResponse(new RequestChannel.Response(fetchReq.request, new FetchResponseSend(response))) + def handleOffsetCommitRequest(request: RequestChannel.Request) { + val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] + + // filter non-exist topics + val invalidRequestsInfo = offsetCommitRequest.requestInfo.filter { case (topicAndPartition, offsetMetadata) => + !metadataCache.contains(topicAndPartition.topic) + } + val filteredRequestInfo = (offsetCommitRequest.requestInfo -- invalidRequestsInfo.keys) + + // the callback for sending an offset commit response + def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { + commitStatus.foreach { case (topicAndPartition, errorCode) => + // we only print warnings for known errors here; only replica manager could see an unknown + // exception while trying to write the offset message to the local log, and it will log + // an error message and write the error code in this case; hence it can be ignored here + if (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.UnknownCode) { + debug("Offset commit request with correlation id %d from client %s on partition %s failed due to %s" + .format(offsetCommitRequest.correlationId, offsetCommitRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(errorCode))) + } + } + val combinedCommitStatus = commitStatus ++ invalidRequestsInfo.map(_._1 -> ErrorMapping.UnknownTopicOrPartitionCode) + val response = OffsetCommitResponse(combinedCommitStatus, offsetCommitRequest.correlationId) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } - } - /** - * Handle a produce request - */ - def handleProducerRequest(request: RequestChannel.Request) { - val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] - val sTime = SystemTime.milliseconds - val localProduceResults = appendToLocalLog(produceRequest) - debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) - - val numPartitionsInError = localProduceResults.count(_.error.isDefined) - produceRequest.data.foreach(partitionAndData => - maybeUnblockDelayedFetchRequests(partitionAndData._1.topic, partitionAndData._1.partition, partitionAndData._2.sizeInBytes)) - - val allPartitionHaveReplicationFactorOne = - !produceRequest.data.keySet.exists( - m => replicaManager.getReplicationFactorForPartition(m.topic, m.partition) != 1) - if(produceRequest.requiredAcks == 0) { - // no operation needed if producer request.required.acks = 0; however, if there is any exception in handling the request, since - // no response is expected by the producer the handler will send a close connection response to the socket server - // to close the socket so that the producer client will know that some exception has happened and will refresh its metadata - if (numPartitionsInError != 0) { - info(("Send the close connection response due to error handling produce request " + - "[clientId = %s, correlationId = %s, topicAndPartition = %s] with Ack=0") - .format(produceRequest.clientId, produceRequest.correlationId, produceRequest.topicPartitionMessageSizeMap.keySet.mkString(","))) - requestChannel.closeConnection(request.processor, request) - } else { - requestChannel.noOperation(request.processor, request) + if (offsetCommitRequest.versionId == 0) { + // for version 0 always store offsets to ZK + val responseInfo = filteredRequestInfo.map { + case (topicAndPartition, metaAndError) => { + val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicAndPartition.topic) + try { + if (metadataCache.getTopicMetadata(Set(topicAndPartition.topic), request.securityProtocol).size <= 0) { + (topicAndPartition, ErrorMapping.UnknownTopicOrPartitionCode) + } else 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]])) + } + } } - } else if (produceRequest.requiredAcks == 1 || - produceRequest.numPartitions <= 0 || - allPartitionHaveReplicationFactorOne || - numPartitionsInError == produceRequest.numPartitions) { - val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.start)).toMap - val response = ProducerResponse(produceRequest.correlationId, statuses) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + + sendResponseCallback(responseInfo) } else { - // create a list of (topic, partition) pairs to use as keys for this delayed request - val producerRequestKeys = produceRequest.data.keys.map( - topicAndPartition => new RequestKey(topicAndPartition)).toSeq - val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.end + 1)).toMap - val delayedProduce = new DelayedProduce(producerRequestKeys, - request, - statuses, - produceRequest, - produceRequest.ackTimeoutMs.toLong) - producerRequestPurgatory.watch(delayedProduce) - - /* - * Replica fetch requests may have arrived (and potentially satisfied) - * delayedProduce requests while they were being added to the purgatory. - * Here, we explicitly check if any of them can be satisfied. - */ - var satisfiedProduceRequests = new mutable.ArrayBuffer[DelayedProduce] - producerRequestKeys.foreach(key => - satisfiedProduceRequests ++= - producerRequestPurgatory.update(key, key)) - debug(satisfiedProduceRequests.size + - " producer requests unblocked during produce to local log.") - satisfiedProduceRequests.foreach(_.respond()) - // we do not need the data anymore - produceRequest.emptyData() - } - } - - case class ProduceResult(key: TopicAndPartition, start: Long, end: Long, error: Option[Throwable] = None) { - def this(key: TopicAndPartition, throwable: Throwable) = - this(key, -1L, -1L, Some(throwable)) - - def errorCode = error match { - case None => ErrorMapping.NoError - case Some(error) => ErrorMapping.codeFor(error.getClass.asInstanceOf[Class[Throwable]]) + // for version 1 and beyond store offsets in offset manager + + // compute the retention time based on the request version: + // if it is v1 or not specified by user, we can use the default retention + val offsetRetention = + if (offsetCommitRequest.versionId <= 1 || + offsetCommitRequest.retentionMs == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) { + coordinator.offsetConfig.offsetsRetentionMs + } else { + offsetCommitRequest.retentionMs + } + + // commit timestamp is always set to now. + // "default" expiration timestamp is now + retention (and retention may be overridden if v2) + // expire timestamp is computed differently for v1 and v2. + // - If v1 and no explicit commit timestamp is provided we use default expiration timestamp. + // - If v1 and explicit commit timestamp is provided we calculate retention from that explicit commit timestamp + // - If v2 we use the default expiration timestamp + val currentTimestamp = SystemTime.milliseconds + val defaultExpireTimestamp = offsetRetention + currentTimestamp + + val offsetData = filteredRequestInfo.mapValues(offsetAndMetadata => + offsetAndMetadata.copy( + commitTimestamp = currentTimestamp, + expireTimestamp = { + if (offsetAndMetadata.commitTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + defaultExpireTimestamp + else + offsetRetention + offsetAndMetadata.commitTimestamp + } + ) + ) + + // call coordinator to handle commit offset + coordinator.handleCommitOffsets( + offsetCommitRequest.groupId, + offsetCommitRequest.consumerId, + offsetCommitRequest.groupGenerationId, + offsetData, + sendResponseCallback) } } /** - * Helper method for handling a parsed producer request + * Handle a produce request */ - private def appendToLocalLog(producerRequest: ProducerRequest): Iterable[ProduceResult] = { - val partitionAndData: Map[TopicAndPartition, MessageSet] = producerRequest.data - trace("Append [%s] to local log ".format(partitionAndData.toString)) - partitionAndData.map {case (topicAndPartition, messages) => - // update stats for incoming bytes rate - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) - - try { - ensureTopicExists(topicAndPartition.topic) - val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) - val info = - partitionOpt match { - case Some(partition) => partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet]) - case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" - .format(topicAndPartition, brokerId)) - - } - val numAppendedMessages = if (info.firstOffset == -1L || info.lastOffset == -1L) 0 else (info.lastOffset - info.firstOffset + 1) + def handleProducerRequest(request: RequestChannel.Request) { + val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] - // update stats for successfully appended messages - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).logBytesAppendRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.logBytesAppendRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) - BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) + // the callback for sending a produce response + def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + var errorInResponse = false + responseStatus.foreach { case (topicAndPartition, status) => + // we only print warnings for known errors here; if it is unknown, it will cause + // an error message in the replica manager + if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { + debug("Produce request with correlation id %d from client %s on partition %s failed due to %s" + .format(produceRequest.correlationId, produceRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(status.error))) + errorInResponse = true + } + } - trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" - .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) - ProduceResult(topicAndPartition, info.firstOffset, info.lastOffset) - } 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 - // for a partition it is the leader for - case e: KafkaStorageException => - fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) - Runtime.getRuntime.halt(1) - null - case utpe: UnknownTopicOrPartitionException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, utpe.getMessage)) - new ProduceResult(topicAndPartition, utpe) - case nle: NotLeaderForPartitionException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nle.getMessage)) - new ProduceResult(topicAndPartition, nle) - case e: Throwable => - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() - BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() - error("Error processing ProducerRequest with correlation id %d from client %s on partition %s" - .format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition), e) - new ProduceResult(topicAndPartition, e) - } + if (produceRequest.requiredAcks == 0) { + // no operation needed if producer request.required.acks = 0; however, if there is any error in handling + // the request, since no response is expected by the producer, the server will close socket server so that + // the producer client will know that some error has happened and will refresh its metadata + if (errorInResponse) { + info("Close connection due to error handling produce request with correlation id %d from client id %s with ack=0" + .format(produceRequest.correlationId, produceRequest.clientId)) + requestChannel.closeConnection(request.processor, request) + } else { + requestChannel.noOperation(request.processor, request) + } + } else { + val response = ProducerResponse(produceRequest.correlationId, responseStatus) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) + } } + + // only allow appending to internal topic partitions + // if the client is not from admin + val internalTopicsAllowed = produceRequest.clientId == AdminUtils.AdminClientId + + // call the replica manager to append messages to the replicas + replicaManager.appendMessages( + produceRequest.ackTimeoutMs.toLong, + produceRequest.requiredAcks, + internalTopicsAllowed, + produceRequest.data, + sendResponseCallback) + + // if the request is put into the purgatory, it will have a held reference + // and hence cannot be garbage collected; hence we clear its data here in + // order to let GC re-claim its memory since it is already appended to log + produceRequest.emptyData() } /** @@ -329,130 +305,44 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] - if(fetchRequest.isFromFollower) { - maybeUpdatePartitionHw(fetchRequest) - // after updating HW, some delayed produce requests may be unblocked - var satisfiedProduceRequests = new mutable.ArrayBuffer[DelayedProduce] - fetchRequest.requestInfo.foreach { - case (topicAndPartition, _) => - val key = new RequestKey(topicAndPartition) - satisfiedProduceRequests ++= producerRequestPurgatory.update(key, key) - } - debug("Replica %d fetch unblocked %d producer requests." - .format(fetchRequest.replicaId, satisfiedProduceRequests.size)) - satisfiedProduceRequests.foreach(_.respond()) - } - val dataRead = readMessageSets(fetchRequest) - val bytesReadable = dataRead.values.map(_.messages.sizeInBytes).sum - if(fetchRequest.maxWait <= 0 || - bytesReadable >= fetchRequest.minBytes || - fetchRequest.numPartitions <= 0) { - debug("Returning fetch response %s for fetch request with correlation id %d to client %s" - .format(dataRead.values.map(_.error).mkString(","), fetchRequest.correlationId, fetchRequest.clientId)) - val response = new FetchResponse(fetchRequest.correlationId, dataRead) - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) - } else { - debug("Putting fetch request with correlation id %d from client %s into purgatory".format(fetchRequest.correlationId, - fetchRequest.clientId)) - // create a list of (topic, partition) pairs to use as keys for this delayed request - val delayedFetchKeys = fetchRequest.requestInfo.keys.toSeq.map(new RequestKey(_)) - val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest, fetchRequest.maxWait, bytesReadable) - fetchRequestPurgatory.watch(delayedFetch) - } - } + // the callback for sending a fetch response + def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { + responsePartitionData.foreach { case (topicAndPartition, data) => + // we only print warnings for known errors here; if it is unknown, it will cause + // an error message in the replica manager already and hence can be ignored here + if (data.error != ErrorMapping.NoError && data.error != ErrorMapping.UnknownCode) { + debug("Fetch request with correlation id %d from client %s on partition %s failed due to %s" + .format(fetchRequest.correlationId, fetchRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(data.error))) + } - private def maybeUpdatePartitionHw(fetchRequest: FetchRequest) { - debug("Maybe update partition HW due to fetch request: %s ".format(fetchRequest)) - fetchRequest.requestInfo.foreach(info => { - val (topic, partition, offset) = (info._1.topic, info._1.partition, info._2.offset) - replicaManager.recordFollowerPosition(topic, partition, fetchRequest.replicaId, offset) - }) - } + // record the bytes out metrics only when the response is being sent + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesOutRate.mark(data.messages.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.messages.sizeInBytes) + } - /** - * Read from all the offset details given and return a map of - * (topic, partition) -> PartitionData - */ - private def readMessageSets(fetchRequest: FetchRequest) = { - val isFetchFromFollower = fetchRequest.isFromFollower - fetchRequest.requestInfo.map - { - case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => - val partitionData = - try { - ensureTopicExists(topic) - val (messages, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, fetchRequest.replicaId) - BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.bytesOutRate.mark(messages.sizeInBytes) - if (!isFetchFromFollower) { - new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, messages) - } else { - debug("Leader %d for partition [%s,%d] received fetch request from follower %d" - .format(brokerId, topic, partition, fetchRequest.replicaId)) - new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, messages) - } - } catch { - // NOTE: Failed fetch requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException - // since failed fetch requests metric is supposed to indicate failure of a broker in handling a fetch request - // for a partition it is the leader for - case utpe: UnknownTopicOrPartitionException => - warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( - fetchRequest.correlationId, fetchRequest.clientId, topic, partition, utpe.getMessage)) - new FetchResponsePartitionData(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) - case nle: NotLeaderForPartitionException => - warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( - fetchRequest.correlationId, fetchRequest.clientId, topic, partition, nle.getMessage)) - new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) - case t: Throwable => - BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() - BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark() - error("Error when processing fetch request for partition [%s,%d] offset %d from %s with correlation id %d" - .format(topic, partition, offset, if (isFetchFromFollower) "follower" else "consumer", fetchRequest.correlationId), t) - new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty) - } - (TopicAndPartition(topic, partition), partitionData) + val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } - } - /** - * Read from a single topic/partition at the given offset upto maxSize bytes - */ - private def readMessageSet(topic: String, - partition: Int, - offset: Long, - maxSize: Int, - fromReplicaId: Int): (MessageSet, Long) = { - // check if the current broker is the leader for the partitions - val localReplica = if(fromReplicaId == Request.DebuggingConsumerId) - replicaManager.getReplicaOrException(topic, partition) - else - replicaManager.getLeaderReplicaIfLocal(topic, partition) - trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize)) - val maxOffsetOpt = - if (Request.isReplicaIdFromFollower(fromReplicaId)) - None - else - Some(localReplica.highWatermark) - val messages = localReplica.log match { - case Some(log) => - log.read(offset, maxSize, maxOffsetOpt) - case None => - error("Leader for partition [%s,%d] on broker %d does not have a local log".format(topic, partition, brokerId)) - MessageSet.Empty - } - (messages, localReplica.highWatermark) + // call the replica manager to fetch messages from the local replica + replicaManager.fetchMessages( + fetchRequest.maxWait.toLong, + fetchRequest.replicaId, + fetchRequest.minBytes, + fetchRequest.requestInfo, + sendResponseCallback) } /** - * Service the offset request API + * Handle an offset request */ def handleOffsetRequest(request: RequestChannel.Request) { val offsetRequest = request.requestObj.asInstanceOf[OffsetRequest] val responseMap = offsetRequest.requestInfo.map(elem => { val (topicAndPartition, partitionOffsetRequestInfo) = elem try { - ensureTopicExists(topicAndPartition.topic) // ensure leader exists val localReplica = if(!offsetRequest.isFromDebuggingClient) replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) @@ -466,10 +356,10 @@ class KafkaApis(val requestChannel: RequestChannel, if (!offsetRequest.isFromOrdinaryClient) { allOffsets } else { - val hw = localReplica.highWatermark + val hw = localReplica.highWatermark.messageOffset if (allOffsets.exists(_ > hw)) hw +: allOffsets.dropWhile(_ > hw) - else + else allOffsets } } @@ -478,35 +368,35 @@ class KafkaApis(val requestChannel: RequestChannel, // NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages // are typically transient and there is no value in logging the entire stack trace for the same case utpe: UnknownTopicOrPartitionException => - warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( + debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition, utpe.getMessage)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case nle: NotLeaderForPartitionException => - warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( + debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition,nle.getMessage)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case e: Throwable => - warn("Error while responding to offset request", e) + error("Error while responding to offset request", e) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), Nil) ) } }) val response = OffsetResponse(offsetRequest.correlationId, responseMap) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } - + def fetchOffsets(logManager: LogManager, topicAndPartition: TopicAndPartition, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { logManager.getLog(topicAndPartition) match { - case Some(log) => + case Some(log) => fetchOffsetsBefore(log, timestamp, maxNumOffsets) - case None => + case None => if (timestamp == OffsetRequest.LatestTime || timestamp == OffsetRequest.EarliestTime) Seq(0L) else Nil } } - - def fetchOffsetsBefore(log: Log, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { + + private 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) @@ -547,373 +437,173 @@ class KafkaApis(val requestChannel: RequestChannel, ret.toSeq.sortBy(- _) } - /** - * Service the topic metadata request API - */ - def handleTopicMetadataRequest(request: RequestChannel.Request) { - val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] - val topicsMetadata = new mutable.ArrayBuffer[TopicMetadata]() - val config = replicaManager.config - var uniqueTopics = Set.empty[String] - uniqueTopics = { - if(metadataRequest.topics.size > 0) - metadataRequest.topics.toSet - else { - partitionMetadataLock synchronized { - metadataCache.keySet.map(_.topic) - } - } - } - val topicMetadataList = - partitionMetadataLock synchronized { - uniqueTopics.map { topic => - if(metadataCache.keySet.map(_.topic).contains(topic)) { - val partitionStateInfo = metadataCache.filter(p => p._1.topic.equals(topic)) - val sortedPartitions = partitionStateInfo.toList.sortWith((m1,m2) => m1._1.partition < m2._1.partition) - val partitionMetadata = sortedPartitions.map { case(topicAndPartition, partitionState) => - val replicas = metadataCache(topicAndPartition).allReplicas - var replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq - var leaderInfo: Option[Broker] = None - var isrInfo: Seq[Broker] = Nil - val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch - val leader = leaderIsrAndEpoch.leaderAndIsr.leader - val isr = leaderIsrAndEpoch.leaderAndIsr.isr - debug("%s".format(topicAndPartition) + ";replicas = " + replicas + ", in sync replicas = " + isr + ", leader = " + leader) - try { - if(aliveBrokers.keySet.contains(leader)) - leaderInfo = Some(aliveBrokers(leader)) - else throw new LeaderNotAvailableException("Leader not available for partition %s".format(topicAndPartition)) - isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null) - if(replicaInfo.size < replicas.size) - throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + - replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) - if(isrInfo.size < isr.size) - throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + - isr.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) - new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) - } catch { - case e: Throwable => - error("Error while fetching metadata for partition %s".format(topicAndPartition), e) - new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, - ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - } + private def getTopicMetadata(topics: Set[String], securityProtocol: SecurityProtocol): Seq[TopicMetadata] = { + val topicResponses = metadataCache.getTopicMetadata(topics, securityProtocol) + if (topics.size > 0 && topicResponses.size != topics.size) { + val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet + val responsesForNonExistentTopics = nonExistentTopics.map { topic => + if (topic == ConsumerCoordinator.OffsetsTopicName || config.autoCreateTopicsEnable) { + try { + if (topic == ConsumerCoordinator.OffsetsTopicName) { + val aliveBrokers = metadataCache.getAliveBrokers + val offsetsTopicReplicationFactor = + if (aliveBrokers.length > 0) + Math.min(config.offsetsTopicReplicationFactor.toInt, aliveBrokers.length) + else + config.offsetsTopicReplicationFactor.toInt + AdminUtils.createTopic(zkClient, topic, config.offsetsTopicPartitions, + offsetsTopicReplicationFactor, + coordinator.offsetsTopicConfigs) + info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" + .format(topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor)) } - new TopicMetadata(topic, partitionMetadata) - } else { - // topic doesn't exist, send appropriate error code - new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode) - } - } - } - - // handle auto create topics - topicMetadataList.foreach { topicMetadata => - topicMetadata.errorCode match { - case ErrorMapping.NoError => topicsMetadata += topicMetadata - case ErrorMapping.UnknownTopicOrPartitionCode => - if (config.autoCreateTopicsEnable) { - try { - AdminUtils.createTopic(zkClient, topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor) + else { + AdminUtils.createTopic(zkClient, topic, config.numPartitions, config.defaultReplicationFactor) info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" - .format(topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor)) - } catch { - case e: TopicExistsException => // let it go, possibly another broker created this topic + .format(topic, config.numPartitions, config.defaultReplicationFactor)) } - topicsMetadata += new TopicMetadata(topicMetadata.topic, topicMetadata.partitionsMetadata, ErrorMapping.LeaderNotAvailableCode) - } else { - topicsMetadata += topicMetadata + } catch { + case e: TopicExistsException => // let it go, possibly another broker created this topic } - case _ => - debug("Error while fetching topic metadata for topic %s due to %s ".format(topicMetadata.topic, - ErrorMapping.exceptionFor(topicMetadata.errorCode).getClass.getName)) - topicsMetadata += topicMetadata + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode) + } else { + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode) + } } + topicResponses.appendAll(responsesForNonExistentTopics) } - trace("Sending topic metadata %s for correlation id %d to client %s".format(topicsMetadata.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) - val response = new TopicMetadataResponse(topicsMetadata.toSeq, metadataRequest.correlationId) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + topicResponses } - /* - * Service the Offset commit API + /** + * Handle a topic metadata request */ - 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 { - ensureTopicExists(topicAndPartition.topic) - 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))) + def handleTopicMetadataRequest(request: RequestChannel.Request) { + val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] + val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet, request.securityProtocol) + val brokers = metadataCache.getAliveBrokers + trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) + val response = new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(request.securityProtocol)), topicMetadata, metadataRequest.correlationId) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } /* - * Service the Offset fetch API + * Handle an offset fetch request */ 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 { - ensureTopicExists(t.topic) - 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() - producerRequestPurgatory.shutdown() - debug("Shut down complete.") - } - - private [kafka] trait MetricKey { - def keyLabel: String - } - private [kafka] object MetricKey { - val globalLabel = "All" - } - private [kafka] case class RequestKey(topic: String, partition: Int) - extends MetricKey { + val response = if (offsetFetchRequest.versionId == 0) { + // version 0 reads offsets from ZK + val responseInfo = offsetFetchRequest.requestInfo.map( topicAndPartition => { + val topicDirs = new ZKGroupTopicDirs(offsetFetchRequest.groupId, topicAndPartition.topic) + try { + if (metadataCache.getTopicMetadata(Set(topicAndPartition.topic), request.securityProtocol).size <= 0) { + (topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition) + } else { + val payloadOpt = ZkUtils.readDataMaybeNull(zkClient, topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1 + payloadOpt match { + case Some(payload) => (topicAndPartition, OffsetMetadataAndError(payload.toLong)) + case None => (topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition) + } + } + } catch { + case e: Throwable => + (topicAndPartition, OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, + ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))) + } + }) - def this(topicAndPartition: TopicAndPartition) = this(topicAndPartition.topic, topicAndPartition.partition) + OffsetFetchResponse(collection.immutable.Map(responseInfo: _*), offsetFetchRequest.correlationId) + } else { + // version 1 reads offsets from Kafka; + val offsets = coordinator.handleFetchOffsets(offsetFetchRequest.groupId, offsetFetchRequest.requestInfo).toMap - def topicAndPartition = TopicAndPartition(topic, partition) + // Note that we do not need to filter the partitions in the + // metadata cache as the topic partitions will be filtered + // in coordinator's offset manager through the offset cache + OffsetFetchResponse(offsets, offsetFetchRequest.correlationId) + } - override def keyLabel = "%s-%d".format(topic, partition) - } + trace("Sending offset fetch response %s for correlation id %d to client %s." + .format(response, offsetFetchRequest.correlationId, offsetFetchRequest.clientId)) - /** - * A delayed fetch request - */ - class DelayedFetch(keys: Seq[RequestKey], request: RequestChannel.Request, val fetch: FetchRequest, delayMs: Long, initialSize: Long) - extends DelayedRequest(keys, request, delayMs) { - val bytesAccumulated = new AtomicLong(initialSize) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } - /** - * A holding pen for fetch requests waiting to be satisfied + /* + * Handle a consumer metadata request */ - class FetchRequestPurgatory(requestChannel: RequestChannel, purgeInterval: Int) - extends RequestPurgatory[DelayedFetch, Int](brokerId, purgeInterval) { - this.logIdent = "[FetchRequestPurgatory-%d] ".format(brokerId) - - /** - * A fetch request is satisfied when it has accumulated enough data to meet the min_bytes field - */ - def checkSatisfied(messageSizeInBytes: Int, delayedFetch: DelayedFetch): Boolean = { - val accumulatedSize = delayedFetch.bytesAccumulated.addAndGet(messageSizeInBytes) - accumulatedSize >= delayedFetch.fetch.minBytes - } + def handleConsumerMetadataRequest(request: RequestChannel.Request) { + val consumerMetadataRequest = request.requestObj.asInstanceOf[ConsumerMetadataRequest] - /** - * When a request expires just answer it with whatever data is present - */ - def expire(delayed: DelayedFetch) { - debug("Expiring fetch request %s.".format(delayed.fetch)) - try { - val topicData = readMessageSets(delayed.fetch) - val response = FetchResponse(delayed.fetch.correlationId, topicData) - val fromFollower = delayed.fetch.isFromFollower - delayedRequestMetrics.recordDelayedFetchExpired(fromFollower) - requestChannel.sendResponse(new RequestChannel.Response(delayed.request, new FetchResponseSend(response))) - } - catch { - case e1: LeaderNotAvailableException => - debug("Leader changed before fetch request %s expired.".format(delayed.fetch)) - case e2: UnknownTopicOrPartitionException => - debug("Replica went offline before fetch request %s expired.".format(delayed.fetch)) - } - } - } + val partition = coordinator.partitionFor(consumerMetadataRequest.group) - class DelayedProduce(keys: Seq[RequestKey], - request: RequestChannel.Request, - initialErrorsAndOffsets: Map[TopicAndPartition, ProducerResponseStatus], - val produce: ProducerRequest, - delayMs: Long) - extends DelayedRequest(keys, request, delayMs) with Logging { - - /** - * Map of (topic, partition) -> partition status - * The values in this map don't need to be synchronized since updates to the - * values are effectively synchronized by the ProducerRequestPurgatory's - * update method - */ - private [kafka] val partitionStatus = keys.map(requestKey => { - val producerResponseStatus = initialErrorsAndOffsets(TopicAndPartition(requestKey.topic, requestKey.partition)) - // if there was an error in writing to the local replica's log, then don't - // wait for acks on this partition - val (acksPending, error, nextOffset) = - if (producerResponseStatus.error == ErrorMapping.NoError) { - // Timeout error state will be cleared when requiredAcks are received - (true, ErrorMapping.RequestTimedOutCode, producerResponseStatus.offset) - } - else (false, producerResponseStatus.error, producerResponseStatus.offset) - - val initialStatus = PartitionStatus(acksPending, error, nextOffset) - trace("Initial partition status for %s = %s".format(requestKey.keyLabel, initialStatus)) - (requestKey, initialStatus) - }).toMap - - def respond() { - val finalErrorsAndOffsets = initialErrorsAndOffsets.map( - status => { - val pstat = partitionStatus(new RequestKey(status._1)) - (status._1, ProducerResponseStatus(pstat.error, pstat.requiredOffset)) - }) - - val response = ProducerResponse(produce.correlationId, finalErrorsAndOffsets) - - requestChannel.sendResponse(new RequestChannel.Response( - request, new BoundedByteBufferSend(response))) - } + // get metadata (and create the topic if necessary) + val offsetsTopicMetadata = getTopicMetadata(Set(ConsumerCoordinator.OffsetsTopicName), request.securityProtocol).head - /** - * Returns true if this delayed produce request is satisfied (or more - * accurately, unblocked) -- this is the case if for every partition: - * Case A: This broker is not the leader: unblock - should return error. - * Case B: This broker is the leader: - * B.1 - If there was a localError (when writing to the local log): unblock - should return error - * B.2 - else, at least requiredAcks replicas should be caught up to this request. - * - * As partitions become acknowledged, we may be able to unblock - * DelayedFetchRequests that are pending on those partitions. - */ - def isSatisfied(followerFetchRequestKey: RequestKey) = { - val topic = followerFetchRequestKey.topic - val partitionId = followerFetchRequestKey.partition - val key = RequestKey(topic, partitionId) - val fetchPartitionStatus = partitionStatus(key) - trace("Checking producer request satisfaction for %s-%d, acksPending = %b" - .format(topic, partitionId, fetchPartitionStatus.acksPending)) - if (fetchPartitionStatus.acksPending) { - val partitionOpt = replicaManager.getPartition(topic, partitionId) - val (hasEnough, errorCode) = partitionOpt match { - case Some(partition) => - partition.checkEnoughReplicasReachOffset(fetchPartitionStatus.requiredOffset, produce.requiredAcks) - case None => - (false, ErrorMapping.UnknownTopicOrPartitionCode) - } - if (errorCode != ErrorMapping.NoError) { - fetchPartitionStatus.acksPending = false - fetchPartitionStatus.error = errorCode - } else if (hasEnough) { - fetchPartitionStatus.acksPending = false - fetchPartitionStatus.error = ErrorMapping.NoError - } - if (!fetchPartitionStatus.acksPending) { - val messageSizeInBytes = produce.topicPartitionMessageSizeMap(followerFetchRequestKey.topicAndPartition) - maybeUnblockDelayedFetchRequests(topic, partitionId, messageSizeInBytes) - } - } - - // unblocked if there are no partitions with pending acks - val satisfied = ! partitionStatus.exists(p => p._2.acksPending) - trace("Producer request satisfaction for %s-%d = %b".format(topic, partitionId, satisfied)) - satisfied - } + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) - case class PartitionStatus(var acksPending: Boolean, - var error: Short, - requiredOffset: Long) { - def setThisBrokerNotLeader() { - error = ErrorMapping.NotLeaderForPartitionCode - acksPending = false - } + val response = + offsetsTopicMetadata.partitionsMetadata.find(_.partitionId == partition).map { partitionMetadata => + partitionMetadata.leader.map { leader => + ConsumerMetadataResponse(Some(leader), ErrorMapping.NoError, consumerMetadataRequest.correlationId) + }.getOrElse(errorResponse) + }.getOrElse(errorResponse) - override def toString = - "acksPending:%b, error: %d, requiredOffset: %d".format( - acksPending, error, requiredOffset - ) - } + trace("Sending consumer metadata %s for correlation id %d to client %s." + .format(response, consumerMetadataRequest.correlationId, consumerMetadataRequest.clientId)) + requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) } - /** - * A holding pen for produce requests waiting to be satisfied. - */ - private [kafka] class ProducerRequestPurgatory(purgeInterval: Int) - extends RequestPurgatory[DelayedProduce, RequestKey](brokerId, purgeInterval) { - this.logIdent = "[ProducerRequestPurgatory-%d] ".format(brokerId) - - protected def checkSatisfied(followerFetchRequestKey: RequestKey, - delayedProduce: DelayedProduce) = - delayedProduce.isSatisfied(followerFetchRequestKey) - - /** - * Handle an expired delayed request - */ - protected def expire(delayedProduce: DelayedProduce) { - for (partitionStatus <- delayedProduce.partitionStatus if partitionStatus._2.acksPending) - delayedRequestMetrics.recordDelayedProducerKeyExpired(partitionStatus._1) - - delayedProduce.respond() - } - } + def handleJoinGroupRequest(request: RequestChannel.Request) { + import JavaConversions._ - private class DelayedRequestMetrics { - private class DelayedProducerRequestMetrics(keyLabel: String = MetricKey.globalLabel) extends KafkaMetricsGroup { - val expiredRequestMeter = newMeter(keyLabel + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } + val joinGroupRequest = request.body.asInstanceOf[JoinGroupRequest] + val respHeader = new ResponseHeader(request.header.correlationId) - - private class DelayedFetchRequestMetrics(forFollower: Boolean) extends KafkaMetricsGroup { - private val metricPrefix = if (forFollower) "Follower" else "Consumer" - - val expiredRequestMeter = newMeter(metricPrefix + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val producerRequestMetricsForKey = { - val valueFactory = (k: MetricKey) => new DelayedProducerRequestMetrics(k.keyLabel + "-") - new Pool[MetricKey, DelayedProducerRequestMetrics](Some(valueFactory)) + // the callback for sending a join-group response + def sendResponseCallback(partitions: Set[TopicAndPartition], consumerId: String, generationId: Int, errorCode: Short) { + val partitionList = partitions.map(tp => new TopicPartition(tp.topic, tp.partition)).toBuffer + val responseBody = new JoinGroupResponse(errorCode, generationId, consumerId, partitionList) + trace("Sending join group response %s for correlation id %d to client %s." + .format(responseBody, request.header.correlationId, request.header.clientId)) + requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, respHeader, responseBody))) } - private val aggregateProduceRequestMetrics = new DelayedProducerRequestMetrics + // let the coordinator to handle join-group + coordinator.handleJoinGroup( + joinGroupRequest.groupId(), + joinGroupRequest.consumerId(), + joinGroupRequest.topics().toSet, + joinGroupRequest.sessionTimeout(), + joinGroupRequest.strategy(), + sendResponseCallback) + } - private val aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true) - private val aggregateNonFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = false) + def handleHeartbeatRequest(request: RequestChannel.Request) { + val heartbeatRequest = request.body.asInstanceOf[HeartbeatRequest] + val respHeader = new ResponseHeader(request.header.correlationId) - def recordDelayedProducerKeyExpired(key: MetricKey) { - val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key) - List(keyMetrics, aggregateProduceRequestMetrics).foreach(_.expiredRequestMeter.mark()) + // the callback for sending a heartbeat response + def sendResponseCallback(errorCode: Short) { + val response = new HeartbeatResponse(errorCode) + trace("Sending heartbeat response %s for correlation id %d to client %s." + .format(response, request.header.correlationId, request.header.clientId)) + requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, respHeader, response))) } - def recordDelayedFetchExpired(forFollower: Boolean) { - val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics - else aggregateNonFollowerFetchRequestMetrics - - metrics.expiredRequestMeter.mark() - } + // let the coordinator to handle heartbeat + coordinator.handleHeartbeat( + heartbeatRequest.groupId(), + heartbeatRequest.consumerId(), + heartbeatRequest.groupGenerationId(), + sendResponseCallback) } -} + def close() { + // TODO currently closing the API is an no-op since the API no longer maintain any modules + // maybe removing the closing call in the end when KafkaAPI becomes a pure stateless layer + debug("Shut down complete.") + } +} diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala old mode 100644 new mode 100755 index 3c3aafc2b3f06..dbe170f87331f --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -5,8 +5,8 @@ * 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 + * + * 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, @@ -17,235 +17,755 @@ package kafka.server +import java.util import java.util.Properties -import kafka.message.{MessageSet, Message} + +import kafka.api.ApiVersion +import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig -import kafka.utils.{VerifiableProperties, ZKConfig, Utils} +import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} +import kafka.utils.CoreUtils +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.config.{ConfigException, AbstractConfig, ConfigDef} +import org.apache.kafka.common.metrics.MetricsReporter +import org.apache.kafka.common.protocol.SecurityProtocol +import scala.collection.{mutable, immutable, JavaConversions, Map} + +object Defaults { + /** ********* Zookeeper Configuration ***********/ + val ZkSessionTimeoutMs = 6000 + val ZkSyncTimeMs = 2000 + + /** ********* General Configuration ***********/ + val MaxReservedBrokerId = 1000 + val BrokerId = -1 + val MessageMaxBytes = 1000000 + MessageSet.LogOverhead + val NumNetworkThreads = 3 + val NumIoThreads = 8 + val BackgroundThreads = 10 + val QueuedMaxRequests = 500 + + /** ********* Socket Server Configuration ***********/ + val Port = 9092 + val HostName: String = new String("") + val SocketSendBufferBytes: Int = 100 * 1024 + val SocketReceiveBufferBytes: Int = 100 * 1024 + val SocketRequestMaxBytes: Int = 100 * 1024 * 1024 + val MaxConnectionsPerIp: Int = Int.MaxValue + val MaxConnectionsPerIpOverrides: String = "" + val ConnectionsMaxIdleMs = 10 * 60 * 1000L + + /** ********* Log Configuration ***********/ + val NumPartitions = 1 + val LogDir = "/tmp/kafka-logs" + val LogSegmentBytes = 1 * 1024 * 1024 * 1024 + val LogRollHours = 24 * 7 + val LogRollJitterHours = 0 + val LogRetentionHours = 24 * 7 + + val LogRetentionBytes = -1L + val LogCleanupIntervalMs = 5 * 60 * 1000L + val Delete = "delete" + val Compact = "compact" + val LogCleanupPolicy = Delete + val LogCleanerThreads = 1 + val LogCleanerIoMaxBytesPerSecond = Double.MaxValue + val LogCleanerDedupeBufferSize = 500 * 1024 * 1024L + val LogCleanerIoBufferSize = 512 * 1024 + val LogCleanerDedupeBufferLoadFactor = 0.9d + val LogCleanerBackoffMs = 15 * 1000 + val LogCleanerMinCleanRatio = 0.5d + val LogCleanerEnable = false + val LogCleanerDeleteRetentionMs = 24 * 60 * 60 * 1000L + val LogIndexSizeMaxBytes = 10 * 1024 * 1024 + val LogIndexIntervalBytes = 4096 + val LogFlushIntervalMessages = Long.MaxValue + val LogDeleteDelayMs = 60000 + val LogFlushSchedulerIntervalMs = Long.MaxValue + val LogFlushOffsetCheckpointIntervalMs = 60000 + val LogPreAllocateEnable = false + val NumRecoveryThreadsPerDataDir = 1 + val AutoCreateTopicsEnable = true + val MinInSyncReplicas = 1 + + /** ********* Replication configuration ***********/ + val ControllerSocketTimeoutMs = 30000 + val ControllerMessageQueueSize = Int.MaxValue + val DefaultReplicationFactor = 1 + val ReplicaLagTimeMaxMs = 10000L + val ReplicaSocketTimeoutMs = ConsumerConfig.SocketTimeout + val ReplicaSocketReceiveBufferBytes = ConsumerConfig.SocketBufferSize + val ReplicaFetchMaxBytes = ConsumerConfig.FetchSize + val ReplicaFetchWaitMaxMs = 500 + val ReplicaFetchMinBytes = 1 + val NumReplicaFetchers = 1 + val ReplicaFetchBackoffMs = 1000 + val ReplicaHighWatermarkCheckpointIntervalMs = 5000L + val FetchPurgatoryPurgeIntervalRequests = 1000 + val ProducerPurgatoryPurgeIntervalRequests = 1000 + val AutoLeaderRebalanceEnable = true + val LeaderImbalancePerBrokerPercentage = 10 + val LeaderImbalanceCheckIntervalSeconds = 300 + val UncleanLeaderElectionEnable = true + val InterBrokerSecurityProtocol = SecurityProtocol.PLAINTEXT.toString + val InterBrokerProtocolVersion = ApiVersion.latestVersion.toString + + /** ********* Controlled shutdown configuration ***********/ + val ControlledShutdownMaxRetries = 3 + val ControlledShutdownRetryBackoffMs = 5000 + val ControlledShutdownEnable = true + + /** ********* Consumer coordinator configuration ***********/ + val ConsumerMinSessionTimeoutMs = 6000 + val ConsumerMaxSessionTimeoutMs = 30000 + + /** ********* Offset management configuration ***********/ + val OffsetMetadataMaxSize = OffsetManagerConfig.DefaultMaxMetadataSize + val OffsetsLoadBufferSize = OffsetManagerConfig.DefaultLoadBufferSize + val OffsetsTopicReplicationFactor = OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor + val OffsetsTopicPartitions: Int = OffsetManagerConfig.DefaultOffsetsTopicNumPartitions + val OffsetsTopicSegmentBytes: Int = OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes + val OffsetsTopicCompressionCodec: Int = OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec.codec + val OffsetsRetentionMinutes: Int = 24 * 60 + val OffsetsRetentionCheckIntervalMs: Long = OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs + val OffsetCommitTimeoutMs = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs + val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks + + val DeleteTopicEnable = false + + val CompressionType = "producer" + + val MetricNumSamples = 2 + val MetricSampleWindowMs = 1000 + val MetricReporterClasses = "" +} -/** - * Configuration settings for the kafka server - */ -class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(props) { +object KafkaConfig { + + def main(args: Array[String]) { + System.out.println(configDef.toHtmlTable) + } + + /** ********* Zookeeper Configuration ***********/ + val ZkConnectProp = "zookeeper.connect" + val ZkSessionTimeoutMsProp = "zookeeper.session.timeout.ms" + val ZkConnectionTimeoutMsProp = "zookeeper.connection.timeout.ms" + val ZkSyncTimeMsProp = "zookeeper.sync.time.ms" + /** ********* General Configuration ***********/ + val MaxReservedBrokerIdProp = "reserved.broker.max.id" + val BrokerIdProp = "broker.id" + val MessageMaxBytesProp = "message.max.bytes" + val NumNetworkThreadsProp = "num.network.threads" + val NumIoThreadsProp = "num.io.threads" + val BackgroundThreadsProp = "background.threads" + val QueuedMaxRequestsProp = "queued.max.requests" + /** ********* Socket Server Configuration ***********/ + val PortProp = "port" + val HostNameProp = "host.name" + val ListenersProp = "listeners" + val AdvertisedHostNameProp: String = "advertised.host.name" + val AdvertisedPortProp = "advertised.port" + val AdvertisedListenersProp = "advertised.listeners" + val SocketSendBufferBytesProp = "socket.send.buffer.bytes" + val SocketReceiveBufferBytesProp = "socket.receive.buffer.bytes" + val SocketRequestMaxBytesProp = "socket.request.max.bytes" + val MaxConnectionsPerIpProp = "max.connections.per.ip" + val MaxConnectionsPerIpOverridesProp = "max.connections.per.ip.overrides" + val ConnectionsMaxIdleMsProp = "connections.max.idle.ms" + /** ********* Log Configuration ***********/ + val NumPartitionsProp = "num.partitions" + val LogDirsProp = "log.dirs" + val LogDirProp = "log.dir" + val LogSegmentBytesProp = "log.segment.bytes" + + val LogRollTimeMillisProp = "log.roll.ms" + val LogRollTimeHoursProp = "log.roll.hours" + + val LogRollTimeJitterMillisProp = "log.roll.jitter.ms" + val LogRollTimeJitterHoursProp = "log.roll.jitter.hours" + + val LogRetentionTimeMillisProp = "log.retention.ms" + val LogRetentionTimeMinutesProp = "log.retention.minutes" + val LogRetentionTimeHoursProp = "log.retention.hours" + + val LogRetentionBytesProp = "log.retention.bytes" + val LogCleanupIntervalMsProp = "log.retention.check.interval.ms" + val LogCleanupPolicyProp = "log.cleanup.policy" + val LogCleanerThreadsProp = "log.cleaner.threads" + val LogCleanerIoMaxBytesPerSecondProp = "log.cleaner.io.max.bytes.per.second" + val LogCleanerDedupeBufferSizeProp = "log.cleaner.dedupe.buffer.size" + val LogCleanerIoBufferSizeProp = "log.cleaner.io.buffer.size" + val LogCleanerDedupeBufferLoadFactorProp = "log.cleaner.io.buffer.load.factor" + val LogCleanerBackoffMsProp = "log.cleaner.backoff.ms" + val LogCleanerMinCleanRatioProp = "log.cleaner.min.cleanable.ratio" + val LogCleanerEnableProp = "log.cleaner.enable" + val LogCleanerDeleteRetentionMsProp = "log.cleaner.delete.retention.ms" + val LogIndexSizeMaxBytesProp = "log.index.size.max.bytes" + val LogIndexIntervalBytesProp = "log.index.interval.bytes" + val LogFlushIntervalMessagesProp = "log.flush.interval.messages" + val LogDeleteDelayMsProp = "log.segment.delete.delay.ms" + val LogFlushSchedulerIntervalMsProp = "log.flush.scheduler.interval.ms" + val LogFlushIntervalMsProp = "log.flush.interval.ms" + val LogFlushOffsetCheckpointIntervalMsProp = "log.flush.offset.checkpoint.interval.ms" + val LogPreAllocateProp = "log.preallocate" + val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir" + val AutoCreateTopicsEnableProp = "auto.create.topics.enable" + val MinInSyncReplicasProp = "min.insync.replicas" + /** ********* Replication configuration ***********/ + val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms" + val DefaultReplicationFactorProp = "default.replication.factor" + val ReplicaLagTimeMaxMsProp = "replica.lag.time.max.ms" + val ReplicaSocketTimeoutMsProp = "replica.socket.timeout.ms" + val ReplicaSocketReceiveBufferBytesProp = "replica.socket.receive.buffer.bytes" + val ReplicaFetchMaxBytesProp = "replica.fetch.max.bytes" + val ReplicaFetchWaitMaxMsProp = "replica.fetch.wait.max.ms" + val ReplicaFetchMinBytesProp = "replica.fetch.min.bytes" + val ReplicaFetchBackoffMsProp = "replica.fetch.backoff.ms" + val NumReplicaFetchersProp = "num.replica.fetchers" + val ReplicaHighWatermarkCheckpointIntervalMsProp = "replica.high.watermark.checkpoint.interval.ms" + val FetchPurgatoryPurgeIntervalRequestsProp = "fetch.purgatory.purge.interval.requests" + val ProducerPurgatoryPurgeIntervalRequestsProp = "producer.purgatory.purge.interval.requests" + val AutoLeaderRebalanceEnableProp = "auto.leader.rebalance.enable" + val LeaderImbalancePerBrokerPercentageProp = "leader.imbalance.per.broker.percentage" + val LeaderImbalanceCheckIntervalSecondsProp = "leader.imbalance.check.interval.seconds" + val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" + val InterBrokerSecurityProtocolProp = "security.inter.broker.protocol" + val InterBrokerProtocolVersionProp = "inter.broker.protocol.version" + /** ********* Controlled shutdown configuration ***********/ + val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries" + val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms" + val ControlledShutdownEnableProp = "controlled.shutdown.enable" + /** ********* Consumer coordinator configuration ***********/ + val ConsumerMinSessionTimeoutMsProp = "consumer.min.session.timeout.ms" + val ConsumerMaxSessionTimeoutMsProp = "consumer.max.session.timeout.ms" + /** ********* Offset management configuration ***********/ + val OffsetMetadataMaxSizeProp = "offset.metadata.max.bytes" + val OffsetsLoadBufferSizeProp = "offsets.load.buffer.size" + val OffsetsTopicReplicationFactorProp = "offsets.topic.replication.factor" + val OffsetsTopicPartitionsProp = "offsets.topic.num.partitions" + val OffsetsTopicSegmentBytesProp = "offsets.topic.segment.bytes" + val OffsetsTopicCompressionCodecProp = "offsets.topic.compression.codec" + val OffsetsRetentionMinutesProp = "offsets.retention.minutes" + val OffsetsRetentionCheckIntervalMsProp = "offsets.retention.check.interval.ms" + val OffsetCommitTimeoutMsProp = "offsets.commit.timeout.ms" + val OffsetCommitRequiredAcksProp = "offsets.commit.required.acks" + + val DeleteTopicEnableProp = "delete.topic.enable" + val CompressionTypeProp = "compression.type" + + val MetricSampleWindowMsProp = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG + val MetricNumSamplesProp: String = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG + val MetricReporterClassesProp: String = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG + + + /* Documentation */ + /** ********* Zookeeper Configuration ***********/ + val ZkConnectDoc = "Zookeeper host string" + val ZkSessionTimeoutMsDoc = "Zookeeper session timeout" + val ZkConnectionTimeoutMsDoc = "The max time that the client waits to establish a connection to zookeeper" + val ZkSyncTimeMsDoc = "How far a ZK follower can be behind a ZK leader" + /** ********* General Configuration ***********/ + val MaxReservedBrokerIdDoc = "Max number that can be used for a broker.id" + val BrokerIdDoc = "The broker id for this server. " + + "To avoid conflicts between zookeeper generated brokerId and user's config.brokerId " + + "added MaxReservedBrokerId and zookeeper sequence starts from MaxReservedBrokerId + 1." + val MessageMaxBytesDoc = "The maximum size of message that the server can receive" + val NumNetworkThreadsDoc = "the number of network threads that the server uses for handling network requests" + val NumIoThreadsDoc = "The number of io threads that the server uses for carrying out network requests" + val BackgroundThreadsDoc = "The number of threads to use for various background processing tasks" + val QueuedMaxRequestsDoc = "The number of queued requests allowed before blocking the network threads" + /** ********* Socket Server Configuration ***********/ + val PortDoc = "the port to listen and accept connections on" + val HostNameDoc = "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" + val ListenersDoc = "Listener List - Comma-separated list of URIs we will listen on and their protocols.\n" + + " Specify hostname as 0.0.0.0 to bind to all interfaces.\n" + + " Leave hostname empty to bind to default interface.\n" + + " Examples of legal listener lists:\n" + + " PLAINTEXT://myhost:9092,TRACE://:9091\n" + + " PLAINTEXT://0.0.0.0:9092, TRACE://localhost:9093\n" + val AdvertisedHostNameDoc = "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 AdvertisedPortDoc = "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 AdvertisedListenersDoc = "Listeners to publish to ZooKeeper for clients to use, if different than the listeners above." + + " In IaaS environments, this may need to be different from the interface to which the broker binds." + + " If this is not set, the value for \"listeners\" will be used." + val SocketSendBufferBytesDoc = "The SO_SNDBUF buffer of the socket sever sockets" + val SocketReceiveBufferBytesDoc = "The SO_RCVBUF buffer of the socket sever sockets" + val SocketRequestMaxBytesDoc = "The maximum number of bytes in a socket request" + val MaxConnectionsPerIpDoc = "The maximum number of connections we allow from each ip address" + val MaxConnectionsPerIpOverridesDoc = "Per-ip or hostname overrides to the default maximum number of connections" + val ConnectionsMaxIdleMsDoc = "Idle connections timeout: the server socket processor threads close the connections that idle more than this" + /** ********* Log Configuration ***********/ + val NumPartitionsDoc = "The default number of log partitions per topic" + val LogDirDoc = "The directory in which the log data is kept (supplemental for " + LogDirsProp + " property)" + val LogDirsDoc = "The directories in which the log data is kept" + val LogSegmentBytesDoc = "The maximum size of a single log file" + val LogRollTimeMillisDoc = "The maximum time before a new log segment is rolled out (in milliseconds)" + val LogRollTimeHoursDoc = "The maximum time before a new log segment is rolled out (in hours), secondary to " + LogRollTimeMillisProp + " property" + + val LogRollTimeJitterMillisDoc = "The maximum jitter to subtract from logRollTimeMillis (in milliseconds)" + val LogRollTimeJitterHoursDoc = "The maximum jitter to subtract from logRollTimeMillis (in hours), secondary to " + LogRollTimeJitterMillisProp + " property" + + val LogRetentionTimeMillisDoc = "The number of milliseconds to keep a log file before deleting it (in milliseconds)" + val LogRetentionTimeMinsDoc = "The number of minutes to keep a log file before deleting it (in minutes), secondary to " + LogRetentionTimeMillisProp + " property" + val LogRetentionTimeHoursDoc = "The number of hours to keep a log file before deleting it (in hours), tertiary to " + LogRetentionTimeMillisProp + " property" + + val LogRetentionBytesDoc = "The maximum size of the log before deleting it" + val LogCleanupIntervalMsDoc = "The frequency in minutes that the log cleaner checks whether any log is eligible for deletion" + val LogCleanupPolicyDoc = "The default cleanup policy for segments beyond the retention window, must be either \"delete\" or \"compact\"" + val LogCleanerThreadsDoc = "The number of background threads to use for log cleaning" + val LogCleanerIoMaxBytesPerSecondDoc = "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 LogCleanerDedupeBufferSizeDoc = "The total memory used for log deduplication across all cleaner threads" + val LogCleanerIoBufferSizeDoc = "The total memory used for log cleaner I/O buffers across all cleaner threads" + val LogCleanerDedupeBufferLoadFactorDoc = "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 LogCleanerBackoffMsDoc = "The amount of time to sleep when there are no logs to clean" + val LogCleanerMinCleanRatioDoc = "The minimum ratio of dirty log to total log for a log to eligible for cleaning" + val LogCleanerEnableDoc = "Should we enable log cleaning?" + val LogCleanerDeleteRetentionMsDoc = "How long are delete records retained?" + val LogIndexSizeMaxBytesDoc = "The maximum size in bytes of the offset index" + val LogIndexIntervalBytesDoc = "The interval with which we add an entry to the offset index" + val LogFlushIntervalMessagesDoc = "The number of messages accumulated on a log partition before messages are flushed to disk " + val LogDeleteDelayMsDoc = "The amount of time to wait before deleting a file from the filesystem" + val LogFlushSchedulerIntervalMsDoc = "The frequency in ms that the log flusher checks whether any log needs to be flushed to disk" + val LogFlushIntervalMsDoc = "The maximum time in ms that a message in any topic is kept in memory before flushed to disk" + val LogFlushOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point" + val LogPreAllocateEnableDoc = "Should pre allocate file when create new segment? If you are using Kafka on Windows, you probably need to set it to true." + val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown" + val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server" + val MinInSyncReplicasDoc = "define the minimum number of replicas in ISR needed to satisfy a produce request with required.acks=-1 (or all)" + /** ********* Replication configuration ***********/ + val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels" + val ControllerMessageQueueSizeDoc = "The buffer size for controller-to-broker-channels" + val DefaultReplicationFactorDoc = "default replication factors for automatically created topics" + val ReplicaLagTimeMaxMsDoc = "If a follower hasn't sent any fetch requests or hasn't consumed up to the leaders log end offset for at least this time," + + " the leader will remove the follower from isr" + val ReplicaSocketTimeoutMsDoc = "The socket timeout for network requests. Its value should be at least replica.fetch.wait.max.ms" + val ReplicaSocketReceiveBufferBytesDoc = "The socket receive buffer for network requests" + val ReplicaFetchMaxBytesDoc = "The number of byes of messages to attempt to fetch" + val ReplicaFetchWaitMaxMsDoc = "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 ReplicaFetchMinBytesDoc = "Minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs" + val NumReplicaFetchersDoc = "Number of fetcher threads used to replicate messages from a source broker. " + + "Increasing this value can increase the degree of I/O parallelism in the follower broker." + val ReplicaFetchBackoffMsDoc = "The amount of time to sleep when fetch partition error occurs." + val ReplicaHighWatermarkCheckpointIntervalMsDoc = "The frequency with which the high watermark is saved out to disk" + val FetchPurgatoryPurgeIntervalRequestsDoc = "The purge interval (in number of requests) of the fetch request purgatory" + val ProducerPurgatoryPurgeIntervalRequestsDoc = "The purge interval (in number of requests) of the producer request purgatory" + val AutoLeaderRebalanceEnableDoc = "Enables auto leader balancing. A background thread checks and triggers leader balance if required at regular intervals" + val LeaderImbalancePerBrokerPercentageDoc = "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 LeaderImbalanceCheckIntervalSecondsDoc = "The frequency with which the partition rebalance check is triggered by the controller" + val UncleanLeaderElectionEnableDoc = "Indicates whether to enable replicas not in the ISR set to be elected as leader as a last resort, even though doing so may result in data loss" + val InterBrokerSecurityProtocolDoc = "Security protocol used to communicate between brokers. Defaults to plain text." + val InterBrokerProtocolVersionDoc = "Specify which version of the inter-broker protocol will be used.\n" + + " This is typically bumped after all brokers were upgraded to a new version.\n" + + " Example of some valid values are: 0.8.0, 0.8.1, 0.8.1.1, 0.8.2, 0.8.2.0, 0.8.2.1, 0.8.3, 0.8.3.0. Check ApiVersion for the full list." + /** ********* Controlled shutdown configuration ***********/ + val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens" + val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." + val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server" + /** ********* Consumer coordinator configuration ***********/ + val ConsumerMinSessionTimeoutMsDoc = "The minimum allowed session timeout for registered consumers" + val ConsumerMaxSessionTimeoutMsDoc = "The maximum allowed session timeout for registered consumers" + /** ********* Offset management configuration ***********/ + val OffsetMetadataMaxSizeDoc = "The maximum size for a metadata entry associated with an offset commit" + val OffsetsLoadBufferSizeDoc = "Batch size for reading from the offsets segments when loading offsets into the cache." + val OffsetsTopicReplicationFactorDoc = "The replication factor for the offsets topic (set higher to ensure availability). " + + "To ensure that the effective replication factor of the offsets topic is the configured value, " + + "the number of alive brokers has to be at least the replication factor at the time of the " + + "first request for the offsets topic. If not, either the offsets topic creation will fail or " + + "it will get a replication factor of min(alive brokers, configured replication factor)" + val OffsetsTopicPartitionsDoc = "The number of partitions for the offset commit topic (should not change after deployment)" + val OffsetsTopicSegmentBytesDoc = "The offsets topic segment bytes should be kept relatively small in order to facilitate faster log compaction and cache loads" + val OffsetsTopicCompressionCodecDoc = "Compression codec for the offsets topic - compression may be used to achieve \"atomic\" commits" + val OffsetsRetentionMinutesDoc = "Log retention window in minutes for offsets topic" + val OffsetsRetentionCheckIntervalMsDoc = "Frequency at which to check for stale offsets" + val OffsetCommitTimeoutMsDoc = "Offset commit will be delayed until all replicas for the offsets topic receive the commit " + + "or this timeout is reached. This is similar to the producer request timeout." + val OffsetCommitRequiredAcksDoc = "The required acks before the commit can be accepted. In general, the default (-1) should not be overridden" + val DeleteTopicEnableDoc = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off" + val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + + "('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + + "'producer' which means retain the original compression codec set by the producer." + + val MetricSampleWindowMsDoc = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC + val MetricNumSamplesDoc = CommonClientConfigs.METRICS_NUM_SAMPLES_DOC + val MetricReporterClassesDoc = CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC + + + private val configDef = { + import ConfigDef.Range._ + import ConfigDef.ValidString._ + import ConfigDef.Type._ + import ConfigDef.Importance._ + + new ConfigDef() + + /** ********* Zookeeper Configuration ***********/ + .define(ZkConnectProp, STRING, HIGH, ZkConnectDoc) + .define(ZkSessionTimeoutMsProp, INT, Defaults.ZkSessionTimeoutMs, HIGH, ZkSessionTimeoutMsDoc) + .define(ZkConnectionTimeoutMsProp, INT, HIGH, ZkConnectionTimeoutMsDoc, false) + .define(ZkSyncTimeMsProp, INT, Defaults.ZkSyncTimeMs, LOW, ZkSyncTimeMsDoc) + + /** ********* General Configuration ***********/ + .define(MaxReservedBrokerIdProp, INT, Defaults.MaxReservedBrokerId, atLeast(0), MEDIUM, MaxReservedBrokerIdProp) + .define(BrokerIdProp, INT, Defaults.BrokerId, HIGH, BrokerIdDoc) + .define(MessageMaxBytesProp, INT, Defaults.MessageMaxBytes, atLeast(0), HIGH, MessageMaxBytesDoc) + .define(NumNetworkThreadsProp, INT, Defaults.NumNetworkThreads, atLeast(1), HIGH, NumNetworkThreadsDoc) + .define(NumIoThreadsProp, INT, Defaults.NumIoThreads, atLeast(1), HIGH, NumIoThreadsDoc) + .define(BackgroundThreadsProp, INT, Defaults.BackgroundThreads, atLeast(1), HIGH, BackgroundThreadsDoc) + .define(QueuedMaxRequestsProp, INT, Defaults.QueuedMaxRequests, atLeast(1), HIGH, QueuedMaxRequestsDoc) + + /** ********* Socket Server Configuration ***********/ + .define(PortProp, INT, Defaults.Port, HIGH, PortDoc) + .define(HostNameProp, STRING, Defaults.HostName, HIGH, HostNameDoc) + .define(ListenersProp, STRING, HIGH, ListenersDoc, false) + .define(AdvertisedHostNameProp, STRING, HIGH, AdvertisedHostNameDoc, false) + .define(AdvertisedPortProp, INT, HIGH, AdvertisedPortDoc, false) + .define(AdvertisedListenersProp, STRING, HIGH, AdvertisedListenersDoc, false) + .define(SocketSendBufferBytesProp, INT, Defaults.SocketSendBufferBytes, HIGH, SocketSendBufferBytesDoc) + .define(SocketReceiveBufferBytesProp, INT, Defaults.SocketReceiveBufferBytes, HIGH, SocketReceiveBufferBytesDoc) + .define(SocketRequestMaxBytesProp, INT, Defaults.SocketRequestMaxBytes, atLeast(1), HIGH, SocketRequestMaxBytesDoc) + .define(MaxConnectionsPerIpProp, INT, Defaults.MaxConnectionsPerIp, atLeast(1), MEDIUM, MaxConnectionsPerIpDoc) + .define(MaxConnectionsPerIpOverridesProp, STRING, Defaults.MaxConnectionsPerIpOverrides, MEDIUM, MaxConnectionsPerIpOverridesDoc) + .define(ConnectionsMaxIdleMsProp, LONG, Defaults.ConnectionsMaxIdleMs, MEDIUM, ConnectionsMaxIdleMsDoc) + + /** ********* Log Configuration ***********/ + .define(NumPartitionsProp, INT, Defaults.NumPartitions, atLeast(1), MEDIUM, NumPartitionsDoc) + .define(LogDirProp, STRING, Defaults.LogDir, HIGH, LogDirDoc) + .define(LogDirsProp, STRING, HIGH, LogDirsDoc, false) + .define(LogSegmentBytesProp, INT, Defaults.LogSegmentBytes, atLeast(Message.MinHeaderSize), HIGH, LogSegmentBytesDoc) + + .define(LogRollTimeMillisProp, LONG, HIGH, LogRollTimeMillisDoc, false) + .define(LogRollTimeHoursProp, INT, Defaults.LogRollHours, atLeast(1), HIGH, LogRollTimeHoursDoc) + + .define(LogRollTimeJitterMillisProp, LONG, HIGH, LogRollTimeJitterMillisDoc, false) + .define(LogRollTimeJitterHoursProp, INT, Defaults.LogRollJitterHours, atLeast(0), HIGH, LogRollTimeJitterHoursDoc) + + .define(LogRetentionTimeMillisProp, LONG, HIGH, LogRetentionTimeMillisDoc, false) + .define(LogRetentionTimeMinutesProp, INT, HIGH, LogRetentionTimeMinsDoc, false) + .define(LogRetentionTimeHoursProp, INT, Defaults.LogRetentionHours, HIGH, LogRetentionTimeHoursDoc) + + .define(LogRetentionBytesProp, LONG, Defaults.LogRetentionBytes, HIGH, LogRetentionBytesDoc) + .define(LogCleanupIntervalMsProp, LONG, Defaults.LogCleanupIntervalMs, atLeast(1), MEDIUM, LogCleanupIntervalMsDoc) + .define(LogCleanupPolicyProp, STRING, Defaults.LogCleanupPolicy, in(Defaults.Compact, Defaults.Delete), MEDIUM, LogCleanupPolicyDoc) + .define(LogCleanerThreadsProp, INT, Defaults.LogCleanerThreads, atLeast(0), MEDIUM, LogCleanerThreadsDoc) + .define(LogCleanerIoMaxBytesPerSecondProp, DOUBLE, Defaults.LogCleanerIoMaxBytesPerSecond, MEDIUM, LogCleanerIoMaxBytesPerSecondDoc) + .define(LogCleanerDedupeBufferSizeProp, LONG, Defaults.LogCleanerDedupeBufferSize, MEDIUM, LogCleanerDedupeBufferSizeDoc) + .define(LogCleanerIoBufferSizeProp, INT, Defaults.LogCleanerIoBufferSize, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc) + .define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LogCleanerDedupeBufferLoadFactor, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc) + .define(LogCleanerBackoffMsProp, LONG, Defaults.LogCleanerBackoffMs, atLeast(0), MEDIUM, LogCleanerBackoffMsDoc) + .define(LogCleanerMinCleanRatioProp, DOUBLE, Defaults.LogCleanerMinCleanRatio, MEDIUM, LogCleanerMinCleanRatioDoc) + .define(LogCleanerEnableProp, BOOLEAN, Defaults.LogCleanerEnable, MEDIUM, LogCleanerEnableDoc) + .define(LogCleanerDeleteRetentionMsProp, LONG, Defaults.LogCleanerDeleteRetentionMs, MEDIUM, LogCleanerDeleteRetentionMsDoc) + .define(LogIndexSizeMaxBytesProp, INT, Defaults.LogIndexSizeMaxBytes, atLeast(4), MEDIUM, LogIndexSizeMaxBytesDoc) + .define(LogIndexIntervalBytesProp, INT, Defaults.LogIndexIntervalBytes, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc) + .define(LogFlushIntervalMessagesProp, LONG, Defaults.LogFlushIntervalMessages, atLeast(1), HIGH, LogFlushIntervalMessagesDoc) + .define(LogDeleteDelayMsProp, LONG, Defaults.LogDeleteDelayMs, atLeast(0), HIGH, LogDeleteDelayMsDoc) + .define(LogFlushSchedulerIntervalMsProp, LONG, Defaults.LogFlushSchedulerIntervalMs, HIGH, LogFlushSchedulerIntervalMsDoc) + .define(LogFlushIntervalMsProp, LONG, HIGH, LogFlushIntervalMsDoc, false) + .define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LogFlushOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc) + .define(LogPreAllocateProp, BOOLEAN, Defaults.LogPreAllocateEnable, MEDIUM, LogPreAllocateEnableDoc) + .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) + .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc) + .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc) + + /** ********* Replication configuration ***********/ + .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) + .define(DefaultReplicationFactorProp, INT, Defaults.DefaultReplicationFactor, MEDIUM, DefaultReplicationFactorDoc) + .define(ReplicaLagTimeMaxMsProp, LONG, Defaults.ReplicaLagTimeMaxMs, HIGH, ReplicaLagTimeMaxMsDoc) + .define(ReplicaSocketTimeoutMsProp, INT, Defaults.ReplicaSocketTimeoutMs, HIGH, ReplicaSocketTimeoutMsDoc) + .define(ReplicaSocketReceiveBufferBytesProp, INT, Defaults.ReplicaSocketReceiveBufferBytes, HIGH, ReplicaSocketReceiveBufferBytesDoc) + .define(ReplicaFetchMaxBytesProp, INT, Defaults.ReplicaFetchMaxBytes, HIGH, ReplicaFetchMaxBytesDoc) + .define(ReplicaFetchWaitMaxMsProp, INT, Defaults.ReplicaFetchWaitMaxMs, HIGH, ReplicaFetchWaitMaxMsDoc) + .define(ReplicaFetchBackoffMsProp, INT, Defaults.ReplicaFetchBackoffMs, atLeast(0), MEDIUM, ReplicaFetchBackoffMsDoc) + .define(ReplicaFetchMinBytesProp, INT, Defaults.ReplicaFetchMinBytes, HIGH, ReplicaFetchMinBytesDoc) + .define(NumReplicaFetchersProp, INT, Defaults.NumReplicaFetchers, HIGH, NumReplicaFetchersDoc) + .define(ReplicaHighWatermarkCheckpointIntervalMsProp, LONG, Defaults.ReplicaHighWatermarkCheckpointIntervalMs, HIGH, ReplicaHighWatermarkCheckpointIntervalMsDoc) + .define(FetchPurgatoryPurgeIntervalRequestsProp, INT, Defaults.FetchPurgatoryPurgeIntervalRequests, MEDIUM, FetchPurgatoryPurgeIntervalRequestsDoc) + .define(ProducerPurgatoryPurgeIntervalRequestsProp, INT, Defaults.ProducerPurgatoryPurgeIntervalRequests, MEDIUM, ProducerPurgatoryPurgeIntervalRequestsDoc) + .define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AutoLeaderRebalanceEnable, HIGH, AutoLeaderRebalanceEnableDoc) + .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LeaderImbalancePerBrokerPercentage, HIGH, LeaderImbalancePerBrokerPercentageDoc) + .define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LeaderImbalanceCheckIntervalSeconds, HIGH, LeaderImbalanceCheckIntervalSecondsDoc) + .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, HIGH, UncleanLeaderElectionEnableDoc) + .define(InterBrokerSecurityProtocolProp, STRING, Defaults.InterBrokerSecurityProtocol, MEDIUM, InterBrokerSecurityProtocolDoc) + .define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, MEDIUM, InterBrokerProtocolVersionDoc) + + /** ********* Controlled shutdown configuration ***********/ + .define(ControlledShutdownMaxRetriesProp, INT, Defaults.ControlledShutdownMaxRetries, MEDIUM, ControlledShutdownMaxRetriesDoc) + .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) + .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc) + + /** ********* Consumer coordinator configuration ***********/ + .define(ConsumerMinSessionTimeoutMsProp, INT, Defaults.ConsumerMinSessionTimeoutMs, MEDIUM, ConsumerMinSessionTimeoutMsDoc) + .define(ConsumerMaxSessionTimeoutMsProp, INT, Defaults.ConsumerMaxSessionTimeoutMs, MEDIUM, ConsumerMaxSessionTimeoutMsDoc) + + /** ********* Offset management configuration ***********/ + .define(OffsetMetadataMaxSizeProp, INT, Defaults.OffsetMetadataMaxSize, HIGH, OffsetMetadataMaxSizeDoc) + .define(OffsetsLoadBufferSizeProp, INT, Defaults.OffsetsLoadBufferSize, atLeast(1), HIGH, OffsetsLoadBufferSizeDoc) + .define(OffsetsTopicReplicationFactorProp, SHORT, Defaults.OffsetsTopicReplicationFactor, atLeast(1), HIGH, OffsetsTopicReplicationFactorDoc) + .define(OffsetsTopicPartitionsProp, INT, Defaults.OffsetsTopicPartitions, atLeast(1), HIGH, OffsetsTopicPartitionsDoc) + .define(OffsetsTopicSegmentBytesProp, INT, Defaults.OffsetsTopicSegmentBytes, atLeast(1), HIGH, OffsetsTopicSegmentBytesDoc) + .define(OffsetsTopicCompressionCodecProp, INT, Defaults.OffsetsTopicCompressionCodec, HIGH, OffsetsTopicCompressionCodecDoc) + .define(OffsetsRetentionMinutesProp, INT, Defaults.OffsetsRetentionMinutes, atLeast(1), HIGH, OffsetsRetentionMinutesDoc) + .define(OffsetsRetentionCheckIntervalMsProp, LONG, Defaults.OffsetsRetentionCheckIntervalMs, atLeast(1), HIGH, OffsetsRetentionCheckIntervalMsDoc) + .define(OffsetCommitTimeoutMsProp, INT, Defaults.OffsetCommitTimeoutMs, atLeast(1), HIGH, OffsetCommitTimeoutMsDoc) + .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) + .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) + .define(CompressionTypeProp, STRING, Defaults.CompressionType, HIGH, CompressionTypeDoc) + .define(MetricNumSamplesProp, INT, Defaults.MetricNumSamples, atLeast(1), LOW, MetricNumSamplesDoc) + .define(MetricSampleWindowMsProp, LONG, Defaults.MetricSampleWindowMs, atLeast(1), LOW, MetricSampleWindowMsDoc) + .define(MetricReporterClassesProp, LIST, Defaults.MetricReporterClasses, LOW, MetricReporterClassesDoc) + } + + def configNames() = { + import scala.collection.JavaConversions._ + configDef.names().toList.sorted + } + + /** + * Check that property names are valid + */ + def validateNames(props: Properties) { + import JavaConversions._ + val names = configDef.names() + for (name <- props.keys) + require(names.contains(name), "Unknown configuration \"%s\".".format(name)) + } + + def fromProps(props: Properties): KafkaConfig = { + KafkaConfig(props) + } - def this(originalProps: Properties) { - this(new VerifiableProperties(originalProps)) - props.verify() + def fromProps(defaults: Properties, overrides: Properties): KafkaConfig = { + val props = new Properties() + props.putAll(defaults) + props.putAll(overrides) + fromProps(props) } - - private def getLogRetentionTimeMillis(): Long = { - var millisInMinute = 60L * 1000L + +} + +case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(KafkaConfig.configDef, props) { + + /** ********* Zookeeper Configuration ***********/ + val zkConnect: String = getString(KafkaConfig.ZkConnectProp) + val zkSessionTimeoutMs: Int = getInt(KafkaConfig.ZkSessionTimeoutMsProp) + val zkConnectionTimeoutMs: java.lang.Integer = + Option(getInt(KafkaConfig.ZkConnectionTimeoutMsProp)).getOrElse(getInt(KafkaConfig.ZkSessionTimeoutMsProp)) + val zkSyncTimeMs: Int = getInt(KafkaConfig.ZkSyncTimeMsProp) + + /** ********* General Configuration ***********/ + val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) + var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) + val numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp) + val backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp) + val queuedMaxRequests = getInt(KafkaConfig.QueuedMaxRequestsProp) + val numIoThreads = getInt(KafkaConfig.NumIoThreadsProp) + val messageMaxBytes = getInt(KafkaConfig.MessageMaxBytesProp) + + /** ********* Socket Server Configuration ***********/ + val hostName = getString(KafkaConfig.HostNameProp) + val port = getInt(KafkaConfig.PortProp) + val advertisedHostName = Option(getString(KafkaConfig.AdvertisedHostNameProp)).getOrElse(hostName) + val advertisedPort: java.lang.Integer = Option(getInt(KafkaConfig.AdvertisedPortProp)).getOrElse(port) + + val socketSendBufferBytes = getInt(KafkaConfig.SocketSendBufferBytesProp) + val socketReceiveBufferBytes = getInt(KafkaConfig.SocketReceiveBufferBytesProp) + val socketRequestMaxBytes = getInt(KafkaConfig.SocketRequestMaxBytesProp) + val maxConnectionsPerIp = getInt(KafkaConfig.MaxConnectionsPerIpProp) + val maxConnectionsPerIpOverrides: Map[String, Int] = + getMap(KafkaConfig.MaxConnectionsPerIpOverridesProp, getString(KafkaConfig.MaxConnectionsPerIpOverridesProp)).map { case (k, v) => (k, v.toInt)} + val connectionsMaxIdleMs = getLong(KafkaConfig.ConnectionsMaxIdleMsProp) + + + /** ********* Log Configuration ***********/ + val autoCreateTopicsEnable = getBoolean(KafkaConfig.AutoCreateTopicsEnableProp) + val numPartitions = getInt(KafkaConfig.NumPartitionsProp) + val logDirs = CoreUtils.parseCsvList( Option(getString(KafkaConfig.LogDirsProp)).getOrElse(getString(KafkaConfig.LogDirProp))) + val logSegmentBytes = getInt(KafkaConfig.LogSegmentBytesProp) + val logFlushIntervalMessages = getLong(KafkaConfig.LogFlushIntervalMessagesProp) + val logCleanerThreads = getInt(KafkaConfig.LogCleanerThreadsProp) + val numRecoveryThreadsPerDataDir = getInt(KafkaConfig.NumRecoveryThreadsPerDataDirProp) + val logFlushSchedulerIntervalMs = getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp) + val logFlushOffsetCheckpointIntervalMs = getInt(KafkaConfig.LogFlushOffsetCheckpointIntervalMsProp).toLong + val logCleanupIntervalMs = getLong(KafkaConfig.LogCleanupIntervalMsProp) + val logCleanupPolicy = getString(KafkaConfig.LogCleanupPolicyProp) + val offsetsRetentionMinutes = getInt(KafkaConfig.OffsetsRetentionMinutesProp) + val offsetsRetentionCheckIntervalMs = getLong(KafkaConfig.OffsetsRetentionCheckIntervalMsProp) + val logRetentionBytes = getLong(KafkaConfig.LogRetentionBytesProp) + val logCleanerDedupeBufferSize = getLong(KafkaConfig.LogCleanerDedupeBufferSizeProp) + val logCleanerDedupeBufferLoadFactor = getDouble(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp) + val logCleanerIoBufferSize = getInt(KafkaConfig.LogCleanerIoBufferSizeProp) + val logCleanerIoMaxBytesPerSecond = getDouble(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp) + val logCleanerDeleteRetentionMs = getLong(KafkaConfig.LogCleanerDeleteRetentionMsProp) + val logCleanerBackoffMs = getLong(KafkaConfig.LogCleanerBackoffMsProp) + val logCleanerMinCleanRatio = getDouble(KafkaConfig.LogCleanerMinCleanRatioProp) + val logCleanerEnable = getBoolean(KafkaConfig.LogCleanerEnableProp) + val logIndexSizeMaxBytes = getInt(KafkaConfig.LogIndexSizeMaxBytesProp) + val logIndexIntervalBytes = getInt(KafkaConfig.LogIndexIntervalBytesProp) + val logDeleteDelayMs = getLong(KafkaConfig.LogDeleteDelayMsProp) + val logRollTimeMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeHoursProp)) + val logRollTimeJitterMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeJitterMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeJitterHoursProp)) + val logFlushIntervalMs: java.lang.Long = Option(getLong(KafkaConfig.LogFlushIntervalMsProp)).getOrElse(getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp)) + val minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp) + val logPreAllocateEnable: Boolean = getBoolean(KafkaConfig.LogPreAllocateProp) + + /** ********* Replication configuration ***********/ + val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp) + val defaultReplicationFactor: Int = getInt(KafkaConfig.DefaultReplicationFactorProp) + val replicaLagTimeMaxMs = getLong(KafkaConfig.ReplicaLagTimeMaxMsProp) + val replicaSocketTimeoutMs = getInt(KafkaConfig.ReplicaSocketTimeoutMsProp) + val replicaSocketReceiveBufferBytes = getInt(KafkaConfig.ReplicaSocketReceiveBufferBytesProp) + val replicaFetchMaxBytes = getInt(KafkaConfig.ReplicaFetchMaxBytesProp) + val replicaFetchWaitMaxMs = getInt(KafkaConfig.ReplicaFetchWaitMaxMsProp) + val replicaFetchMinBytes = getInt(KafkaConfig.ReplicaFetchMinBytesProp) + val replicaFetchBackoffMs = getInt(KafkaConfig.ReplicaFetchBackoffMsProp) + val numReplicaFetchers = getInt(KafkaConfig.NumReplicaFetchersProp) + val replicaHighWatermarkCheckpointIntervalMs = getLong(KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp) + val fetchPurgatoryPurgeIntervalRequests = getInt(KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp) + val producerPurgatoryPurgeIntervalRequests = getInt(KafkaConfig.ProducerPurgatoryPurgeIntervalRequestsProp) + val autoLeaderRebalanceEnable = getBoolean(KafkaConfig.AutoLeaderRebalanceEnableProp) + val leaderImbalancePerBrokerPercentage = getInt(KafkaConfig.LeaderImbalancePerBrokerPercentageProp) + val leaderImbalanceCheckIntervalSeconds = getLong(KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp) + val uncleanLeaderElectionEnable = getBoolean(KafkaConfig.UncleanLeaderElectionEnableProp) + val interBrokerSecurityProtocol = SecurityProtocol.valueOf(getString(KafkaConfig.InterBrokerSecurityProtocolProp)) + val interBrokerProtocolVersion = ApiVersion(getString(KafkaConfig.InterBrokerProtocolVersionProp)) + + /** ********* Controlled shutdown configuration ***********/ + val controlledShutdownMaxRetries = getInt(KafkaConfig.ControlledShutdownMaxRetriesProp) + val controlledShutdownRetryBackoffMs = getLong(KafkaConfig.ControlledShutdownRetryBackoffMsProp) + val controlledShutdownEnable = getBoolean(KafkaConfig.ControlledShutdownEnableProp) + + /** ********* Consumer coordinator configuration ***********/ + val consumerMinSessionTimeoutMs = getInt(KafkaConfig.ConsumerMinSessionTimeoutMsProp) + val consumerMaxSessionTimeoutMs = getInt(KafkaConfig.ConsumerMaxSessionTimeoutMsProp) + + /** ********* Offset management configuration ***********/ + val offsetMetadataMaxSize = getInt(KafkaConfig.OffsetMetadataMaxSizeProp) + val offsetsLoadBufferSize = getInt(KafkaConfig.OffsetsLoadBufferSizeProp) + val offsetsTopicReplicationFactor = getShort(KafkaConfig.OffsetsTopicReplicationFactorProp) + val offsetsTopicPartitions = getInt(KafkaConfig.OffsetsTopicPartitionsProp) + val offsetCommitTimeoutMs = getInt(KafkaConfig.OffsetCommitTimeoutMsProp) + val offsetCommitRequiredAcks = getShort(KafkaConfig.OffsetCommitRequiredAcksProp) + val offsetsTopicSegmentBytes = getInt(KafkaConfig.OffsetsTopicSegmentBytesProp) + val offsetsTopicCompressionCodec = Option(getInt(KafkaConfig.OffsetsTopicCompressionCodecProp)).map(value => CompressionCodec.getCompressionCodec(value)).orNull + + /** ********* Metric Configuration **************/ + val metricNumSamples = getInt(KafkaConfig.MetricNumSamplesProp) + val metricSampleWindowMs = getLong(KafkaConfig.MetricSampleWindowMsProp) + val metricReporterClasses: java.util.List[MetricsReporter] = getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter]) + + val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) + val compressionType = getString(KafkaConfig.CompressionTypeProp) + + + val listeners = getListeners + val advertisedListeners = getAdvertisedListeners + val logRetentionTimeMillis = getLogRetentionTimeMillis + + private def getLogRetentionTimeMillis: Long = { + val millisInMinute = 60L * 1000L val millisInHour = 60L * millisInMinute - if(props.containsKey("log.retention.minutes")){ - millisInMinute * props.getIntInRange("log.retention.minutes", (1, Int.MaxValue)) + + val millis: java.lang.Long = + Option(getLong(KafkaConfig.LogRetentionTimeMillisProp)).getOrElse( + Option(getInt(KafkaConfig.LogRetentionTimeMinutesProp)) match { + case Some(mins) => millisInMinute * mins + case None => getInt(KafkaConfig.LogRetentionTimeHoursProp) * millisInHour + }) + + if (millis < 0) return -1 + millis + } + + private def getMap(propName: String, propValue: String): Map[String, String] = { + try { + CoreUtils.parseCsvMap(propValue) + } catch { + case e: Exception => throw new IllegalArgumentException("Error parsing configuration property '%s': %s".format(propName, e.getMessage)) + } + } + + private def validateUniquePortAndProtocol(listeners: String) { + + val endpoints = try { + val listenerList = CoreUtils.parseCsvList(listeners) + listenerList.map(listener => EndPoint.createEndPoint(listener)) + } catch { + case e: Exception => throw new IllegalArgumentException("Error creating broker listeners from '%s': %s".format(listeners, e.getMessage)) + } + val distinctPorts = endpoints.map(ep => ep.port).distinct + val distinctProtocols = endpoints.map(ep => ep.protocolType).distinct + + require(distinctPorts.size == endpoints.size, "Each listener must have a different port") + require(distinctProtocols.size == endpoints.size, "Each listener must have a different protocol") + } + + // If the user did not define listeners but did define host or port, let's use them in backward compatible way + // If none of those are defined, we default to PLAINTEXT://:9092 + private def getListeners(): immutable.Map[SecurityProtocol, EndPoint] = { + if (getString(KafkaConfig.ListenersProp) != null) { + validateUniquePortAndProtocol(getString(KafkaConfig.ListenersProp)) + CoreUtils.listenerListToEndPoints(getString(KafkaConfig.ListenersProp)) } else { - millisInHour * props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue)) + CoreUtils.listenerListToEndPoints("PLAINTEXT://" + hostName + ":" + port) } - } - /*********** General Configuration ***********/ - - /* the broker id for this server */ - val brokerId: Int = props.getIntInRange("broker.id", (0, Int.MaxValue)) - - /* the maximum size of message that the server can receive */ - val messageMaxBytes = props.getIntInRange("message.max.bytes", 1000000 + MessageSet.LogOverhead, (0, Int.MaxValue)) - - /* the number of network threads that the server uses for handling network requests */ - val numNetworkThreads = props.getIntInRange("num.network.threads", 3, (1, Int.MaxValue)) - - /* 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)) - - /*********** Socket Server Configuration ***********/ - - /* the port to listen and accept connections on */ - 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 */ - 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) - - /* the SO_RCVBUFF buffer of the socket sever sockets */ - val socketReceiveBufferBytes: Int = props.getInt("socket.receive.buffer.bytes", 100*1024) - - /* the maximum number of bytes in a socket request */ - val socketRequestMaxBytes: Int = props.getIntInRange("socket.request.max.bytes", 100*1024*1024, (1, Int.MaxValue)) - - /*********** Log Configuration ***********/ - - /* the default number of log partitions per topic */ - val numPartitions = props.getIntInRange("num.partitions", 1, (1, Int.MaxValue)) - - /* the directories in which the log data is kept */ - val logDirs = Utils.parseCsvList(props.getString("log.dirs", props.getString("log.dir", "/tmp/kafka-logs"))) - require(logDirs.size > 0) - - /* 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 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 to keep a log file before deleting it */ - val logRetentionTimeMillis = getLogRetentionTimeMillis + // If the user defined advertised listeners, we use those + // If he didn't but did define advertised host or port, we'll use those and fill in the missing value from regular host / port or defaults + // If none of these are defined, we'll use the listeners + private def getAdvertisedListeners(): immutable.Map[SecurityProtocol, EndPoint] = { + if (getString(KafkaConfig.AdvertisedListenersProp) != null) { + validateUniquePortAndProtocol(getString(KafkaConfig.AdvertisedListenersProp)) + CoreUtils.listenerListToEndPoints(getString(KafkaConfig.AdvertisedListenersProp)) + } else if (getString(KafkaConfig.AdvertisedHostNameProp) != null || getInt(KafkaConfig.AdvertisedPortProp) != null) { + CoreUtils.listenerListToEndPoints("PLAINTEXT://" + advertisedHostName + ":" + advertisedPort) + } else { + getListeners() + } + } - /* the maximum size of the log before deleting it */ - val logRetentionBytes = props.getLong("log.retention.bytes", -1) - - /* the frequency in minutes that the log cleaner checks whether any log is eligible for deletion */ - 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)) - - /* the interval with which we add an entry to the offset index */ - 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.getLongInRange("log.flush.interval.messages", Long.MaxValue, (1, Long.MaxValue)) - - /* 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.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.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) - - /*********** Replication configuration ***********/ - - /* the socket timeout for controller-to-broker channels */ - val controllerSocketTimeoutMs = props.getInt("controller.socket.timeout.ms", 30000) - - /* the buffer size for controller-to-broker-channels */ - val controllerMessageQueueSize= props.getInt("controller.message.queue.size", 10) - - /* default replication factors for automatically created topics */ - val defaultReplicationFactor = props.getInt("default.replication.factor", 1) - - /* If a follower hasn't sent any fetch requests during this time, the leader will remove the follower from isr */ - val replicaLagTimeMaxMs = props.getLong("replica.lag.time.max.ms", 10000) - - /* If the lag in messages between a leader and a follower exceeds this number, the leader will remove the follower from isr */ - val replicaLagMaxMessages = props.getLong("replica.lag.max.messages", 4000) - - /* the socket timeout for network requests */ - val replicaSocketTimeoutMs = props.getInt("replica.socket.timeout.ms", ConsumerConfig.SocketTimeout) - - /* the socket receive buffer for network requests */ - val replicaSocketReceiveBufferBytes = props.getInt("replica.socket.receive.buffer.bytes", ConsumerConfig.SocketBufferSize) - - /* the number of byes of messages to attempt to fetch */ - val replicaFetchMaxBytes = props.getIntInRange("replica.fetch.max.bytes", ConsumerConfig.FetchSize, (messageMaxBytes, Int.MaxValue)) - - /* 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) - - /* number of fetcher threads used to replicate messages from a source broker. - * Increasing this value can increase the degree of I/O parallelism in the follower broker. */ - val numReplicaFetchers = props.getInt("num.replica.fetchers", 1) - - /* the frequency with which the high watermark is saved out to disk */ - val replicaHighWatermarkCheckpointIntervalMs = props.getLong("replica.high.watermark.checkpoint.interval.ms", 5000L) - - /* the purge interval (in number of requests) of the fetch request purgatory */ - val fetchPurgatoryPurgeIntervalRequests = props.getInt("fetch.purgatory.purge.interval.requests", 10000) - - /* 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 ***********/ + private def getMetricClasses(metricClasses: java.util.List[String]): java.util.List[MetricsReporter] = { - /** Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens */ - val controlledShutdownMaxRetries = props.getInt("controlled.shutdown.max.retries", 3) + val reporterList = new util.ArrayList[MetricsReporter](); + val iterator = metricClasses.iterator() - /** Before each retry, the system needs time to recover from the state that caused the previous failure (Controller - * fail over, replica lag etc). This config determines the amount of time to wait before retrying. */ - val controlledShutdownRetryBackoffMs = props.getInt("controlled.shutdown.retry.backoff.ms", 5000) + while (iterator.hasNext) { + val reporterName = iterator.next() + if (!reporterName.isEmpty) { + val reporter: MetricsReporter = CoreUtils.createObject[MetricsReporter](reporterName) + reporter.configure(originals) + reporterList.add(reporter) + } + } - /* enable controlled shutdown of the server */ - val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", false) + reporterList - /*********** Misc configuration ***********/ - - /* the maximum size for a metadata entry associated with an offset commit */ - val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", 1024) + } + + validateValues() + + private def validateValues() { + require(brokerId >= -1 && brokerId <= maxReservedBrokerId, "broker.id must be equal or greater than -1 and not greater than reserved.broker.max.id") + require(logRollTimeMillis >= 1, "log.roll.ms must be equal or greater than 1") + require(logRollTimeJitterMillis >= 0, "log.roll.jitter.ms must be equal or greater than 0") + require(logRetentionTimeMillis >= 1 || logRetentionTimeMillis == -1, "log.retention.ms must be unlimited (-1) or, equal or greater than 1") + require(logDirs.size > 0) + require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.") + require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" + + " to prevent unnecessary socket timeouts") + require(replicaFetchMaxBytes >= messageMaxBytes, "replica.fetch.max.bytes should be equal or greater than message.max.bytes") + require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs, "replica.fetch.wait.max.ms should always be at least replica.lag.time.max.ms" + + " to prevent frequent changes in ISR") + require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor, + "offsets.commit.required.acks must be greater or equal -1 and less or equal to offsets.topic.replication.factor") + require(BrokerCompressionCodec.isValid(compressionType), "compression.type : " + compressionType + " is not valid." + + " Valid options are " + BrokerCompressionCodec.brokerCompressionOptions.mkString(",")) + } } diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 8c69d095bfa9f..ea0c996ae5d1c 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -17,7 +17,9 @@ package kafka.server +import kafka.cluster.EndPoint import kafka.utils._ +import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkStateListener, ZkClient} import java.net.InetAddress @@ -31,16 +33,16 @@ import java.net.InetAddress * Right now our definition of health is fairly naive. If we register in zk we are healthy, otherwise * we are dead. */ -class KafkaHealthcheck(private val brokerId: Int, - private val advertisedHost: String, - private val advertisedPort: Int, +class KafkaHealthcheck(private val brokerId: Int, + private val advertisedEndpoints: Map[SecurityProtocol, EndPoint], private val zkSessionTimeoutMs: Int, private val zkClient: ZkClient) extends Logging { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId - + val sessionExpireListener = new SessionExpireListener + def startup() { - zkClient.subscribeStateChanges(new SessionExpireListener) + zkClient.subscribeStateChanges(sessionExpireListener) register() } @@ -48,13 +50,19 @@ class KafkaHealthcheck(private val brokerId: Int, * Register this broker as "alive" in zookeeper */ def register() { - val advertisedHostName = - if(advertisedHost == null || advertisedHost.trim.isEmpty) - InetAddress.getLocalHost.getCanonicalHostName - else - advertisedHost val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt - ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedHostName, advertisedPort, zkSessionTimeoutMs, jmxPort) + val updatedEndpoints = advertisedEndpoints.mapValues(endpoint => + if (endpoint.host == null || endpoint.host.trim.isEmpty) + EndPoint(InetAddress.getLocalHost.getCanonicalHostName, endpoint.port, endpoint.protocolType) + else + endpoint + ) + + // the default host and port are here for compatibility with older client + // only PLAINTEXT is supported as default + // if the broker doesn't listen on PLAINTEXT protocol, an empty endpoint will be registered and older clients will be unable to connect + val plaintextEndpoint = updatedEndpoints.getOrElse(SecurityProtocol.PLAINTEXT, new EndPoint(null,-1,null)) + ZkUtils.registerBrokerInZk(zkClient, brokerId, plaintextEndpoint.host, plaintextEndpoint.port, updatedEndpoints, zkSessionTimeoutMs, jmxPort) } /** @@ -81,6 +89,11 @@ class KafkaHealthcheck(private val brokerId: Int, info("done re-registering broker") info("Subscribing to %s path to watch for new topics".format(ZkUtils.BrokerTopicsPath)) } + + override def handleSessionEstablishmentError(error: Throwable): Unit = { + fatal("Could not establish session with zookeeper", error) + System.exit(-1) + } } } diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala old mode 100644 new mode 100755 index 871212bf9110c..a1558afed20bc --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -21,17 +21,35 @@ import kafka.network._ import kafka.utils._ import kafka.metrics.KafkaMetricsGroup import java.util.concurrent.TimeUnit +import com.yammer.metrics.core.Meter +import org.apache.kafka.common.utils.Utils /** * A thread that answers kafka requests. */ -class KafkaRequestHandler(id: Int, brokerId: Int, val requestChannel: RequestChannel, apis: KafkaApis) extends Runnable with Logging { +class KafkaRequestHandler(id: Int, + brokerId: Int, + val aggregateIdleMeter: Meter, + val totalHandlerThreads: Int, + val requestChannel: RequestChannel, + apis: KafkaApis) extends Runnable with Logging { this.logIdent = "[Kafka Request Handler " + id + " on Broker " + brokerId + "], " - def run() { + def run() { while(true) { try { - val req = requestChannel.receiveRequest() + var req : RequestChannel.Request = null + while (req == null) { + // We use a single meter for aggregate idle percentage for the thread pool. + // Since meter is calculated as total_recorded_value / time_window and + // time_window is independent of the number of threads, each recorded idle + // time should be discounted by # threads. + val startSelectTime = SystemTime.nanoseconds + req = requestChannel.receiveRequest(300) + val idleTime = SystemTime.nanoseconds - startSelectTime + aggregateIdleMeter.mark(idleTime / totalHandlerThreads) + } + if(req eq RequestChannel.AllDone) { debug("Kafka request handler %d on broker %d received shut down command".format( id, brokerId)) @@ -52,12 +70,16 @@ class KafkaRequestHandler(id: Int, brokerId: Int, val requestChannel: RequestCha class KafkaRequestHandlerPool(val brokerId: Int, val requestChannel: RequestChannel, val apis: KafkaApis, - numThreads: Int) extends Logging { + numThreads: Int) extends Logging with KafkaMetricsGroup { + + /* a meter to track the average free capacity of the request handlers */ + private val aggregateIdleMeter = newMeter("RequestHandlerAvgIdlePercent", "percent", TimeUnit.NANOSECONDS) + this.logIdent = "[Kafka Request Handler on Broker " + brokerId + "], " val threads = new Array[Thread](numThreads) val runnables = new Array[KafkaRequestHandler](numThreads) for(i <- 0 until numThreads) { - runnables(i) = new KafkaRequestHandler(i, brokerId, requestChannel, apis) + runnables(i) = new KafkaRequestHandler(i, brokerId, aggregateIdleMeter, numThreads, requestChannel, apis) threads(i) = Utils.daemonThread("kafka-request-handler-" + i, runnables(i)) threads(i).start() } @@ -72,23 +94,30 @@ class KafkaRequestHandlerPool(val brokerId: Int, } } -class BrokerTopicMetrics(name: String) extends KafkaMetricsGroup { - val messagesInRate = newMeter(name + "MessagesInPerSec", "messages", TimeUnit.SECONDS) - val bytesInRate = newMeter(name + "BytesInPerSec", "bytes", TimeUnit.SECONDS) - val bytesOutRate = newMeter(name + "BytesOutPerSec", "bytes", TimeUnit.SECONDS) - val logBytesAppendRate = newMeter(name + "LogBytesAppendedPerSec", "bytes", TimeUnit.SECONDS) - val failedProduceRequestRate = newMeter(name + "FailedProduceRequestsPerSec", "requests", TimeUnit.SECONDS) - val failedFetchRequestRate = newMeter(name + "FailedFetchRequestsPerSec", "requests", TimeUnit.SECONDS) +class BrokerTopicMetrics(name: Option[String]) extends KafkaMetricsGroup { + val tags: scala.collection.Map[String, String] = name match { + case None => scala.collection.Map.empty + case Some(topic) => Map("topic" -> topic) + } + + val messagesInRate = newMeter("MessagesInPerSec", "messages", TimeUnit.SECONDS, tags) + val bytesInRate = newMeter("BytesInPerSec", "bytes", TimeUnit.SECONDS, tags) + val bytesOutRate = newMeter("BytesOutPerSec", "bytes", TimeUnit.SECONDS, tags) + val bytesRejectedRate = newMeter("BytesRejectedPerSec", "bytes", TimeUnit.SECONDS, tags) + val failedProduceRequestRate = newMeter("FailedProduceRequestsPerSec", "requests", TimeUnit.SECONDS, tags) + val failedFetchRequestRate = newMeter("FailedFetchRequestsPerSec", "requests", TimeUnit.SECONDS, tags) + val totalProduceRequestRate = newMeter("TotalProduceRequestsPerSec", "requests", TimeUnit.SECONDS, tags) + val totalFetchRequestRate = newMeter("TotalFetchRequestsPerSec", "requests", TimeUnit.SECONDS, tags) } object BrokerTopicStats extends Logging { - private val valueFactory = (k: String) => new BrokerTopicMetrics(k) + private val valueFactory = (k: String) => new BrokerTopicMetrics(Some(k)) private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory)) - private val allTopicsStats = new BrokerTopicMetrics("AllTopics") + private val allTopicsStats = new BrokerTopicMetrics(None) def getBrokerAllTopicsStats(): BrokerTopicMetrics = allTopicsStats def getBrokerTopicStats(topic: String): BrokerTopicMetrics = { - stats.getAndMaybePut(topic + "-") + stats.getAndMaybePut(topic) } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala old mode 100644 new mode 100755 index 5e34f95e64eaf..18917bc4464b9 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -17,104 +17,216 @@ package kafka.server +import java.util +import java.util.Properties + 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 kafka.utils._ +import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.network.NetworkReceive + +import scala.collection.{JavaConversions, mutable} import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} -import kafka.cluster.Broker +import kafka.cluster.{EndPoint, Broker} import kafka.api.{ControlledShutdownResponse, ControlledShutdownRequest} -import kafka.common.ErrorMapping -import kafka.network.{Receive, BlockingChannel, SocketServer} +import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBrokerIdException} +import kafka.network.{BlockingChannel, SocketServer} +import kafka.metrics.KafkaMetricsGroup +import com.yammer.metrics.core.Gauge +import kafka.coordinator.{GroupManagerConfig, ConsumerCoordinator} /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required * to start up and shutdown a single Kafka node. */ -class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logging { - this.logIdent = "[Kafka Server " + config.brokerId + "], " - private var isShuttingDown = new AtomicBoolean(false) +class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logging with KafkaMetricsGroup { + private val startupComplete = new AtomicBoolean(false) + private val isShuttingDown = new AtomicBoolean(false) + private val isStartingUp = new AtomicBoolean(false) + private var shutdownLatch = new CountDownLatch(1) - private var startupComplete = new AtomicBoolean(false); - val correlationId: AtomicInteger = new AtomicInteger(0) + + private val metricConfig: MetricConfig = new MetricConfig() + .samples(config.metricNumSamples) + .timeWindow(config.metricSampleWindowMs, TimeUnit.MILLISECONDS) + private val jmxPrefix: String = "kafka.server" + private val reporters: java.util.List[MetricsReporter] = config.metricReporterClasses + reporters.add(new JmxReporter(jmxPrefix)) + + + + // This exists so SocketServer (which uses Client libraries) can use the client Time objects without having to convert all of Kafka to use them + // Once we get rid of kafka.utils.time, we can get rid of this too + private val socketServerTime: org.apache.kafka.common.utils.Time = new org.apache.kafka.common.utils.SystemTime() + + val brokerState: BrokerState = new BrokerState + + var apis: KafkaApis = null var socketServer: SocketServer = null var requestHandlerPool: KafkaRequestHandlerPool = null + var logManager: LogManager = null - var kafkaHealthcheck: KafkaHealthcheck = null - var topicConfigManager: TopicConfigManager = null + var replicaManager: ReplicaManager = null - var apis: KafkaApis = null + + var topicConfigManager: TopicConfigManager = null + + var consumerCoordinator: ConsumerCoordinator = null + var kafkaController: KafkaController = null + val kafkaScheduler = new KafkaScheduler(config.backgroundThreads) + + var kafkaHealthcheck: KafkaHealthcheck = null + val metadataCache: MetadataCache = new MetadataCache(config.brokerId) + + + var zkClient: ZkClient = null + val correlationId: AtomicInteger = new AtomicInteger(0) + val brokerMetaPropsFile = "meta.properties" + val brokerMetadataCheckpoints = config.logDirs.map(logDir => (logDir, new BrokerMetadataCheckpoint(new File(logDir + File.separator +brokerMetaPropsFile)))).toMap + + newGauge( + "BrokerState", + new Gauge[Int] { + def value = brokerState.currentState + } + ) /** * 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") - isShuttingDown = new AtomicBoolean(false) - shutdownLatch = new CountDownLatch(1) - - /* start scheduler */ - kafkaScheduler.startup() - - /* setup zookeeper */ - zkClient = initZk() - - /* start log manager */ - logManager = createLogManager(zkClient) - logManager.startup() - - socketServer = new SocketServer(config.brokerId, - config.hostName, - config.port, - config.numNetworkThreads, - config.queuedMaxRequests, - config.socketSendBufferBytes, - config.socketReceiveBufferBytes, - config.socketRequestMaxBytes) - socketServer.startup() - - 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() - - replicaManager.startup() - - kafkaController.startup() - - topicConfigManager = new TopicConfigManager(zkClient, logManager) - topicConfigManager.startup() - - /* tell everyone we are alive */ - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) - kafkaHealthcheck.startup() - - - registerStats() - startupComplete.set(true); - info("started") + try { + info("starting") + + if(isShuttingDown.get) + throw new IllegalStateException("Kafka server is still shutting down, cannot re-start!") + + if(startupComplete.get) + return + + val canStartup = isStartingUp.compareAndSet(false, true) + if (canStartup) { + brokerState.newState(Starting) + + /* start scheduler */ + kafkaScheduler.startup() + + /* setup zookeeper */ + zkClient = initZk() + + /* start log manager */ + logManager = createLogManager(zkClient, brokerState) + logManager.startup() + + /* generate brokerId */ + config.brokerId = getBrokerId + this.logIdent = "[Kafka Server " + config.brokerId + "], " + + val metrics = new Metrics(metricConfig, reporters, socketServerTime) + + + socketServer = new SocketServer(config.brokerId, + config.listeners, + config.numNetworkThreads, + config.queuedMaxRequests, + config.socketSendBufferBytes, + config.socketReceiveBufferBytes, + config.socketRequestMaxBytes, + config.maxConnectionsPerIp, + config.connectionsMaxIdleMs, + config.maxConnectionsPerIpOverrides, + socketServerTime, + metrics) + socketServer.startup() + + /* start replica manager */ + replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) + replicaManager.startup() + + /* start kafka controller */ + kafkaController = new KafkaController(config, zkClient, brokerState) + kafkaController.startup() + + /* start kafka coordinator */ + consumerCoordinator = ConsumerCoordinator.create(config, zkClient, replicaManager, kafkaScheduler) + consumerCoordinator.startup() + + /* start processing requests */ + apis = new KafkaApis(socketServer.requestChannel, replicaManager, consumerCoordinator, + kafkaController, zkClient, config.brokerId, config, metadataCache) + requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) + brokerState.newState(RunningAsBroker) + + Mx4jLoader.maybeLoad() + + /* start topic config manager */ + topicConfigManager = new TopicConfigManager(zkClient, logManager) + topicConfigManager.startup() + + /* tell everyone we are alive */ + val listeners = config.advertisedListeners.map {case(protocol, endpoint) => + if (endpoint.port == 0) + (protocol, EndPoint(endpoint.host, socketServer.boundPort(), endpoint.protocolType)) + else + (protocol, endpoint) + } + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, listeners, config.zkSessionTimeoutMs, zkClient) + kafkaHealthcheck.startup() + + /* register broker metrics */ + registerStats() + + shutdownLatch = new CountDownLatch(1) + startupComplete.set(true) + isStartingUp.set(false) + info("started") + } + } + catch { + case e: Throwable => + fatal("Fatal error during KafkaServer startup. Prepare to shutdown", e) + isStartingUp.set(false) + shutdown() + throw e + } } - + private def initZk(): ZkClient = { info("Connecting to zookeeper on " + config.zkConnect) - val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + + val chroot = { + if (config.zkConnect.indexOf("/") > 0) + config.zkConnect.substring(config.zkConnect.indexOf("/")) + else + "" + } + + if (chroot.length > 1) { + val zkConnForChrootCreation = config.zkConnect.substring(0, config.zkConnect.indexOf("/")) + val zkClientForChrootCreation = ZkUtils.createZkClient(zkConnForChrootCreation, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) + ZkUtils.makeSurePersistentPathExists(zkClientForChrootCreation, chroot) + info("Created zookeeper path " + chroot) + zkClientForChrootCreation.close() + } + + val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) ZkUtils.setupCommonPaths(zkClient) zkClient } + /** * Forces some dynamic jmx beans to be registered on server startup. */ @@ -134,11 +246,12 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg // the shutdown. var remainingRetries = config.controlledShutdownMaxRetries info("Starting controlled shutdown") - var channel : BlockingChannel = null; + var channel : BlockingChannel = null var prevController : Broker = null - var shutdownSuceeded : Boolean =false + var shutdownSucceeded : Boolean = false try { - while (!shutdownSuceeded && remainingRetries > 0) { + brokerState.newState(PendingControlledShutdown) + while (!shutdownSucceeded && remainingRetries > 0) { remainingRetries = remainingRetries - 1 // 1. Find the controller and establish a connection to it. @@ -154,7 +267,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (channel != null) { channel.disconnect() } - channel = new BlockingChannel(broker.host, broker.port, + channel = new BlockingChannel(broker.getBrokerEndPoint(config.interBrokerSecurityProtocol).host, + broker.getBrokerEndPoint(config.interBrokerSecurityProtocol).port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) @@ -167,16 +281,17 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg // 2. issue a controlled shutdown to the controller if (channel != null) { - var response: Receive = null + var response: NetworkReceive = null try { // send the controlled shutdown request val request = new ControlledShutdownRequest(correlationId.getAndIncrement, config.brokerId) channel.send(request) + response = channel.receive() - val shutdownResponse = ControlledShutdownResponse.readFrom(response.buffer) + val shutdownResponse = ControlledShutdownResponse.readFrom(response.payload()) if (shutdownResponse.errorCode == ErrorMapping.NoError && shutdownResponse.partitionsRemaining != null && shutdownResponse.partitionsRemaining.size == 0) { - shutdownSuceeded = true + shutdownSucceeded = true info ("Controlled shutdown succeeded") } else { @@ -188,10 +303,11 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg case ioe: java.io.IOException => channel.disconnect() channel = null + warn("Error during controlled shutdown, possibly because leader movement took longer than the configured socket.timeout.ms: %s".format(ioe.getMessage)) // ignore and try again } } - if (!shutdownSuceeded) { + if (!shutdownSucceeded) { Thread.sleep(config.controlledShutdownRetryBackoffMs) warn("Retrying controlled shutdown after the previous attempt failed...") } @@ -203,7 +319,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg channel = null } } - if (!shutdownSuceeded) { + if (!shutdownSucceeded) { warn("Proceeding to do an unclean shutdown as all the controlled shutdown attempts failed") } } @@ -214,29 +330,47 @@ 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") - val canShutdown = isShuttingDown.compareAndSet(false, true); - if (canShutdown) { - Utils.swallow(controlledShutdown()) - if(socketServer != null) - Utils.swallow(socketServer.shutdown()) - if(requestHandlerPool != null) - Utils.swallow(requestHandlerPool.shutdown()) - Utils.swallow(kafkaScheduler.shutdown()) - if(apis != null) - Utils.swallow(apis.close()) - if(replicaManager != null) - 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") + try { + info("shutting down") + + if(isStartingUp.get) + throw new IllegalStateException("Kafka server is still starting up, cannot shut down!") + + val canShutdown = isShuttingDown.compareAndSet(false, true) + if (canShutdown && shutdownLatch.getCount > 0) { + CoreUtils.swallow(controlledShutdown()) + brokerState.newState(BrokerShuttingDown) + if(socketServer != null) + CoreUtils.swallow(socketServer.shutdown()) + if(requestHandlerPool != null) + CoreUtils.swallow(requestHandlerPool.shutdown()) + CoreUtils.swallow(kafkaScheduler.shutdown()) + if(apis != null) + CoreUtils.swallow(apis.close()) + if(replicaManager != null) + CoreUtils.swallow(replicaManager.shutdown()) + if(logManager != null) + CoreUtils.swallow(logManager.shutdown()) + if(consumerCoordinator != null) + CoreUtils.swallow(consumerCoordinator.shutdown()) + if(kafkaController != null) + CoreUtils.swallow(kafkaController.shutdown()) + if(zkClient != null) + CoreUtils.swallow(zkClient.close()) + + brokerState.newState(NotRunning) + + startupComplete.set(false) + isShuttingDown.set(false) + shutdownLatch.countDown() + info("shut down completed") + } + } + catch { + case e: Throwable => + fatal("Fatal error during KafkaServer shutdown.", e) + isShuttingDown.set(false) + throw e } } @@ -246,22 +380,13 @@ 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 + + def boundPort(): Int = socketServer.boundPort() + + private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = { + val defaultProps = copyKafkaConfigToLog(config.originals) + val defaultLogConfig = LogConfig(defaultProps) + val configs = AdminUtils.fetchAllTopicConfigs(zkClient).mapValues(LogConfig.fromProps(defaultProps, _)) // read the log configurations from zookeeper val cleanerConfig = CleanerConfig(numThreads = config.logCleanerThreads, @@ -276,13 +401,97 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg topicConfigs = configs, defaultConfig = defaultLogConfig, cleanerConfig = cleanerConfig, + ioThreads = config.numRecoveryThreadsPerDataDir, flushCheckMs = config.logFlushSchedulerIntervalMs, flushCheckpointMs = config.logFlushOffsetCheckpointIntervalMs, retentionCheckMs = config.logCleanupIntervalMs, scheduler = kafkaScheduler, + brokerState = brokerState, time = time) } -} + // Copy the subset of properties that are relevant to Logs + // I'm listing out individual properties here since the names are slightly different in each Config class... + private def copyKafkaConfigToLog(serverProps: java.util.Map[String, Object]): java.util.Map[String, Object] = { + + val logProps = new util.HashMap[String, Object]() + val entryset = serverProps.entrySet.iterator + while (entryset.hasNext) { + val entry = entryset.next + entry.getKey match { + case KafkaConfig.LogSegmentBytesProp => logProps.put(LogConfig.SegmentBytesProp, entry.getValue) + case KafkaConfig.LogRollTimeMillisProp => logProps.put(LogConfig.SegmentMsProp, entry.getValue) + case KafkaConfig.LogRollTimeJitterMillisProp => logProps.put(LogConfig.SegmentJitterMsProp, entry.getValue) + case KafkaConfig.LogIndexSizeMaxBytesProp => logProps.put(LogConfig.SegmentIndexBytesProp, entry.getValue) + case KafkaConfig.LogFlushIntervalMessagesProp => logProps.put(LogConfig.FlushMessagesProp, entry.getValue) + case KafkaConfig.LogFlushIntervalMsProp => logProps.put(LogConfig.FlushMsProp, entry.getValue) + case KafkaConfig.LogRetentionBytesProp => logProps.put(LogConfig.RetentionBytesProp, entry.getValue) + case KafkaConfig.LogRetentionTimeMillisProp => logProps.put(LogConfig.RetentionMsProp, entry.getValue) + case KafkaConfig.MessageMaxBytesProp => logProps.put(LogConfig.MaxMessageBytesProp, entry.getValue) + case KafkaConfig.LogIndexIntervalBytesProp => logProps.put(LogConfig.IndexIntervalBytesProp, entry.getValue) + case KafkaConfig.LogCleanerDeleteRetentionMsProp => logProps.put(LogConfig.DeleteRetentionMsProp, entry.getValue) + case KafkaConfig.LogDeleteDelayMsProp => logProps.put(LogConfig.FileDeleteDelayMsProp, entry.getValue) + case KafkaConfig.LogCleanerMinCleanRatioProp => logProps.put(LogConfig.MinCleanableDirtyRatioProp, entry.getValue) + case KafkaConfig.LogCleanupPolicyProp => logProps.put(LogConfig.CleanupPolicyProp, entry.getValue) + case KafkaConfig.MinInSyncReplicasProp => logProps.put(LogConfig.MinInSyncReplicasProp, entry.getValue) + case KafkaConfig.CompressionTypeProp => logProps.put(LogConfig.CompressionTypeProp, entry.getValue) + case KafkaConfig.UncleanLeaderElectionEnableProp => logProps.put(LogConfig.UncleanLeaderElectionEnableProp, entry.getValue) + case KafkaConfig.LogPreAllocateProp => logProps.put(LogConfig.PreAllocateEnableProp, entry.getValue) + case _ => // we just leave those out + } + } + logProps + } + + /** + * Generates new brokerId or reads from meta.properties based on following conditions + *
          + *
        1. config has no broker.id provided , generates a broker.id based on Zookeeper's sequence + *
        2. stored broker.id in meta.properties doesn't match in all the log.dirs throws InconsistentBrokerIdException + *
        3. config has broker.id and meta.properties contains broker.id if they don't match throws InconsistentBrokerIdException + *
        4. config has broker.id and there is no meta.properties file, creates new meta.properties and stores broker.id + *
            + * @return A brokerId. + */ + private def getBrokerId: Int = { + var brokerId = config.brokerId + var logDirsWithoutMetaProps: List[String] = List() + val brokerIdSet = mutable.HashSet[Int]() + + for (logDir <- config.logDirs) { + val brokerMetadataOpt = brokerMetadataCheckpoints(logDir).read() + brokerMetadataOpt match { + case Some(brokerMetadata: BrokerMetadata) => + brokerIdSet.add(brokerMetadata.brokerId) + case None => + logDirsWithoutMetaProps ++= List(logDir) + } + } + + if(brokerIdSet.size > 1) + throw new InconsistentBrokerIdException("Failed to match brokerId across logDirs") + else if(brokerId >= 0 && brokerIdSet.size == 1 && brokerIdSet.last != brokerId) + throw new InconsistentBrokerIdException("Configured brokerId %s doesn't match stored brokerId %s in meta.properties".format(brokerId, brokerIdSet.last)) + else if(brokerIdSet.size == 0 && brokerId < 0) // generate a new brokerId from Zookeeper + brokerId = generateBrokerId + else if(brokerIdSet.size == 1) // pick broker.id from meta.properties + brokerId = brokerIdSet.last + for(logDir <- logDirsWithoutMetaProps) { + val checkpoint = brokerMetadataCheckpoints(logDir) + checkpoint.write(new BrokerMetadata(brokerId)) + } + brokerId + } + + private def generateBrokerId: Int = { + try { + ZkUtils.getBrokerSequenceId(zkClient, config.maxReservedBrokerId) + } catch { + case e: Exception => + error("Failed to generate broker.id due to ", e) + throw new GenerateBrokerIdException("Failed to generate broker.id", e) + } + } +} diff --git a/core/src/main/scala/kafka/server/KafkaServerStartable.scala b/core/src/main/scala/kafka/server/KafkaServerStartable.scala index acda52b801714..1c1b75b4137a8 100644 --- a/core/src/main/scala/kafka/server/KafkaServerStartable.scala +++ b/core/src/main/scala/kafka/server/KafkaServerStartable.scala @@ -17,26 +17,22 @@ package kafka.server +import kafka.common.AppInfo import kafka.utils.Logging class KafkaServerStartable(val serverConfig: KafkaConfig) extends Logging { - private var server : KafkaServer = null - - init - - private def init() { - server = new KafkaServer(serverConfig) - } + private val server = new KafkaServer(serverConfig) def startup() { try { server.startup() + AppInfo.registerInfo() } catch { case e: Throwable => - fatal("Fatal error during KafkaServerStable startup. Prepare to shutdown", e) - shutdown() + fatal("Fatal error during KafkaServerStartable startup. Prepare to shutdown", e) + // KafkaServer already calls shutdown() internally, so this is purely for logging & the exit code System.exit(1) } } @@ -52,6 +48,14 @@ class KafkaServerStartable(val serverConfig: KafkaConfig) extends Logging { } } + /** + * Allow setting broker state from the startable. + * This is needed when a custom kafka server startable want to emit new states that it introduces. + */ + def setServerState(newState: Byte) { + server.brokerState.newState(newState) + } + def awaitShutdown() = server.awaitShutdown diff --git a/core/src/main/scala/kafka/server/LogOffsetMetadata.scala b/core/src/main/scala/kafka/server/LogOffsetMetadata.scala new file mode 100644 index 0000000000000..00b60fe152c68 --- /dev/null +++ b/core/src/main/scala/kafka/server/LogOffsetMetadata.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.server + +import org.apache.kafka.common.KafkaException + +object LogOffsetMetadata { + val UnknownOffsetMetadata = new LogOffsetMetadata(-1, 0, 0) + val UnknownSegBaseOffset = -1L + val UnknownFilePosition = -1 + + class OffsetOrdering extends Ordering[LogOffsetMetadata] { + override def compare(x: LogOffsetMetadata , y: LogOffsetMetadata ): Int = { + x.offsetDiff(y).toInt + } + } + +} + +/* + * A log offset structure, including: + * 1. the message offset + * 2. the base message offset of the located segment + * 3. the physical position on the located segment + */ +case class LogOffsetMetadata(messageOffset: Long, + segmentBaseOffset: Long = LogOffsetMetadata.UnknownSegBaseOffset, + relativePositionInSegment: Int = LogOffsetMetadata.UnknownFilePosition) { + + // check if this offset is already on an older segment compared with the given offset + def offsetOnOlderSegment(that: LogOffsetMetadata): Boolean = { + if (messageOffsetOnly()) + throw new KafkaException("%s cannot compare its segment info with %s since it only has message offset info".format(this, that)) + + this.segmentBaseOffset < that.segmentBaseOffset + } + + // check if this offset is on the same segment with the given offset + def offsetOnSameSegment(that: LogOffsetMetadata): Boolean = { + if (messageOffsetOnly()) + throw new KafkaException("%s cannot compare its segment info with %s since it only has message offset info".format(this, that)) + + this.segmentBaseOffset == that.segmentBaseOffset + } + + // check if this offset is before the given offset + def precedes(that: LogOffsetMetadata): Boolean = this.messageOffset < that.messageOffset + + // compute the number of messages between this offset to the given offset + def offsetDiff(that: LogOffsetMetadata): Long = { + this.messageOffset - that.messageOffset + } + + // compute the number of bytes between this offset to the given offset + // if they are on the same segment and this offset precedes the given offset + def positionDiff(that: LogOffsetMetadata): Int = { + if(!offsetOnSameSegment(that)) + throw new KafkaException("%s cannot compare its segment position with %s since they are not on the same segment".format(this, that)) + if(messageOffsetOnly()) + throw new KafkaException("%s cannot compare its segment position with %s since it only has message offset info".format(this, that)) + + this.relativePositionInSegment - that.relativePositionInSegment + } + + // decide if the offset metadata only contains message offset info + def messageOffsetOnly(): Boolean = { + segmentBaseOffset == LogOffsetMetadata.UnknownSegBaseOffset && relativePositionInSegment == LogOffsetMetadata.UnknownFilePosition + } + + override def toString = messageOffset.toString + " [" + segmentBaseOffset + " : " + relativePositionInSegment + "]" + +} diff --git a/core/src/main/scala/kafka/server/MessageSetSend.scala b/core/src/main/scala/kafka/server/MessageSetSend.scala deleted file mode 100644 index 566764850cc60..0000000000000 --- a/core/src/main/scala/kafka/server/MessageSetSend.scala +++ /dev/null @@ -1,71 +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 java.nio._ -import java.nio.channels._ -import kafka.network._ -import kafka.message._ -import kafka.utils._ -import kafka.common.ErrorMapping - -/** - * A zero-copy message response that writes the bytes needed directly from the file - * wholly in kernel space - */ -@nonthreadsafe -private[server] class MessageSetSend(val messages: MessageSet, val errorCode: Short) extends Send { - - private var sent: Int = 0 - private val size: Int = messages.sizeInBytes - private val header = ByteBuffer.allocate(6) - header.putInt(size + 2) - header.putShort(errorCode) - header.rewind() - - var complete: Boolean = false - - def this(messages: MessageSet) = this(messages, ErrorMapping.NoError) - - def this() = this(MessageSet.Empty) - - def writeTo(channel: GatheringByteChannel): Int = { - expectIncomplete() - var written = 0 - if(header.hasRemaining) - written += channel.write(header) - if(!header.hasRemaining) { - val fileBytesSent = messages.writeTo(channel, sent, size - sent) - written += fileBytesSent - sent += fileBytesSent - } - - if(logger.isTraceEnabled) - if (channel.isInstanceOf[SocketChannel]) { - val socketChannel = channel.asInstanceOf[SocketChannel] - logger.trace(sent + " bytes written to " + socketChannel.socket.getRemoteSocketAddress() + " expecting to send " + size + " bytes") - } - - if(sent >= size) - complete = true - written - } - - def sendSize: Int = size + header.capacity - -} diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala new file mode 100755 index 0000000000000..9a9205f78ff20 --- /dev/null +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -0,0 +1,164 @@ +/** + * 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.cluster.{BrokerEndPoint,Broker} +import kafka.common.{ErrorMapping, ReplicaNotAvailableException, LeaderNotAvailableException} +import kafka.common.TopicAndPartition + +import kafka.api._ +import kafka.controller.KafkaController.StateChangeLogger +import org.apache.kafka.common.protocol.SecurityProtocol +import scala.collection.{Seq, Set, mutable} +import kafka.utils.Logging +import kafka.utils.CoreUtils._ + +import java.util.concurrent.locks.ReentrantReadWriteLock + +/** + * A cache for the state (e.g., current leader) of each partition. This cache is updated through + * UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously. + */ +private[server] class MetadataCache(brokerId: Int) extends Logging { + private val cache: mutable.Map[String, mutable.Map[Int, PartitionStateInfo]] = + new mutable.HashMap[String, mutable.Map[Int, PartitionStateInfo]]() + private var aliveBrokers: Map[Int, Broker] = Map() + private val partitionMetadataLock = new ReentrantReadWriteLock() + + this.logIdent = "[Kafka Metadata Cache on broker %d] ".format(brokerId) + + def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol) = { + + val isAllTopics = topics.isEmpty + val topicsRequested = if(isAllTopics) cache.keySet else topics + val topicResponses: mutable.ListBuffer[TopicMetadata] = new mutable.ListBuffer[TopicMetadata] + inReadLock(partitionMetadataLock) { + for (topic <- topicsRequested) { + if (isAllTopics || cache.contains(topic)) { + val partitionStateInfos = cache(topic) + val partitionMetadata = partitionStateInfos.map { + case (partitionId, partitionState) => + val replicas = partitionState.allReplicas + val replicaInfo: Seq[BrokerEndPoint] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq.map(_.getBrokerEndPoint(protocol)) + var leaderInfo: Option[BrokerEndPoint] = None + var leaderBrokerInfo: Option[Broker] = None + var isrInfo: Seq[BrokerEndPoint] = Nil + val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch + val leader = leaderIsrAndEpoch.leaderAndIsr.leader + val isr = leaderIsrAndEpoch.leaderAndIsr.isr + val topicPartition = TopicAndPartition(topic, partitionId) + try { + leaderBrokerInfo = aliveBrokers.get(leader) + if (!leaderBrokerInfo.isDefined) + throw new LeaderNotAvailableException("Leader not available for %s".format(topicPartition)) + else + leaderInfo = Some(leaderBrokerInfo.get.getBrokerEndPoint(protocol)) + isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).map(_.getBrokerEndPoint(protocol)) + if (replicaInfo.size < replicas.size) + throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + + replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) + if (isrInfo.size < isr.size) + throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + + isr.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) + new PartitionMetadata(partitionId, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) + } catch { + case e: Throwable => + debug("Error while fetching metadata for %s: %s".format(topicPartition, e.toString)) + new PartitionMetadata(partitionId, leaderInfo, replicaInfo, isrInfo, + ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) + } + } + topicResponses += new TopicMetadata(topic, partitionMetadata.toSeq) + } + } + } + topicResponses + } + + def getAliveBrokers = { + inReadLock(partitionMetadataLock) { + aliveBrokers.values.toSeq + } + } + + def addOrUpdatePartitionInfo(topic: String, + partitionId: Int, + stateInfo: PartitionStateInfo) { + inWriteLock(partitionMetadataLock) { + cache.get(topic) match { + case Some(infos) => infos.put(partitionId, stateInfo) + case None => { + val newInfos: mutable.Map[Int, PartitionStateInfo] = new mutable.HashMap[Int, PartitionStateInfo] + cache.put(topic, newInfos) + newInfos.put(partitionId, stateInfo) + } + } + } + } + + def getPartitionInfo(topic: String, partitionId: Int): Option[PartitionStateInfo] = { + inReadLock(partitionMetadataLock) { + cache.get(topic) match { + case Some(partitionInfos) => partitionInfos.get(partitionId) + case None => None + } + } + } + + def updateCache(updateMetadataRequest: UpdateMetadataRequest, + brokerId: Int, + stateChangeLogger: StateChangeLogger) { + inWriteLock(partitionMetadataLock) { + aliveBrokers = updateMetadataRequest.aliveBrokers.map(b => (b.id, b)).toMap + updateMetadataRequest.partitionStateInfos.foreach { case(tp, info) => + if (info.leaderIsrAndControllerEpoch.leaderAndIsr.leader == LeaderAndIsr.LeaderDuringDelete) { + removePartitionInfo(tp.topic, tp.partition) + stateChangeLogger.trace(("Broker %d deleted partition %s from metadata cache in response to UpdateMetadata request " + + "sent by controller %d epoch %d with correlation id %d") + .format(brokerId, tp, updateMetadataRequest.controllerId, + updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId)) + } else { + addOrUpdatePartitionInfo(tp.topic, tp.partition, info) + stateChangeLogger.trace(("Broker %d cached leader info %s for partition %s in response to UpdateMetadata request " + + "sent by controller %d epoch %d with correlation id %d") + .format(brokerId, info, tp, updateMetadataRequest.controllerId, + updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId)) + } + } + } + } + + def contains(topic: String): Boolean = { + inReadLock(partitionMetadataLock) { + cache.contains(topic) + } + } + + private def removePartitionInfo(topic: String, partitionId: Int) = { + cache.get(topic) match { + case Some(infos) => { + infos.remove(partitionId) + if(infos.isEmpty) { + cache.remove(topic) + } + true + } + case None => false + } + } +} diff --git a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala index 19f61a9718a7f..8c5b0546908d3 100644 --- a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala +++ b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala @@ -34,7 +34,8 @@ class OffsetCheckpoint(val file: File) extends Logging { // 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)) + val fileOutputStream = new FileOutputStream(temp) + val writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream)) try { // write the current version writer.write(0.toString) @@ -50,8 +51,9 @@ class OffsetCheckpoint(val file: File) extends Logging { writer.newLine() } - // flush and overwrite old file + // flush the buffer and then fsync the underlying file writer.flush() + fileOutputStream.getFD().sync() } finally { writer.close() } @@ -90,7 +92,7 @@ class OffsetCheckpoint(val file: File) extends Logging { val topic = pieces(0) val partition = pieces(1).toInt val offset = pieces(2).toLong - offsets += (TopicAndPartition(pieces(0), partition) -> offset) + offsets += (TopicAndPartition(topic, partition) -> offset) line = reader.readLine() } if(offsets.size != expectedSize) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala new file mode 100755 index 0000000000000..47b6ce93da320 --- /dev/null +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -0,0 +1,615 @@ +/** + * 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.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.protocol.types.{Struct, Schema, Field} +import org.apache.kafka.common.protocol.types.Type.STRING +import org.apache.kafka.common.protocol.types.Type.INT32 +import org.apache.kafka.common.protocol.types.Type.INT64 +import org.apache.kafka.common.utils.Utils + +import kafka.utils._ +import kafka.common._ +import kafka.log.FileMessageSet +import kafka.message._ +import kafka.metrics.KafkaMetricsGroup +import kafka.common.TopicAndPartition +import kafka.tools.MessageFormatter +import kafka.api.ProducerResponseStatus +import kafka.coordinator.ConsumerCoordinator + +import scala.Some +import scala.collection._ +import java.io.PrintStream +import java.util.concurrent.atomic.AtomicBoolean +import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit + +import com.yammer.metrics.core.Gauge +import org.I0Itec.zkclient.ZkClient + +/** + * Configuration settings for in-built offset management + * @param maxMetadataSize The maximum allowed metadata for any offset commit. + * @param loadBufferSize Batch size for reading from the offsets segments when loading offsets into the cache. + * @param offsetsRetentionMs Offsets older than this retention period will be discarded. + * @param offsetsRetentionCheckIntervalMs Frequency at which to check for expired offsets. + * @param offsetsTopicNumPartitions The number of partitions for the offset commit topic (should not change after deployment). + * @param offsetsTopicSegmentBytes The offsets topic segment bytes should be kept relatively small to facilitate faster + * log compaction and faster offset loads + * @param offsetsTopicReplicationFactor The replication factor for the offset commit topic (set higher to ensure availability). + * @param offsetsTopicCompressionCodec Compression codec for the offsets topic - compression should be turned on in + * order to achieve "atomic" commits. + * @param offsetCommitTimeoutMs The offset commit will be delayed until all replicas for the offsets topic receive the + * commit or this timeout is reached. (Similar to the producer request timeout.) + * @param offsetCommitRequiredAcks The required acks before the commit can be accepted. In general, the default (-1) + * should not be overridden. + */ +case class OffsetManagerConfig(maxMetadataSize: Int = OffsetManagerConfig.DefaultMaxMetadataSize, + loadBufferSize: Int = OffsetManagerConfig.DefaultLoadBufferSize, + offsetsRetentionMs: Long = OffsetManagerConfig.DefaultOffsetRetentionMs, + offsetsRetentionCheckIntervalMs: Long = OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions: Int = OffsetManagerConfig.DefaultOffsetsTopicNumPartitions, + offsetsTopicSegmentBytes: Int = OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes, + offsetsTopicReplicationFactor: Short = OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor, + offsetsTopicCompressionCodec: CompressionCodec = OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec, + offsetCommitTimeoutMs: Int = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs, + offsetCommitRequiredAcks: Short = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks) + +object OffsetManagerConfig { + val DefaultMaxMetadataSize = 4096 + val DefaultLoadBufferSize = 5*1024*1024 + val DefaultOffsetRetentionMs = 24*60*60*1000L + val DefaultOffsetsRetentionCheckIntervalMs = 600000L + val DefaultOffsetsTopicNumPartitions = 50 + val DefaultOffsetsTopicSegmentBytes = 100*1024*1024 + val DefaultOffsetsTopicReplicationFactor = 3.toShort + val DefaultOffsetsTopicCompressionCodec = NoCompressionCodec + val DefaultOffsetCommitTimeoutMs = 5000 + val DefaultOffsetCommitRequiredAcks = (-1).toShort +} + +class OffsetManager(val config: OffsetManagerConfig, + replicaManager: ReplicaManager, + zkClient: ZkClient, + scheduler: Scheduler) extends Logging with KafkaMetricsGroup { + + /* offsets and metadata cache */ + private val offsetsCache = new Pool[GroupTopicPartition, OffsetAndMetadata] + private val followerTransitionLock = new Object + private val loadingPartitions: mutable.Set[Int] = mutable.Set() + private val cleanupOrLoadMutex = new Object + private val shuttingDown = new AtomicBoolean(false) + + this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: " + + scheduler.schedule(name = "delete-expired-consumer-offsets", + fun = deleteExpiredOffsets, + period = config.offsetsRetentionCheckIntervalMs, + unit = TimeUnit.MILLISECONDS) + + newGauge("NumOffsets", + new Gauge[Int] { + def value = offsetsCache.size + } + ) + + newGauge("NumGroups", + new Gauge[Int] { + def value = offsetsCache.keys.map(_.group).toSet.size + } + ) + + private def deleteExpiredOffsets() { + debug("Collecting expired offsets.") + val startMs = SystemTime.milliseconds + + val numExpiredOffsetsRemoved = cleanupOrLoadMutex synchronized { + val expiredOffsets = offsetsCache.filter { case (groupTopicPartition, offsetAndMetadata) => + offsetAndMetadata.expireTimestamp < startMs + } + + debug("Found %d expired offsets.".format(expiredOffsets.size)) + + // delete the expired offsets from the table and generate tombstone messages to remove them from the log + val tombstonesForPartition = expiredOffsets.map { case (groupTopicAndPartition, offsetAndMetadata) => + val offsetsPartition = partitionFor(groupTopicAndPartition.group) + trace("Removing expired offset and metadata for %s: %s".format(groupTopicAndPartition, offsetAndMetadata)) + + offsetsCache.remove(groupTopicAndPartition) + + val commitKey = OffsetManager.offsetCommitKey(groupTopicAndPartition.group, + groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition) + + (offsetsPartition, new Message(bytes = null, key = commitKey)) + }.groupBy { case (partition, tombstone) => partition } + + // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say, + // if we crash or leaders move) since the new leaders will get rid of expired offsets during their own purge cycles. + tombstonesForPartition.flatMap { case (offsetsPartition, tombstones) => + val partitionOpt = replicaManager.getPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) + partitionOpt.map { partition => + val appendPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) + val messages = tombstones.map(_._2).toSeq + + trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) + + try { + // do not need to require acks since even if the tombsone is lost, + // it will be appended again in the next purge cycle + partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages: _*)) + tombstones.size + } + catch { + case t: Throwable => + error("Failed to mark %d expired offsets for deletion in %s.".format(messages.size, appendPartition), t) + // ignore and continue + 0 + } + } + }.sum + } + + info("Removed %d expired offsets in %d milliseconds.".format(numExpiredOffsetsRemoved, SystemTime.milliseconds - startMs)) + } + + + def partitionFor(group: String): Int = Utils.abs(group.hashCode) % config.offsetsTopicNumPartitions + + /** + * Fetch the current offset for the given group/topic/partition from the underlying offsets storage. + * + * @param key The requested group-topic-partition + * @return If the key is present, return the offset and metadata; otherwise return None + */ + private def getOffset(key: GroupTopicPartition) = { + val offsetAndMetadata = offsetsCache.get(key) + if (offsetAndMetadata == null) + OffsetMetadataAndError.NoOffset + else + OffsetMetadataAndError(offsetAndMetadata.offset, offsetAndMetadata.metadata, ErrorMapping.NoError) + } + + /** + * Put the (already committed) offset for the given group/topic/partition into the cache. + * + * @param key The group-topic-partition + * @param offsetAndMetadata The offset/metadata to be stored + */ + private def putOffset(key: GroupTopicPartition, offsetAndMetadata: OffsetAndMetadata) { + offsetsCache.put(key, offsetAndMetadata) + } + + /* + * Check if the offset metadata length is valid + */ + def validateOffsetMetadataLength(metadata: String) : Boolean = { + metadata == null || metadata.length() <= config.maxMetadataSize + } + + /** + * Store offsets by appending it to the replicated log and then inserting to cache + */ + def storeOffsets(groupId: String, + consumerId: String, + generationId: Int, + offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], + responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { + // first filter out partitions with offset metadata size exceeding limit + val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => + validateOffsetMetadataLength(offsetAndMetadata.metadata) + } + + // construct the message set to append + val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + new Message( + key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), + bytes = OffsetManager.offsetCommitValue(offsetAndMetadata) + ) + }.toSeq + + val offsetTopicPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, partitionFor(groupId)) + + val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> + new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) + + // set the callback function to insert offsets into cache after log append completed + def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + // the append response should only contain the topics partition + if (responseStatus.size != 1 || ! responseStatus.contains(offsetTopicPartition)) + throw new IllegalStateException("Append status %s should only have one partition %s" + .format(responseStatus, offsetTopicPartition)) + + // construct the commit response status and insert + // the offset and metadata to cache if the append status has no error + val status = responseStatus(offsetTopicPartition) + + val responseCode = + if (status.error == ErrorMapping.NoError) { + filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => + putOffset(GroupTopicPartition(groupId, topicAndPartition), offsetAndMetadata) + } + ErrorMapping.NoError + } else { + debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" + .format(filteredOffsetMetadata, groupId, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) + + // transform the log append error code to the corresponding the commit status error code + if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) + ErrorMapping.ConsumerCoordinatorNotAvailableCode + else if (status.error == ErrorMapping.NotLeaderForPartitionCode) + ErrorMapping.NotCoordinatorForConsumerCode + else if (status.error == ErrorMapping.MessageSizeTooLargeCode + || status.error == ErrorMapping.MessageSetSizeTooLargeCode + || status.error == ErrorMapping.InvalidFetchSizeCode) + Errors.INVALID_COMMIT_OFFSET_SIZE.code + else + status.error + } + + + // compute the final error codes for the commit response + val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) + (topicAndPartition, responseCode) + else + (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) + } + + // finally trigger the callback logic passed from the API layer + responseCallback(commitStatus) + } + + // call replica manager to append the offset messages + replicaManager.appendMessages( + config.offsetCommitTimeoutMs.toLong, + config.offsetCommitRequiredAcks, + true, // allow appending to internal offset topic + offsetsAndMetadataMessageSet, + putCacheCallback) + } + + /** + * The most important guarantee that this API provides is that it should never return a stale offset. i.e., it either + * returns the current offset or it begins to sync the cache from the log (and returns an error code). + */ + def getOffsets(group: String, topicPartitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = { + trace("Getting offsets %s for group %s.".format(topicPartitions, group)) + + val offsetsPartition = partitionFor(group) + + /** + * followerTransitionLock protects against fetching from an empty/cleared offset cache (i.e., cleared due to a + * leader->follower transition). i.e., even if leader-is-local is true a follower transition can occur right after + * the check and clear the cache. i.e., we would read from the empty cache and incorrectly return NoOffset. + */ + followerTransitionLock synchronized { + if (leaderIsLocal(offsetsPartition)) { + if (loadingPartitions synchronized loadingPartitions.contains(offsetsPartition)) { + debug("Cannot fetch offsets for group %s due to ongoing offset load.".format(group)) + topicPartitions.map { topicAndPartition => + val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) + (groupTopicPartition.topicPartition, OffsetMetadataAndError.OffsetsLoading) + }.toMap + } else { + if (topicPartitions.size == 0) { + // Return offsets for all partitions owned by this consumer group. (this only applies to consumers that commit offsets to Kafka.) + offsetsCache.filter(_._1.group == group).map { case(groupTopicPartition, offsetAndMetadata) => + (groupTopicPartition.topicPartition, OffsetMetadataAndError(offsetAndMetadata.offset, offsetAndMetadata.metadata, ErrorMapping.NoError)) + }.toMap + } else { + topicPartitions.map { topicAndPartition => + val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) + (groupTopicPartition.topicPartition, getOffset(groupTopicPartition)) + }.toMap + } + } + } else { + debug("Could not fetch offsets for group %s (not offset coordinator).".format(group)) + topicPartitions.map { topicAndPartition => + val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) + (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotCoordinatorForGroup) + }.toMap + } + } + } + + /** + * Asynchronously read the partition from the offsets topic and populate the cache + */ + def loadOffsetsFromLog(offsetsPartition: Int) { + + val topicPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition) + + loadingPartitions synchronized { + if (loadingPartitions.contains(offsetsPartition)) { + info("Offset load from %s already in progress.".format(topicPartition)) + } else { + loadingPartitions.add(offsetsPartition) + scheduler.schedule(topicPartition.toString, loadOffsets) + } + } + + def loadOffsets() { + info("Loading offsets from " + topicPartition) + + val startMs = SystemTime.milliseconds + try { + replicaManager.logManager.getLog(topicPartition) match { + case Some(log) => + var currOffset = log.logSegments.head.baseOffset + val buffer = ByteBuffer.allocate(config.loadBufferSize) + // loop breaks if leader changes at any time during the load, since getHighWatermark is -1 + cleanupOrLoadMutex synchronized { + while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) { + buffer.clear() + val messages = log.read(currOffset, config.loadBufferSize).messageSet.asInstanceOf[FileMessageSet] + messages.readInto(buffer, 0) + val messageSet = new ByteBufferMessageSet(buffer) + messageSet.foreach { msgAndOffset => + require(msgAndOffset.message.key != null, "Offset entry key should not be null") + val key = OffsetManager.readMessageKey(msgAndOffset.message.key) + if (msgAndOffset.message.payload == null) { + if (offsetsCache.remove(key) != null) + trace("Removed offset for %s due to tombstone entry.".format(key)) + else + trace("Ignoring redundant tombstone for %s.".format(key)) + } else { + // special handling for version 0: + // set the expiration time stamp as commit time stamp + server default retention time + val value = OffsetManager.readMessageValue(msgAndOffset.message.payload) + putOffset(key, value.copy ( + expireTimestamp = { + if (value.expireTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + value.commitTimestamp + config.offsetsRetentionMs + else + value.expireTimestamp + } + )) + trace("Loaded offset %s for %s.".format(value, key)) + } + currOffset = msgAndOffset.nextOffset + } + } + } + + if (!shuttingDown.get()) + info("Finished loading offsets from %s in %d milliseconds." + .format(topicPartition, SystemTime.milliseconds - startMs)) + case None => + warn("No log found for " + topicPartition) + } + } + catch { + case t: Throwable => + error("Error in loading offsets from " + topicPartition, t) + } + finally { + loadingPartitions synchronized loadingPartitions.remove(offsetsPartition) + } + } + } + + private def getHighWatermark(partitionId: Int): Long = { + val partitionOpt = replicaManager.getPartition(ConsumerCoordinator.OffsetsTopicName, partitionId) + + val hw = partitionOpt.map { partition => + partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L) + }.getOrElse(-1L) + + hw + } + + def leaderIsLocal(partition: Int) = { getHighWatermark(partition) != -1L } + + /** + * When this broker becomes a follower for an offsets topic partition clear out the cache for groups that belong to + * that partition. + * @param offsetsPartition Groups belonging to this partition of the offsets topic will be deleted from the cache. + */ + def removeOffsetsFromCacheForPartition(offsetsPartition: Int) { + var numRemoved = 0 + followerTransitionLock synchronized { + offsetsCache.keys.foreach { key => + if (partitionFor(key.group) == offsetsPartition) { + offsetsCache.remove(key) + numRemoved += 1 + } + } + } + + if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." + .format(numRemoved, TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition))) + + } + + def shutdown() { + shuttingDown.set(true) + } + +} + +object OffsetManager { + + private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) + + private val CURRENT_OFFSET_SCHEMA_VERSION = 1.toShort + + private val OFFSET_COMMIT_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING), + new Field("topic", STRING), + new Field("partition", INT32)) + private val KEY_GROUP_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("group") + private val KEY_TOPIC_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("topic") + private val KEY_PARTITION_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("partition") + + private val OFFSET_COMMIT_VALUE_SCHEMA_V0 = new Schema(new Field("offset", INT64), + new Field("metadata", STRING, "Associated metadata.", ""), + new Field("timestamp", INT64)) + + private val OFFSET_COMMIT_VALUE_SCHEMA_V1 = new Schema(new Field("offset", INT64), + new Field("metadata", STRING, "Associated metadata.", ""), + new Field("commit_timestamp", INT64), + new Field("expire_timestamp", INT64)) + + private val VALUE_OFFSET_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") + private val VALUE_METADATA_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") + private val VALUE_TIMESTAMP_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") + + private val VALUE_OFFSET_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("offset") + private val VALUE_METADATA_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("metadata") + private val VALUE_COMMIT_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("commit_timestamp") + private val VALUE_EXPIRE_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("expire_timestamp") + + // map of versions to schemas + private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0), + 1 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V1)) + + private val CURRENT_SCHEMA = schemaFor(CURRENT_OFFSET_SCHEMA_VERSION) + + private def schemaFor(version: Int) = { + val schemaOpt = OFFSET_SCHEMAS.get(version) + schemaOpt match { + case Some(schema) => schema + case _ => throw new KafkaException("Unknown offset schema version " + version) + } + } + + /** + * Generates the key for offset commit message for given (group, topic, partition) + * + * @return key for offset commit message + */ + private def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = { + val key = new Struct(CURRENT_SCHEMA.keySchema) + key.set(KEY_GROUP_FIELD, group) + key.set(KEY_TOPIC_FIELD, topic) + key.set(KEY_PARTITION_FIELD, partition) + + val byteBuffer = ByteBuffer.allocate(2 /* version */ + key.sizeOf) + byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) + key.writeTo(byteBuffer) + byteBuffer.array() + } + + /** + * Generates the payload for offset commit message from given offset and metadata + * + * @param offsetAndMetadata consumer's current offset and metadata + * @return payload for offset commit message + */ + private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { + // generate commit value with schema version 1 + val value = new Struct(CURRENT_SCHEMA.valueSchema) + value.set(VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) + value.set(VALUE_METADATA_FIELD_V1, offsetAndMetadata.metadata) + value.set(VALUE_COMMIT_TIMESTAMP_FIELD_V1, offsetAndMetadata.commitTimestamp) + value.set(VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.expireTimestamp) + val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) + byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) + value.writeTo(byteBuffer) + byteBuffer.array() + } + + /** + * Decodes the offset messages' key + * + * @param buffer input byte-buffer + * @return an GroupTopicPartition object + */ + private def readMessageKey(buffer: ByteBuffer): GroupTopicPartition = { + val version = buffer.getShort() + val keySchema = schemaFor(version).keySchema + val key = keySchema.read(buffer).asInstanceOf[Struct] + + val group = key.get(KEY_GROUP_FIELD).asInstanceOf[String] + val topic = key.get(KEY_TOPIC_FIELD).asInstanceOf[String] + val partition = key.get(KEY_PARTITION_FIELD).asInstanceOf[Int] + + GroupTopicPartition(group, TopicAndPartition(topic, partition)) + } + + /** + * Decodes the offset messages' payload and retrieves offset and metadata from it + * + * @param buffer input byte-buffer + * @return an offset-metadata object from the message + */ + private def readMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { + val structAndVersion = readMessageValueStruct(buffer) + + if (structAndVersion.value == null) { // tombstone + null + } else { + if (structAndVersion.version == 0) { + val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] + val metadata = structAndVersion.value.get(VALUE_METADATA_FIELD_V0).asInstanceOf[String] + val timestamp = structAndVersion.value.get(VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] + + OffsetAndMetadata(offset, metadata, timestamp) + } else if (structAndVersion.version == 1) { + val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] + val metadata = structAndVersion.value.get(VALUE_METADATA_FIELD_V1).asInstanceOf[String] + val commitTimestamp = structAndVersion.value.get(VALUE_COMMIT_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + val expireTimestamp = structAndVersion.value.get(VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + + OffsetAndMetadata(offset, metadata, commitTimestamp, expireTimestamp) + } else { + throw new IllegalStateException("Unknown offset message version") + } + } + } + + private def readMessageValueStruct(buffer: ByteBuffer): MessageValueStructAndVersion = { + if(buffer == null) { // tombstone + MessageValueStructAndVersion(null, -1) + } else { + val version = buffer.getShort() + val valueSchema = schemaFor(version).valueSchema + val value = valueSchema.read(buffer).asInstanceOf[Struct] + + MessageValueStructAndVersion(value, version) + } + } + + // Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false. + // (specify --formatter "kafka.server.OffsetManager\$OffsetsMessageFormatter" when consuming __consumer_offsets) + class OffsetsMessageFormatter extends MessageFormatter { + def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) { + val formattedKey = if (key == null) "NULL" else OffsetManager.readMessageKey(ByteBuffer.wrap(key)).toString + val formattedValue = if (value == null) "NULL" else OffsetManager.readMessageValueStruct(ByteBuffer.wrap(value)).value.toString + output.write(formattedKey.getBytes) + output.write("::".getBytes) + output.write(formattedValue.getBytes) + output.write("\n".getBytes) + } + } + +} + +case class MessageValueStructAndVersion(value: Struct, version: Short) + +case class GroupTopicPartition(group: String, topicPartition: TopicAndPartition) { + + def this(group: String, topic: String, partition: Int) = + this(group, new TopicAndPartition(topic, partition)) + + override def toString = + "[%s,%s,%d]".format(group, topicPartition.topic, topicPartition.partition) + +} diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala index 351dbbad3bdb7..ef38ed3e0bfb6 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala @@ -17,13 +17,13 @@ package kafka.server -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val replicaMgr: ReplicaManager) extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId, "Replica", brokerConfig.numReplicaFetchers) { - override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { + override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { new ReplicaFetcherThread("ReplicaFetcherThread-%d-%d".format(fetcherId, sourceBroker.id), sourceBroker, brokerConfig, replicaMgr) } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 73e605eb31bc7..c89d00b5976ff 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -17,13 +17,15 @@ package kafka.server -import kafka.cluster.Broker +import kafka.admin.AdminUtils +import kafka.cluster.BrokerEndPoint +import kafka.log.LogConfig import kafka.message.ByteBufferMessageSet import kafka.api.{OffsetRequest, FetchResponsePartitionData} import kafka.common.{KafkaStorageException, TopicAndPartition} class ReplicaFetcherThread(name:String, - sourceBroker: Broker, + sourceBroker: BrokerEndPoint, brokerConfig: KafkaConfig, replicaMgr: ReplicaManager) extends AbstractFetcherThread(name = name, @@ -35,6 +37,7 @@ class ReplicaFetcherThread(name:String, fetcherBrokerId = brokerConfig.brokerId, maxWait = brokerConfig.replicaFetchWaitMaxMs, minBytes = brokerConfig.replicaFetchMinBytes, + fetchBackOffMs = brokerConfig.replicaFetchBackoffMs, isInterruptible = false) { // process fetched data @@ -45,16 +48,19 @@ class ReplicaFetcherThread(name:String, val replica = replicaMgr.getReplica(topic, partitionId).get val messageSet = partitionData.messages.asInstanceOf[ByteBufferMessageSet] - if (fetchOffset != replica.logEndOffset) - throw new RuntimeException("Offset mismatch: fetched offset = %d, log end offset = %d.".format(fetchOffset, replica.logEndOffset)) + if (fetchOffset != replica.logEndOffset.messageOffset) + throw new RuntimeException("Offset mismatch: fetched offset = %d, log end offset = %d.".format(fetchOffset, replica.logEndOffset.messageOffset)) trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d" - .format(replica.brokerId, replica.logEndOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.hw)) + .format(replica.brokerId, replica.logEndOffset.messageOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.hw)) replica.log.get.append(messageSet, assignOffsets = false) trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s" - .format(replica.brokerId, replica.logEndOffset, messageSet.sizeInBytes, topicAndPartition)) - val followerHighWatermark = replica.logEndOffset.min(partitionData.hw) - replica.highWatermark = followerHighWatermark - trace("Follower %d set replica highwatermark for partition [%s,%d] to %d" + .format(replica.brokerId, replica.logEndOffset.messageOffset, messageSet.sizeInBytes, topicAndPartition)) + val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.hw) + // for the follower replica, we do not need to keep + // its segment base offset the physical position, + // these values will be computed upon making the leader + replica.highWatermark = new LogOffsetMetadata(followerHighWatermark) + trace("Follower %d set replica high watermark for partition [%s,%d] to %s" .format(replica.brokerId, topic, partitionId, followerHighWatermark)) } catch { case e: KafkaStorageException => @@ -80,10 +86,22 @@ 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 < replica.logEndOffset) { + if (leaderEndOffset < replica.logEndOffset.messageOffset) { + // Prior to truncating the follower's log, ensure that doing so is not disallowed by the configuration for unclean leader election. + // This situation could only happen if the unclean election configuration for a topic changes while a replica is down. Otherwise, + // we should never encounter this situation since a non-ISR leader cannot be elected if disallowed by the broker configuration. + if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, + topicAndPartition.topic)).uncleanLeaderElectionEnable) { + // Log a fatal error and shutdown the broker to ensure that data loss does not unexpectedly occur. + fatal("Halting because log truncation is not allowed for topic %s,".format(topicAndPartition.topic) + + " Current leader %d's latest offset %d is less than replica %d's latest offset %d" + .format(sourceBroker.id, leaderEndOffset, brokerConfig.brokerId, replica.logEndOffset.messageOffset)) + Runtime.getRuntime.halt(1) + } + 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)) + warn("Replica %d for partition %s reset its fetch offset from %d to current leader %d's latest offset %d" + .format(brokerConfig.brokerId, topicAndPartition, replica.logEndOffset.messageOffset, sourceBroker.id, leaderEndOffset)) leaderEndOffset } else { /** @@ -93,15 +111,15 @@ 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) + warn("Replica %d for partition %s reset its fetch offset from %d to current leader %d's start offset %d" + .format(brokerConfig.brokerId, topicAndPartition, replica.logEndOffset.messageOffset, sourceBroker.id, 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 } } // any logic for partitions whose leader has changed def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) { - // no handler needed since the controller will make the changes accordingly + delayPartitions(partitions, brokerConfig.replicaFetchBackoffMs.toLong) } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 21bba48affb38..795220e7f63d1 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -16,31 +16,92 @@ */ package kafka.server -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.api._ +import kafka.common._ import kafka.utils._ -import kafka.log.LogManager +import kafka.cluster.{BrokerEndPoint, Partition, Replica} +import kafka.log.{LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup -import com.yammer.metrics.core.Gauge -import java.util.concurrent.TimeUnit -import kafka.common._ -import kafka.api.{StopReplicaRequest, PartitionStateInfo, LeaderAndIsrRequest} import kafka.controller.KafkaController -import org.apache.log4j.Logger +import kafka.message.{ByteBufferMessageSet, MessageSet} +import kafka.api.ProducerResponseStatus +import kafka.common.TopicAndPartition +import kafka.api.PartitionFetchInfo + +import org.apache.kafka.common.protocol.Errors + +import java.util.concurrent.atomic.AtomicBoolean +import java.io.{IOException, File} +import java.util.concurrent.TimeUnit + +import scala.Some +import scala.collection._ + +import org.I0Itec.zkclient.ZkClient +import com.yammer.metrics.core.Gauge + +/* + * Result metadata of a log append operation on the log + */ +case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } +} + +/* + * Result metadata of a log read operation on the log + * @param info @FetchDataInfo returned by the @Log read + * @param hw high watermark of the local replica + * @param readSize amount of data that was read from the log i.e. size of the fetch + * @param isReadFromLogEnd true if the request read up to the log end offset snapshot + * when the read was initiated, false otherwise + * @param error Exception if error encountered while reading from the log + */ +case class LogReadResult(info: FetchDataInfo, + hw: Long, + readSize: Int, + isReadFromLogEnd : Boolean, + error: Option[Throwable] = None) { + + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } + + override def toString = { + "Fetch Data: [%s], HW: [%d], readSize: [%d], isReadFromLogEnd: [%b], error: [%s]" + .format(info, hw, readSize, isReadFromLogEnd, error) + } +} + +object LogReadResult { + val UnknownLogReadResult = LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, + MessageSet.Empty), + -1L, + -1, + false) +} +case class BecomeLeaderOrFollowerResult(responseMap: collection.Map[(String, Int), Short], + updatedLeaders: Set[Partition], + updatedFollowers: Set[Partition], + errorCode: Short) { + + override def toString = { + "updated leaders: [%s], updated followers: [%s], update results: [%s], global error: [%d]" + .format(updatedLeaders, updatedFollowers, responseMap, errorCode) + } +} object ReplicaManager { - val UnknownLogEndOffset = -1L val HighWatermarkFilename = "replication-offset-checkpoint" } -class ReplicaManager(val config: KafkaConfig, - time: Time, - val zkClient: ZkClient, +class ReplicaManager(val config: KafkaConfig, + private val time: Time, + val zkClient: ZkClient, scheduler: Scheduler, val logManager: LogManager, val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup { @@ -48,23 +109,24 @@ class ReplicaManager(val config: KafkaConfig, @volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1 private val localBrokerId = config.brokerId 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 OffsetCheckpoint(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap + val highWatermarkCheckpoints = config.logDirs.map(dir => (new File(dir).getAbsolutePath, 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) + val stateChangeLogger = KafkaController.stateChangeLogger + + val delayedProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( + purgatoryName = "Produce", config.brokerId, config.producerPurgatoryPurgeIntervalRequests) + val delayedFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch]( + purgatoryName = "Fetch", config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) newGauge( "LeaderCount", new Gauge[Int] { def value = { - leaderPartitionsLock synchronized { - leaderPartitions.size - } + getLeaderPartitions().size } } ) @@ -84,9 +146,7 @@ class ReplicaManager(val config: KafkaConfig, val isrShrinkRate = newMeter("IsrShrinksPerSec", "shrinks", TimeUnit.SECONDS) def underReplicatedPartitionCount(): Int = { - leaderPartitionsLock synchronized { - leaderPartitions.count(_.isUnderReplicated) - } + getLeaderPartitions().count(_.isUnderReplicated) } def startHighWaterMarksCheckPointThread() = { @@ -95,17 +155,27 @@ class ReplicaManager(val config: KafkaConfig, } /** - * This function is only used in two places: in Partition.updateISR() and KafkaApis.handleProducerRequest(). - * In the former case, the partition should have been created, in the latter case, return -1 will put the request into purgatory + * Try to complete some delayed produce requests with the request key; + * this can be triggered when: + * + * 1. The partition HW has changed (for acks = -1) + * 2. A follower replica's fetch operation is received (for acks > 1) */ - def getReplicationFactorForPartition(topic: String, partitionId: Int) = { - val partitionOpt = getPartition(topic, partitionId) - partitionOpt match { - case Some(partition) => - partition.replicationFactor - case None => - -1 - } + def tryCompleteDelayedProduce(key: DelayedOperationKey) { + val completed = delayedProducePurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d producer requests.".format(key.keyLabel, completed)) + } + + /** + * Try to complete some delayed fetch requests with the request key; + * this can be triggered when: + * + * 1. The partition HW has changed (for regular fetch) + * 2. A new message set is appended to the local log (for follower fetch) + */ + def tryCompleteDelayedFetch(key: DelayedOperationKey) { + val completed = delayedFetchPurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, completed)) } def startup() { @@ -119,15 +189,21 @@ class ReplicaManager(val config: KafkaConfig, val errorCode = ErrorMapping.NoError getPartition(topic, partitionId) match { case Some(partition) => - leaderPartitionsLock synchronized { - leaderPartitions -= partition - } if(deletePartition) { val removedPartition = allPartitions.remove((topic, partitionId)) if (removedPartition != null) removedPartition.delete() // this will delete the local log } - case None => //do nothing if replica no longer exists. This can happen during delete topic retries + case None => + // Delete log and corresponding folders in case replica manager doesn't hold them anymore. + // This could happen when topic is being deleted while broker is down and recovers. + if(deletePartition) { + val topicAndPartition = TopicAndPartition(topic, partitionId) + + if(logManager.getLog(topicAndPartition).isDefined) { + logManager.deleteLog(topicAndPartition) + } + } stateChangeLogger.trace("Broker %d ignoring stop replica (delete=%s) for partition [%s,%d] as replica doesn't exist on broker" .format(localBrokerId, deletePartition, topic, partitionId)) } @@ -157,10 +233,10 @@ class ReplicaManager(val config: KafkaConfig, } } - def getOrCreatePartition(topic: String, partitionId: Int, replicationFactor: Int): Partition = { + def getOrCreatePartition(topic: String, partitionId: Int): Partition = { var partition = allPartitions.get((topic, partitionId)) if (partition == null) { - allPartitions.putIfNotExists((topic, partitionId), new Partition(topic, partitionId, replicationFactor, time, this)) + allPartitions.putIfNotExists((topic, partitionId), new Partition(topic, partitionId, time, this)) partition = allPartitions.get((topic, partitionId)) } partition @@ -177,7 +253,7 @@ class ReplicaManager(val config: KafkaConfig, def getReplicaOrException(topic: String, partition: Int): Replica = { val replicaOpt = getReplica(topic, partition) if(replicaOpt.isDefined) - return replicaOpt.get + replicaOpt.get else throw new ReplicaNotAvailableException("Replica %d is not available for partition [%s,%d]".format(config.brokerId, topic, partition)) } @@ -205,30 +281,304 @@ class ReplicaManager(val config: KafkaConfig, } } - def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): (collection.Map[(String, Int), Short], Short) = { + /** + * Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas; + * the callback function will be triggered either when timeout or the required acks are satisfied + */ + def appendMessages(timeout: Long, + requiredAcks: Short, + internalTopicsAllowed: Boolean, + messagesPerPartition: Map[TopicAndPartition, MessageSet], + responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { + + if (isValidRequiredAcks(requiredAcks)) { + + val sTime = SystemTime.milliseconds + val localProduceResults = appendToLocalLog(internalTopicsAllowed, messagesPerPartition, requiredAcks) + debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) + + val produceStatus = localProduceResults.map { case (topicAndPartition, result) => + topicAndPartition -> + ProducePartitionStatus( + result.info.lastOffset + 1, // required offset + ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status + } + + if (delayedRequestRequired(requiredAcks, messagesPerPartition, localProduceResults)) { + // create delayed produce operation + val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) + val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback) + + // create a list of (topic, partition) pairs to use as keys for this delayed produce operation + val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq + + // try to complete the request immediately, otherwise put it into the purgatory + // this is because while the delayed produce operation is being created, new + // requests may arrive and hence make this operation completable. + delayedProducePurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) + + } else { + // we can respond immediately + val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) + responseCallback(produceResponseStatus) + } + } else { + // If required.acks is outside accepted range, something is wrong with the client + // Just return an error and don't handle the request at all + val responseStatus = messagesPerPartition.map { + case (topicAndPartition, messageSet) => + (topicAndPartition -> + ProducerResponseStatus(Errors.INVALID_REQUIRED_ACKS.code, + LogAppendInfo.UnknownLogAppendInfo.firstOffset)) + } + responseCallback(responseStatus) + } + } + + // If all the following conditions are true, we need to put a delayed produce request and wait for replication to complete + // + // 1. required acks = -1 + // 2. there is data to append + // 3. at least one partition append was successful (fewer errors than partitions) + private def delayedRequestRequired(requiredAcks: Short, messagesPerPartition: Map[TopicAndPartition, MessageSet], + localProduceResults: Map[TopicAndPartition, LogAppendResult]): Boolean = { + requiredAcks == -1 && + messagesPerPartition.size > 0 && + localProduceResults.values.count(_.error.isDefined) < messagesPerPartition.size + } + + private def isValidRequiredAcks(requiredAcks: Short): Boolean = { + requiredAcks == -1 || requiredAcks == 1 || requiredAcks == 0 + } + + /** + * Append the messages to the local replica logs + */ + private def appendToLocalLog(internalTopicsAllowed: Boolean, + messagesPerPartition: Map[TopicAndPartition, MessageSet], + requiredAcks: Short): Map[TopicAndPartition, LogAppendResult] = { + trace("Append [%s] to local log ".format(messagesPerPartition)) + messagesPerPartition.map { case (topicAndPartition, messages) => + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).totalProduceRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark() + + // reject appending to internal topics if it is not allowed + if (Topic.InternalTopics.contains(topicAndPartition.topic) && !internalTopicsAllowed) { + + (topicAndPartition, LogAppendResult( + LogAppendInfo.UnknownLogAppendInfo, + Some(new InvalidTopicException("Cannot append to internal topic %s".format(topicAndPartition.topic))))) + } else { + try { + val partitionOpt = getPartition(topicAndPartition.topic, topicAndPartition.partition) + val info = partitionOpt match { + case Some(partition) => + partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet], requiredAcks) + case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" + .format(topicAndPartition, localBrokerId)) + } + + val numAppendedMessages = + if (info.firstOffset == -1L || info.lastOffset == -1L) + 0 + else + info.lastOffset - info.firstOffset + 1 + + // update stats for successfully appended bytes and messages as bytesInRate and messageInRate + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) + 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.sizeInBytes, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) + (topicAndPartition, LogAppendResult(info)) + } catch { + // NOTE: Failed produce requests metric is not incremented for known exceptions + // it is supposed to indicate un-expected failures of a broker in handling a produce request + case e: KafkaStorageException => + fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) + Runtime.getRuntime.halt(1) + (topicAndPartition, null) + case utpe: UnknownTopicOrPartitionException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(utpe))) + case nle: NotLeaderForPartitionException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(nle))) + case mtle: MessageSizeTooLargeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mtle))) + case mstle: MessageSetSizeTooLargeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mstle))) + case imse : InvalidMessageSizeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(imse))) + case t: Throwable => + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() + error("Error processing append operation on partition %s".format(topicAndPartition), t) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(t))) + } + } + } + } + + /** + * Fetch messages from the leader replica, and wait until enough data can be fetched and return; + * the callback function will be triggered either when timeout or required fetch info is satisfied + */ + def fetchMessages(timeout: Long, + replicaId: Int, + fetchMinBytes: Int, + fetchInfo: immutable.Map[TopicAndPartition, PartitionFetchInfo], + responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + + val isFromFollower = replicaId >= 0 + val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId + val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) + + // read from local logs + val logReadResults = readFromLocalLog(fetchOnlyFromLeader, fetchOnlyCommitted, fetchInfo) + + // if the fetch comes from the follower, + // update its corresponding log end offset + if(Request.isValidBrokerId(replicaId)) + updateFollowerLogReadResults(replicaId, logReadResults) + + // check if this fetch request can be satisfied right away + val bytesReadable = logReadResults.values.map(_.info.messageSet.sizeInBytes).sum + val errorReadingData = logReadResults.values.foldLeft(false) ((errorIncurred, readResult) => + errorIncurred || (readResult.errorCode != ErrorMapping.NoError)) + + // respond immediately if 1) fetch request does not want to wait + // 2) fetch request does not require any data + // 3) has enough data to respond + // 4) some error happens while reading data + if(timeout <= 0 || fetchInfo.size <= 0 || bytesReadable >= fetchMinBytes || errorReadingData) { + val fetchPartitionData = logReadResults.mapValues(result => + FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + responseCallback(fetchPartitionData) + } else { + // construct the fetch results from the read results + val fetchPartitionStatus = logReadResults.map { case (topicAndPartition, result) => + (topicAndPartition, FetchPartitionStatus(result.info.fetchOffsetMetadata, fetchInfo.get(topicAndPartition).get)) + } + val fetchMetadata = FetchMetadata(fetchMinBytes, fetchOnlyFromLeader, fetchOnlyCommitted, isFromFollower, fetchPartitionStatus) + val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, responseCallback) + + // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation + val delayedFetchKeys = fetchPartitionStatus.keys.map(new TopicPartitionOperationKey(_)).toSeq + + // try to complete the request immediately, otherwise put it into the purgatory; + // this is because while the delayed fetch operation is being created, new requests + // may arrive and hence make this operation completable. + delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys) + } + } + + /** + * Read from a single topic/partition at the given offset upto maxSize bytes + */ + def readFromLocalLog(fetchOnlyFromLeader: Boolean, + readOnlyCommitted: Boolean, + readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]): Map[TopicAndPartition, LogReadResult] = { + + readPartitionInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => + BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.mark() + + val partitionDataAndOffsetInfo = + try { + trace("Fetching log segment for topic %s, partition %d, offset %d, size %d".format(topic, partition, offset, fetchSize)) + + // decide whether to only fetch from leader + val localReplica = if (fetchOnlyFromLeader) + getLeaderReplicaIfLocal(topic, partition) + else + getReplicaOrException(topic, partition) + + // decide whether to only fetch committed data (i.e. messages below high watermark) + val maxOffsetOpt = if (readOnlyCommitted) + Some(localReplica.highWatermark.messageOffset) + else + None + + /* Read the LogOffsetMetadata prior to performing the read from the log. + * We use the LogOffsetMetadata to determine if a particular replica is in-sync or not. + * Using the log end offset after performing the read can lead to a race condition + * where data gets appended to the log immediately after the replica has consumed from it + * This can cause a replica to always be out of sync. + */ + val initialLogEndOffset = localReplica.logEndOffset + val logReadInfo = localReplica.log match { + case Some(log) => + log.read(offset, fetchSize, maxOffsetOpt) + case None => + error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) + FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) + } + + val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0 + + LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, fetchSize, readToEndOfLog, None) + } catch { + // NOTE: Failed fetch requests metric is not incremented for known exceptions since it + // is supposed to indicate un-expected failure of a broker in handling a fetch request + case utpe: UnknownTopicOrPartitionException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(utpe)) + case nle: NotLeaderForPartitionException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(nle)) + case rnae: ReplicaNotAvailableException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(rnae)) + case oor : OffsetOutOfRangeException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(oor)) + case e: Throwable => + BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() + error("Error processing fetch operation on partition [%s,%d] offset %d".format(topic, partition, offset)) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(e)) + } + (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) + } + } + + def maybeUpdateMetadataCache(updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) { + replicaStateChangeLock synchronized { + if(updateMetadataRequest.controllerEpoch < controllerEpoch) { + val stateControllerEpochErrorMessage = ("Broker %d received update metadata request with correlation id %d from an " + + "old controller %d with epoch %d. Latest known controller epoch is %d").format(localBrokerId, + updateMetadataRequest.correlationId, updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, + controllerEpoch) + stateChangeLogger.warn(stateControllerEpochErrorMessage) + throw new ControllerMovedException(stateControllerEpochErrorMessage) + } else { + metadataCache.updateCache(updateMetadataRequest, localBrokerId, stateChangeLogger) + controllerEpoch = updateMetadataRequest.controllerEpoch + } + } + } + + def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): BecomeLeaderOrFollowerResult = { 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) { + val responseMap = new 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)) + stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d since " + + "its controller epoch %d is old. Latest known controller epoch is %d").format(localBrokerId, leaderAndISRRequest.controllerId, + leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerEpoch, controllerEpoch)) } - (responseMap, ErrorMapping.StaleControllerEpochCode) + BecomeLeaderOrFollowerResult(responseMap, Set.empty[Partition], Set.empty[Partition], 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 partitionState = new mutable.HashMap[Partition, PartitionStateInfo]() + leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partitionId), partitionStateInfo) => + val partition = getOrCreatePartition(topic, partitionId) 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 @@ -236,27 +586,34 @@ class ReplicaManager(val config: KafkaConfig, 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)) + stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " + + "epoch %d for partition [%s,%d] as itself is not in assigned replica list %s") + .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, + topic, partition.partitionId, partitionStateInfo.allReplicas.mkString(","))) } } 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)) + stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " + + "epoch %d for partition [%s,%d] since its associated leader epoch %d is old. Current leader epoch is %d") + .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, + topic, partition.partitionId, partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch, partitionLeaderEpoch)) responseMap.put((topic, partitionId), ErrorMapping.StaleLeaderEpochCode) } } - val partitionsTobeLeader = partitionState - .filter{ case (partition, partitionStateInfo) => partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId} + 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) + val partitionsBecomeLeader = if (!partitionsTobeLeader.isEmpty) + makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap) + else + Set.empty[Partition] + val partitionsBecomeFollower = if (!partitionsToBeFollower.isEmpty) + makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap) + else + Set.empty[Partition] // 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 @@ -265,13 +622,13 @@ class ReplicaManager(val config: KafkaConfig, hwThreadInitialized = true } replicaFetcherManager.shutdownIdleFetcherThreads() - (responseMap, ErrorMapping.NoError) + BecomeLeaderOrFollowerResult(responseMap, partitionsBecomeLeader, partitionsBecomeFollower, ErrorMapping.NoError) } } } /* - * Make the current broker to become follower for a given set of partitions by: + * Make the current broker to become leader for a given set of partitions by: * * 1. Stop fetchers for these partitions * 2. Update the partition metadata in cache @@ -281,9 +638,11 @@ class ReplicaManager(val config: KafkaConfig, * 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, + private def makeLeaders(controllerId: Int, + epoch: Int, partitionState: Map[Partition, PartitionStateInfo], - correlationId: Int, responseMap: mutable.Map[(String, Int), Short]) = { + correlationId: Int, + responseMap: mutable.Map[(String, Int), Short]): Set[Partition] = { 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") @@ -304,10 +663,6 @@ class ReplicaManager(val config: KafkaConfig, 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 ++= partitionState.keySet - } } catch { case e: Throwable => partitionState.foreach { state => @@ -325,6 +680,8 @@ class ReplicaManager(val config: KafkaConfig, "for the become-leader transition for partition %s") .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) } + + partitionState.keySet } /* @@ -343,62 +700,86 @@ class ReplicaManager(val config: KafkaConfig, * 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 => + private def makeFollowers(controllerId: Int, + epoch: Int, + partitionState: Map[Partition, PartitionStateInfo], + leaders: Set[BrokerEndPoint], + correlationId: Int, + responseMap: mutable.Map[(String, Int), Short]) : Set[Partition] = { + 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)))) + .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 partitionsToMakeFollower: mutable.Set[Partition] = mutable.Set() + try { - leaderPartitionsLock synchronized { - leaderPartitions --= partitionState.keySet - } - partitionState.foreach{ case (partition, leaderIsrAndControllerEpoch) => - partition.makeFollower(controllerId, leaderIsrAndControllerEpoch, leaders, correlationId)} + // TODO: Delete leaders from LeaderAndIsrRequest + partitionState.foreach{ case (partition, partitionStateInfo) => + val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch + val newLeaderBrokerId = leaderIsrAndControllerEpoch.leaderAndIsr.leader + leaders.find(_.id == newLeaderBrokerId) match { + // Only change partition state when the leader is available + case Some(leaderBroker) => + if (partition.makeFollower(controllerId, partitionStateInfo, correlationId)) + partitionsToMakeFollower += partition + else + stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " + + "controller %d epoch %d for partition [%s,%d] since the new leader %d is the same as the old leader") + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, + partition.topic, partition.partitionId, newLeaderBrokerId)) + case None => + // The leader broker should always be present in the leaderAndIsrRequest. + // If not, we should record the error message and abort the transition process for this partition + stateChangeLogger.error(("Broker %d received LeaderAndIsrRequest with correlation id %d from controller" + + " %d epoch %d for partition [%s,%d] but cannot become follower since the new leader %d is unavailable.") + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, + partition.topic, partition.partitionId, newLeaderBrokerId)) + // Create the local replica even if the leader is unavailable. This is required to ensure that we include + // the partition's high watermark in the checkpoint file (see KAFKA-1647) + partition.getOrCreateReplica() + } + } - replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(new TopicAndPartition(_))) - partitionState.foreach { state => + replicaFetcherManager.removeFetcherForPartitions(partitionsToMakeFollower.map(new TopicAndPartition(_))) + partitionsToMakeFollower.foreach { partition => 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))) + .format(localBrokerId, controllerId, epoch, correlationId, TopicAndPartition(partition.topic, partition.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 " + + logManager.truncateTo(partitionsToMakeFollower.map(partition => (new TopicAndPartition(partition), partition.getOrCreateReplica().highWatermark.messageOffset)).toMap) + + partitionsToMakeFollower.foreach { partition => + stateChangeLogger.trace(("Broker %d truncated logs and checkpointed recovery boundaries for partition [%s,%d] 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)) + partition.topic, partition.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(","))) - } + + if (isShuttingDown.get()) { + partitionsToMakeFollower.foreach { partition => + stateChangeLogger.trace(("Broker %d skipped the adding-fetcher step of the become-follower state change with correlation id %d from " + + "controller %d epoch %d for partition [%s,%d] since it is shutting down").format(localBrokerId, correlationId, + controllerId, epoch, partition.topic, partition.partitionId)) } - 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))) + // we do not need to check if the leader exists again since this has been done at the beginning of this process + val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map(partition => + new TopicAndPartition(partition) -> BrokerAndInitialOffset( + leaders.find(_.id == partition.leaderReplicaIdOpt.get).get, + partition.getReplica().get.logEndOffset.messageOffset)).toMap + replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset) + + partitionsToMakeFollower.foreach { partition => + stateChangeLogger.trace(("Broker %d started fetcher to new leader as part of become-follower request from controller " + + "%d epoch %d with correlation id %d for partition [%s,%d]") + .format(localBrokerId, controllerId, epoch, correlationId, partition.topic, partition.partitionId)) } } } catch { @@ -415,34 +796,41 @@ class ReplicaManager(val config: KafkaConfig, "for the become-follower transition for partition %s") .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) } + + partitionsToMakeFollower } private def maybeShrinkIsr(): Unit = { trace("Evaluating ISR list of partitions to see which replicas can be removed from the ISR") - var curLeaderPartitions: List[Partition] = null - leaderPartitionsLock synchronized { - curLeaderPartitions = leaderPartitions.toList - } - curLeaderPartitions.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages)) + allPartitions.values.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs)) } - def recordFollowerPosition(topic: String, partitionId: Int, replicaId: Int, offset: Long) = { - val partitionOpt = getPartition(topic, partitionId) - if(partitionOpt.isDefined) { - partitionOpt.get.updateLeaderHWAndMaybeExpandIsr(replicaId, offset) - } else { - warn("While recording the follower position, the partition [%s,%d] hasn't been created, skip updating leader HW".format(topic, partitionId)) + private def updateFollowerLogReadResults(replicaId: Int, readResults: Map[TopicAndPartition, LogReadResult]) { + debug("Recording follower broker %d log read results: %s ".format(replicaId, readResults)) + readResults.foreach { case (topicAndPartition, readResult) => + getPartition(topicAndPartition.topic, topicAndPartition.partition) match { + case Some(partition) => + partition.updateReplicaLogReadResult(replicaId, readResult) + + // for producer requests with ack > 1, we need to check + // if they can be unblocked after some follower's log end offsets have moved + tryCompleteDelayedProduce(new TopicPartitionOperationKey(topicAndPartition)) + case None => + warn("While recording the replica LEO, the partition %s hasn't been created.".format(topicAndPartition)) + } } } - /** - * Flushes the highwatermark value for all partitions to the highwatermark file - */ + private def getLeaderPartitions() : List[Partition] = { + allPartitions.values.filter(_.leaderReplicaIfLocal().isDefined).toList + } + + // Flushes the highwatermark value for all partitions to the highwatermark file def checkpointHighWatermarks() { val replicas = allPartitions.values.map(_.getReplica(config.brokerId)).collect{case Some(replica) => replica} - val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParent) + val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParentFile.getAbsolutePath) for((dir, reps) <- replicasByDir) { - val hwms = reps.map(r => (new TopicAndPartition(r) -> r.highWatermark)).toMap + val hwms = reps.map(r => (new TopicAndPartition(r) -> r.highWatermark.messageOffset)).toMap try { highWatermarkCheckpoints(dir).write(hwms) } catch { @@ -453,10 +841,14 @@ class ReplicaManager(val config: KafkaConfig, } } - def shutdown() { - info("Shut down") + // High watermark do not need to be checkpointed only when under unit tests + def shutdown(checkpointHW: Boolean = true) { + info("Shutting down") replicaFetcherManager.shutdown() - checkpointHighWatermarks() - info("Shutted down completely") + delayedFetchPurgatory.shutdown() + delayedProducePurgatory.shutdown() + if (checkpointHW) + checkpointHighWatermarks() + info("Shut down completely") } } diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala deleted file mode 100644 index c064c5c4cf119..0000000000000 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ /dev/null @@ -1,285 +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 scala.collection._ -import java.util.concurrent._ -import java.util.concurrent.atomic._ -import kafka.network._ -import kafka.utils._ -import kafka.metrics.KafkaMetricsGroup -import java.util -import com.yammer.metrics.core.Gauge - - -/** - * A request whose processing needs to be delayed for at most the given delayMs - * The associated keys are used for bookeeping, and represent the "trigger" that causes this request to check if it is satisfied, - * for example a key could be a (topic, partition) pair. - */ -class DelayedRequest(val keys: Seq[Any], val request: RequestChannel.Request, delayMs: Long) extends DelayedItem[RequestChannel.Request](request, delayMs) { - val satisfied = new AtomicBoolean(false) -} - -/** - * A helper class for dealing with asynchronous requests with a timeout. A DelayedRequest has a request to delay - * and also a list of keys that can trigger the action. Implementations can add customized logic to control what it means for a given - * request to be satisfied. For example it could be that we are waiting for user-specified number of acks on a given (topic, partition) - * to be able to respond to a request or it could be that we are waiting for a given number of bytes to accumulate on a given request - * to be able to respond to that request (in the simple case we might wait for at least one byte to avoid busy waiting). - * - * For us the key is generally a (topic, partition) pair. - * By calling - * watch(delayedRequest) - * we will add triggers for each of the given keys. It is up to the user to then call - * val satisfied = update(key, request) - * when a request relevant to the given key occurs. This triggers bookeeping logic and returns back any requests satisfied by this - * new request. - * - * An implementation provides extends two helper functions - * def checkSatisfied(request: R, delayed: T): Boolean - * this function returns true if the given request (in combination with whatever previous requests have happened) satisfies the delayed - * request delayed. This method will likely also need to do whatever bookkeeping is necessary. - * - * The second function is - * def expire(delayed: T) - * this function handles delayed requests that have hit their time limit without being satisfied. - * - */ -abstract class RequestPurgatory[T <: DelayedRequest, R](brokerId: Int = 0, purgeInterval: Int = 10000) - extends Logging with KafkaMetricsGroup { - - /* a list of requests watching each key */ - private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers)) - - private val requestCounter = new AtomicInteger(0) - - newGauge( - "PurgatorySize", - new Gauge[Int] { - def value = watchersForKey.values.map(_.numRequests).sum + expiredRequestReaper.numRequests - } - ) - - newGauge( - "NumDelayedRequests", - new Gauge[Int] { - def value = expiredRequestReaper.unsatisfied.get() - } - ) - - /* background thread expiring requests that have been waiting too long */ - private val expiredRequestReaper = new ExpiredRequestReaper - private val expirationThread = Utils.newThread(name="request-expiration-task", runnable=expiredRequestReaper, daemon=false) - expirationThread.start() - - /** - * Add a new delayed request watching the contained keys - */ - def watch(delayedRequest: T) { - requestCounter.getAndIncrement() - - for(key <- delayedRequest.keys) { - var lst = watchersFor(key) - lst.add(delayedRequest) - } - expiredRequestReaper.enqueue(delayedRequest) - } - - /** - * Update any watchers and return a list of newly satisfied requests. - */ - def update(key: Any, request: R): Seq[T] = { - val w = watchersForKey.get(key) - if(w == null) - Seq.empty - else - w.collectSatisfiedRequests(request) - } - - private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key) - - /** - * Check if this request satisfied this delayed request - */ - protected def checkSatisfied(request: R, delayed: T): Boolean - - /** - * Handle an expired delayed request - */ - protected def expire(delayed: T) - - /** - * Shutdown the expirey thread - */ - def shutdown() { - expiredRequestReaper.shutdown() - } - - /** - * A linked list of DelayedRequests watching some key with some associated - * bookkeeping logic. - */ - private class Watchers { - - - private val requests = new util.ArrayList[T] - - def numRequests = requests.size - - def add(t: T) { - synchronized { - requests.add(t) - } - } - - def purgeSatisfied(): Int = { - synchronized { - val iter = requests.iterator() - var purged = 0 - while(iter.hasNext) { - val curr = iter.next - if(curr.satisfied.get()) { - iter.remove() - purged += 1 - } - } - purged - } - } - - def collectSatisfiedRequests(request: R): Seq[T] = { - val response = new mutable.ArrayBuffer[T] - synchronized { - val iter = requests.iterator() - while(iter.hasNext) { - val curr = iter.next - if(curr.satisfied.get) { - // another thread has satisfied this request, remove it - iter.remove() - } else { - // synchronize on curr to avoid any race condition with expire - // on client-side. - val satisfied = curr synchronized checkSatisfied(request, curr) - if(satisfied) { - iter.remove() - val updated = curr.satisfied.compareAndSet(false, true) - if(updated == true) { - response += curr - expiredRequestReaper.satisfyRequest() - } - } - } - } - } - response - } - } - - /** - * Runnable to expire requests that have sat unfullfilled past their deadline - */ - private class ExpiredRequestReaper extends Runnable with Logging { - this.logIdent = "ExpiredRequestReaper-%d ".format(brokerId) - - private val delayed = new DelayQueue[T] - private val running = new AtomicBoolean(true) - private val shutdownLatch = new CountDownLatch(1) - - /* The count of elements in the delay queue that are unsatisfied */ - private [kafka] val unsatisfied = new AtomicInteger(0) - - def numRequests = delayed.size() - - /** Main loop for the expiry thread */ - def run() { - while(running.get) { - try { - val curr = pollExpired() - if (curr != null) { - curr synchronized { - expire(curr) - } - } - if (requestCounter.get >= purgeInterval) { // see if we need to force a full purge - requestCounter.set(0) - val purged = purgeSatisfied() - debug("Purged %d requests from delay queue.".format(purged)) - val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum - debug("Purged %d (watcher) requests.".format(numPurgedFromWatchers)) - } - } catch { - case e: Exception => - error("Error in long poll expiry thread: ", e) - } - } - shutdownLatch.countDown() - } - - /** Add a request to be expired */ - def enqueue(t: T) { - delayed.add(t) - unsatisfied.incrementAndGet() - } - - /** Shutdown the expiry thread*/ - def shutdown() { - debug("Shutting down.") - running.set(false) - shutdownLatch.await() - debug("Shut down complete.") - } - - /** Record the fact that we satisfied a request in the stats for the expiry queue */ - def satisfyRequest(): Unit = unsatisfied.getAndDecrement() - - /** - * Get the next expired event - */ - private def pollExpired(): T = { - while(true) { - val curr = delayed.poll(200L, TimeUnit.MILLISECONDS) - if (curr == null) - return null.asInstanceOf[T] - val updated = curr.satisfied.compareAndSet(false, true) - if(updated) { - unsatisfied.getAndDecrement() - return curr - } - } - throw new RuntimeException("This should not happen") - } - - /** - * Delete all expired events from the delay queue - */ - private def purgeSatisfied(): Int = { - var purged = 0 - val iter = delayed.iterator() - while(iter.hasNext) { - val curr = iter.next() - if(curr.satisfied.get) { - iter.remove() - purged += 1 - } - } - purged - } - } - -} diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala index d41fd33d91406..01b1b0a8efe6a 100644 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ b/core/src/main/scala/kafka/server/TopicConfigManager.scala @@ -30,16 +30,17 @@ import org.I0Itec.zkclient.{IZkChildListener, ZkClient} * It works as follows. * * Config is stored under the path - * /brokers/topics//config + * /config/topics/ * 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 + * /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 + * /config/changes/config_change_13321 + * This is just a notification--the actual config change is stored only once under the /config/topics/ path. * * 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 @@ -59,7 +60,7 @@ import org.I0Itec.zkclient.{IZkChildListener, ZkClient} */ class TopicConfigManager(private val zkClient: ZkClient, private val logManager: LogManager, - private val changeExpirationMs: Long = 10*60*1000, + private val changeExpirationMs: Long = 15*60*1000, private val time: Time = SystemTime) extends Logging { private var lastExecutedChange = -1L @@ -86,7 +87,7 @@ class TopicConfigManager(private val zkClient: ZkClient, */ private def processConfigChanges(notifications: Seq[String]) { if (notifications.size > 0) { - info("Processing %d topic config change notification(s)...".format(notifications.size)) + info("Processing config change notification(s)...") val now = time.milliseconds val logs = logManager.logsByTopicPartition.toBuffer val logsByTopic = logs.groupBy(_._1.topic).mapValues(_.map(_._2)) @@ -94,26 +95,38 @@ class TopicConfigManager(private val zkClient: ZkClient, 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 */ - error("Ignoring topic config change %d for topic %s since the change has expired") - } else { - error("Ignoring topic config change %d for topic %s since the topic may have been deleted") + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode) + if(jsonOpt.isDefined) { + val json = jsonOpt.get + val topic = json.substring(1, json.length - 1) // hacky way to dequote + if (logsByTopic.contains(topic)) { + /* combine the default properties with the overrides in zk to create the new LogConfig */ + val props = new Properties() + props.putAll(logManager.defaultConfig.originals) + props.putAll(AdminUtils.fetchTopicConfig(zkClient, topic)) + val logConfig = LogConfig(props) + for (log <- logsByTopic(topic)) + log.config = logConfig + info("Processed topic config change %d for topic %s, setting new config to %s.".format(changeId, topic, props)) + purgeObsoleteNotifications(now, notifications) } - ZkUtils.deletePath(zkClient, changeZnode) } + lastExecutedChange = changeId + } + } + } + } + + private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) { + for(notification <- notifications.sorted) { + val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.TopicConfigChangesPath + "/" + notification) + if(jsonOpt.isDefined) { + val changeZnode = ZkUtils.TopicConfigChangesPath + "/" + notification + if (now - stat.getCtime > changeExpirationMs) { + debug("Purging config change notification " + notification) + ZkUtils.deletePath(zkClient, changeZnode) + } else { + return } } } diff --git a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala old mode 100644 new mode 100755 index e5b6ff1e2544b..a5c5fb3a9385d --- 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._ +import kafka.utils.CoreUtils._ import kafka.utils.{Json, SystemTime, Logging} import org.I0Itec.zkclient.exception.ZkNodeExistsException import org.I0Itec.zkclient.IZkDataListener @@ -50,9 +50,27 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, } } + private def getControllerID(): Int = { + readDataMaybeNull(controllerContext.zkClient, electionPath)._1 match { + case Some(controller) => KafkaController.parseControllerId(controller) + case None => -1 + } + } + def elect: Boolean = { val timestamp = SystemTime.milliseconds.toString val electString = Json.encode(Map("version" -> 1, "brokerid" -> brokerId, "timestamp" -> timestamp)) + + leaderId = getControllerID + /* + * We can get here during the initial startup and the handleDeleted ZK callback. Because of the potential race condition, + * it's possible that the controller has already been elected when we get here. This check will prevent the following + * createEphemeralPath method from getting into an infinite loop if this broker is already the controller. + */ + if(leaderId != -1) { + debug("Broker %d has been elected as leader, so stopping the election process.".format(leaderId)) + return amILeader + } try { createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, electionPath, electString, brokerId, @@ -64,15 +82,13 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, } 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 - } - } + leaderId = getControllerID + if (leaderId != -1) debug("Broker %d was elected as leader instead of broker %d".format(leaderId, brokerId)) + else + warn("A leader has been elected but just resigned, this will result in another round of election") + case e2: Throwable => error("Error while electing or becoming leader on broker %d".format(brokerId), e2) resign() diff --git a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala old mode 100644 new mode 100755 similarity index 66% rename from core/src/main/scala/kafka/consumer/ConsoleConsumer.scala rename to core/src/main/scala/kafka/tools/ConsoleConsumer.scala index dc066c23e9c22..a3bee58a207ec --- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package kafka.consumer +package kafka.tools import scala.collection.JavaConversions._ import org.I0Itec.zkclient._ @@ -27,7 +27,8 @@ import kafka.message._ import kafka.serializer._ import kafka.utils._ import kafka.metrics.KafkaMetricsReporter - +import kafka.consumer.{Blacklist,Whitelist,ConsumerConfig,Consumer} +import org.apache.kafka.common.utils.Utils /** * Consumer that dumps messages out to standard out. @@ -54,47 +55,11 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("urls") .ofType(classOf[String]) - val groupIdOpt = parser.accepts("group", "The group id to consume on.") + + val consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.") .withRequiredArg - .describedAs("gid") - .defaultsTo("console-consumer-" + new Random().nextInt(100000)) + .describedAs("config file") .ofType(classOf[String]) - val fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.") - .withRequiredArg - .describedAs("size") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1024 * 1024) - val minFetchBytesOpt = parser.accepts("min-fetch-bytes", "The min number of bytes each fetch request waits for.") - .withRequiredArg - .describedAs("bytes") - .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(100) - val socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.") - .withRequiredArg - .describedAs("size") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(2 * 1024 * 1024) - val socketTimeoutMsOpt = parser.accepts("socket-timeout-ms", "The socket timeout used for the connection to the broker") - .withRequiredArg - .describedAs("ms") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(ConsumerConfig.SocketTimeout) - val refreshMetadataBackoffMsOpt = parser.accepts("refresh-leader-backoff-ms", "Backoff time before refreshing metadata") - .withRequiredArg - .describedAs("ms") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(ConsumerConfig.RefreshMetadataBackoffMs) - val consumerTimeoutMsOpt = parser.accepts("consumer-timeout-ms", "consumer throws timeout exception after waiting this much " + - "of time without incoming messages") - .withRequiredArg - .describedAs("prop") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(-1) val messageFormatterOpt = parser.accepts("formatter", "The name of a class to use for formatting kafka messages for display.") .withRequiredArg .describedAs("class") @@ -104,13 +69,9 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("prop") .ofType(classOf[String]) + val deleteConsumerOffsetsOpt = parser.accepts("delete-consumer-offsets", "If specified, the consumer path in zookeeper is deleted when starting up") val resetBeginningOpt = parser.accepts("from-beginning", "If the consumer does not already have an established offset to consume from, " + "start with the earliest message present in the log rather than the latest message.") - val autoCommitIntervalOpt = parser.accepts("autocommit.interval.ms", "The time interval at which to save the current offset in ms") - .withRequiredArg - .describedAs("ms") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(ConsumerConfig.AutoCommitInterval) val maxMessagesOpt = parser.accepts("max-messages", "The maximum number of messages to consume before exiting. If not set, consumption is continual.") .withRequiredArg .describedAs("num_messages") @@ -124,15 +85,15 @@ object ConsoleConsumer extends Logging { .describedAs("metrics dictory") .ofType(classOf[java.lang.String]) - + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "The console consumer is a tool that reads data from Kafka and outputs it to standard output.") + + var groupIdPassed = true val options: OptionSet = tryParse(parser, args) CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt) val topicOrFilterOpt = List(topicIdOpt, whitelistOpt, blacklistOpt).filter(options.has) - if (topicOrFilterOpt.size != 1) { - error("Exactly one of whitelist/blacklist/topic is required.") - parser.printHelpOn(System.err) - System.exit(1) - } + if (topicOrFilterOpt.size != 1) + CommandLineUtils.printUsageAndDie(parser, "Exactly one of whitelist/blacklist/topic is required.") val topicArg = options.valueOf(topicOrFilterOpt.head) val filterSpec = if (options.has(blacklistOpt)) new Blacklist(topicArg) @@ -153,39 +114,46 @@ object ConsoleConsumer extends Logging { KafkaMetricsReporter.startReporters(verifiableProps) } - val props = new Properties() - props.put("group.id", options.valueOf(groupIdOpt)) - props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) - props.put("socket.timeout.ms", options.valueOf(socketTimeoutMsOpt).toString) - props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString) - props.put("fetch.min.bytes", options.valueOf(minFetchBytesOpt).toString) - props.put("fetch.wait.max.ms", options.valueOf(maxWaitMsOpt).toString) - props.put("auto.commit.enable", "true") - props.put("auto.commit.interval.ms", options.valueOf(autoCommitIntervalOpt).toString) - props.put("auto.offset.reset", if(options.has(resetBeginningOpt)) "smallest" else "largest") - props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) - props.put("consumer.timeout.ms", options.valueOf(consumerTimeoutMsOpt).toString) - props.put("refresh.leader.backoff.ms", options.valueOf(refreshMetadataBackoffMsOpt).toString) + val consumerProps = if (options.has(consumerConfigOpt)) + Utils.loadProps(options.valueOf(consumerConfigOpt)) + else + new Properties() - val config = new ConsumerConfig(props) - val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false + if(!consumerProps.containsKey("group.id")) { + consumerProps.put("group.id","console-consumer-" + new Random().nextInt(100000)) + groupIdPassed=false + } + consumerProps.put("auto.offset.reset", if(options.has(resetBeginningOpt)) "smallest" else "largest") + consumerProps.put("zookeeper.connect", options.valueOf(zkConnectOpt)) - val messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)) - val formatterArgs = MessageFormatter.tryParseFormatterArgs(options.valuesOf(messageFormatterArgOpt)) + if (!checkZkPathExists(options.valueOf(zkConnectOpt),"/brokers/ids")) { + System.err.println("No brokers found.") + System.exit(1) + } - val maxMessages = if(options.has(maxMessagesOpt)) options.valueOf(maxMessagesOpt).intValue else -1 + if (!options.has(deleteConsumerOffsetsOpt) && options.has(resetBeginningOpt) && + checkZkPathExists(options.valueOf(zkConnectOpt),"/consumers/" + consumerProps.getProperty("group.id")+ "/offsets")) { + System.err.println("Found previous offset information for this group "+consumerProps.getProperty("group.id") + +". Please use --delete-consumer-offsets to delete previous offsets metadata") + System.exit(1) + } - val connector = Consumer.create(config) + if(options.has(deleteConsumerOffsetsOpt)) + ZkUtils.maybeDeletePath(options.valueOf(zkConnectOpt), "/consumers/" + consumerProps.getProperty("group.id")) - if(options.has(resetBeginningOpt)) - ZkUtils.maybeDeletePath(options.valueOf(zkConnectOpt), "/consumers/" + options.valueOf(groupIdOpt)) + val config = new ConsumerConfig(consumerProps) + val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false + val messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)) + val formatterArgs = CommandLineUtils.parseKeyValueArgs(options.valuesOf(messageFormatterArgOpt)) + val maxMessages = if(options.has(maxMessagesOpt)) options.valueOf(maxMessagesOpt).intValue else -1 + val connector = Consumer.create(config) Runtime.getRuntime.addShutdownHook(new Thread() { override def run() { connector.shutdown() // if there is no group specified then avoid polluting zookeeper with persistent group data, this is a hack - if(!options.has(groupIdOpt)) - ZkUtils.maybeDeletePath(options.valueOf(zkConnectOpt), "/consumers/" + options.valueOf(groupIdOpt)) + if(!groupIdPassed) + ZkUtils.maybeDeletePath(options.valueOf(zkConnectOpt), "/consumers/" + consumerProps.get("group.id")) } }) @@ -239,34 +207,16 @@ object ConsoleConsumer extends Logging { } } - def tryCleanupZookeeper(zkUrl: String, groupId: String) { + def checkZkPathExists(zkUrl: String, path: String): Boolean = { try { - val dir = "/consumers/" + groupId - info("Cleaning up temporary zookeeper data under " + dir + ".") - val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) - zk.deleteRecursive(dir) - zk.close() + val zk = ZkUtils.createZkClient(zkUrl, 30*1000,30*1000); + zk.exists(path) } catch { - case _: Throwable => // swallow + case _: Throwable => false } } } - -object MessageFormatter { - def tryParseFormatterArgs(args: Iterable[String]): Properties = { - val splits = args.map(_ split "=").filterNot(_ == null).filterNot(_.length == 0) - if(!splits.forall(_.length == 2)) { - System.err.println("Invalid parser arguments: " + args.mkString(" ")) - System.exit(1) - } - val props = new Properties - for(a <- splits) - props.put(a(0), a(1)) - props - } -} - trait MessageFormatter { def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) def init(props: Properties) {} @@ -277,7 +227,7 @@ class DefaultMessageFormatter extends MessageFormatter { var printKey = false var keySeparator = "\t".getBytes var lineSeparator = "\n".getBytes - + override def init(props: Properties) { if(props.containsKey("print.key")) printKey = props.getProperty("print.key").trim.toLowerCase.equals("true") @@ -286,7 +236,7 @@ class DefaultMessageFormatter extends MessageFormatter { if(props.containsKey("line.separator")) lineSeparator = props.getProperty("line.separator").getBytes } - + def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) { if(printKey) { output.write(if (key == null) "null".getBytes() else key) diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala new file mode 100644 index 0000000000000..6971e6e4dcc15 --- /dev/null +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -0,0 +1,325 @@ +/** + * 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 kafka.common._ +import kafka.message._ +import kafka.serializer._ +import kafka.utils.{ToolsUtils, CommandLineUtils} +import kafka.producer.{NewShinyProducer,OldProducer,KeyedMessage} + +import java.util.Properties +import java.io._ + +import joptsimple._ +import org.apache.kafka.clients.producer.ProducerConfig + +object ConsoleProducer { + + def main(args: Array[String]) { + + val config = new ProducerConfig(args) + val reader = Class.forName(config.readerClass).newInstance().asInstanceOf[MessageReader] + reader.init(System.in, getReaderProps(config)) + + try { + val producer = + if(config.useNewProducer) { + new NewShinyProducer(getNewProducerProps(config)) + } else { + new OldProducer(getOldProducerProps(config)) + } + + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run() { + producer.close() + } + }) + + var message: KeyedMessage[Array[Byte], Array[Byte]] = null + do { + message = reader.readMessage() + if(message != null) + producer.send(message.topic, message.key, message.message) + } while(message != null) + } catch { + case e: Exception => + e.printStackTrace + System.exit(1) + } + System.exit(0) + } + + def getReaderProps(config: ProducerConfig): Properties = { + val props = new Properties + props.put("topic",config.topic) + props.putAll(config.cmdLineProps) + props + } + + def getOldProducerProps(config: ProducerConfig): Properties = { + + val props = new Properties + + props.putAll(config.extraProducerProps) + + props.put("metadata.broker.list", config.brokerList) + props.put("compression.codec", config.compressionCodec) + props.put("producer.type", if(config.sync) "sync" else "async") + props.put("batch.num.messages", config.batchSize.toString) + props.put("message.send.max.retries", config.messageSendMaxRetries.toString) + props.put("retry.backoff.ms", config.retryBackoffMs.toString) + props.put("queue.buffering.max.ms", config.sendTimeout.toString) + props.put("queue.buffering.max.messages", config.queueSize.toString) + props.put("queue.enqueue.timeout.ms", config.queueEnqueueTimeoutMs.toString) + props.put("request.required.acks", config.requestRequiredAcks.toString) + props.put("request.timeout.ms", config.requestTimeoutMs.toString) + props.put("key.serializer.class", config.keyEncoderClass) + props.put("serializer.class", config.valueEncoderClass) + props.put("send.buffer.bytes", config.socketBuffer.toString) + props.put("topic.metadata.refresh.interval.ms", config.metadataExpiryMs.toString) + props.put("client.id", "console-producer") + + props + } + + def getNewProducerProps(config: ProducerConfig): Properties = { + + val props = new Properties + + props.putAll(config.extraProducerProps) + + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.brokerList) + props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, config.compressionCodec) + props.put(ProducerConfig.SEND_BUFFER_CONFIG, config.socketBuffer.toString) + props.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, config.retryBackoffMs.toString) + props.put(ProducerConfig.METADATA_MAX_AGE_CONFIG, config.metadataExpiryMs.toString) + props.put(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG, config.metadataFetchTimeoutMs.toString) + props.put(ProducerConfig.ACKS_CONFIG, config.requestRequiredAcks.toString) + props.put(ProducerConfig.TIMEOUT_CONFIG, config.requestTimeoutMs.toString) + props.put(ProducerConfig.RETRIES_CONFIG, config.messageSendMaxRetries.toString) + props.put(ProducerConfig.LINGER_MS_CONFIG, config.sendTimeout.toString) + if(config.queueEnqueueTimeoutMs != -1) + props.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "false") + props.put(ProducerConfig.BUFFER_MEMORY_CONFIG, config.maxMemoryBytes.toString) + props.put(ProducerConfig.BATCH_SIZE_CONFIG, config.maxPartitionMemoryBytes.toString) + props.put(ProducerConfig.CLIENT_ID_CONFIG, "console-producer") + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + + props + } + + class ProducerConfig(args: Array[String]) { + val parser = new OptionParser + val topicOpt = parser.accepts("topic", "REQUIRED: The topic id to produce messages to.") + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) + val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.") + .withRequiredArg + .describedAs("broker-list") + .ofType(classOf[String]) + val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") + val compressionCodecOpt = parser.accepts("compression-codec", "The compression codec: either 'none', 'gzip', 'snappy', or 'lz4'." + + "If specified without value, then it defaults to 'gzip'") + .withOptionalArg() + .describedAs("compression-codec") + .ofType(classOf[String]) + val batchSizeOpt = parser.accepts("batch-size", "Number of messages to send in a single batch if they are not being sent synchronously.") + .withRequiredArg + .describedAs("size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(200) + val messageSendMaxRetriesOpt = parser.accepts("message-send-max-retries", "Brokers can fail receiving the message for multiple reasons, and being unavailable transiently is just one of them. This property specifies the number of retires before the producer give up and drop this message.") + .withRequiredArg + .ofType(classOf[java.lang.Integer]) + .defaultsTo(3) + val retryBackoffMsOpt = parser.accepts("retry-backoff-ms", "Before each retry, the producer refreshes the metadata of relevant topics. Since leader election takes a bit of time, this property specifies the amount of time that the producer waits before refreshing the metadata.") + .withRequiredArg + .ofType(classOf[java.lang.Integer]) + .defaultsTo(100) + val sendTimeoutOpt = parser.accepts("timeout", "If set and the producer is running in asynchronous mode, this gives the maximum amount of time" + + " a message will queue awaiting sufficient batch size. The value is given in ms.") + .withRequiredArg + .describedAs("timeout_ms") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1000) + val queueSizeOpt = parser.accepts("queue-size", "If set and the producer is running in asynchronous mode, this gives the maximum amount of " + + " messages will queue awaiting sufficient batch size.") + .withRequiredArg + .describedAs("queue_size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(10000) + val queueEnqueueTimeoutMsOpt = parser.accepts("queue-enqueuetimeout-ms", "Timeout for event enqueue") + .withRequiredArg + .describedAs("queue enqueuetimeout ms") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(Int.MaxValue) + val requestRequiredAcksOpt = parser.accepts("request-required-acks", "The required acks of the producer requests") + .withRequiredArg + .describedAs("request required acks") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(0) + val requestTimeoutMsOpt = parser.accepts("request-timeout-ms", "The ack timeout of the producer requests. Value must be non-negative and non-zero") + .withRequiredArg + .describedAs("request timeout ms") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1500) + val metadataExpiryMsOpt = parser.accepts("metadata-expiry-ms", + "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any leadership changes.") + .withRequiredArg + .describedAs("metadata expiration interval") + .ofType(classOf[java.lang.Long]) + .defaultsTo(5*60*1000L) + val metadataFetchTimeoutMsOpt = parser.accepts("metadata-fetch-timeout-ms", + "The amount of time to block waiting to fetch metadata about a topic the first time a record is sent to that topic.") + .withRequiredArg + .describedAs("metadata fetch timeout") + .ofType(classOf[java.lang.Long]) + .defaultsTo(60*1000L) + val maxMemoryBytesOpt = parser.accepts("max-memory-bytes", + "The total memory used by the producer to buffer records waiting to be sent to the server.") + .withRequiredArg + .describedAs("total memory in bytes") + .ofType(classOf[java.lang.Long]) + .defaultsTo(32 * 1024 * 1024L) + val maxPartitionMemoryBytesOpt = parser.accepts("max-partition-memory-bytes", + "The buffer size allocated for a partition. When records are received which are smaller than this size the producer " + + "will attempt to optimistically group them together until this size is reached.") + .withRequiredArg + .describedAs("memory in bytes per partition") + .ofType(classOf[java.lang.Long]) + .defaultsTo(16 * 1024L) + val valueEncoderOpt = parser.accepts("value-serializer", "The class name of the message encoder implementation to use for serializing values.") + .withRequiredArg + .describedAs("encoder_class") + .ofType(classOf[java.lang.String]) + .defaultsTo(classOf[DefaultEncoder].getName) + val keyEncoderOpt = parser.accepts("key-serializer", "The class name of the message encoder implementation to use for serializing keys.") + .withRequiredArg + .describedAs("encoder_class") + .ofType(classOf[java.lang.String]) + .defaultsTo(classOf[DefaultEncoder].getName) + val messageReaderOpt = parser.accepts("line-reader", "The class name of the class to use for reading lines from standard in. " + + "By default each line is read as a separate message.") + .withRequiredArg + .describedAs("reader_class") + .ofType(classOf[java.lang.String]) + .defaultsTo(classOf[LineMessageReader].getName) + val socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.") + .withRequiredArg + .describedAs("size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1024*100) + val propertyOpt = parser.accepts("property", "A mechanism to pass user-defined properties in the form key=value to the message reader. " + + "This allows custom configuration for a user-defined message reader.") + .withRequiredArg + .describedAs("prop") + .ofType(classOf[String]) + val producerPropertyOpt = parser.accepts("producer-property", "A mechanism to pass user-defined properties in the form key=value to the producer. ") + .withRequiredArg + .describedAs("producer_prop") + .ofType(classOf[String]) + val useNewProducerOpt = parser.accepts("new-producer", "Use the new producer implementation.") + + val options = parser.parse(args : _*) + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "Read data from standard input and publish it to Kafka.") + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, brokerListOpt) + + import scala.collection.JavaConversions._ + val useNewProducer = options.has(useNewProducerOpt) + val topic = options.valueOf(topicOpt) + val brokerList = options.valueOf(brokerListOpt) + ToolsUtils.validatePortOrDie(parser,brokerList) + val sync = options.has(syncOpt) + val compressionCodecOptionValue = options.valueOf(compressionCodecOpt) + val compressionCodec = if (options.has(compressionCodecOpt)) + if (compressionCodecOptionValue == null || compressionCodecOptionValue.isEmpty) + DefaultCompressionCodec.name + else compressionCodecOptionValue + else NoCompressionCodec.name + val batchSize = options.valueOf(batchSizeOpt) + val sendTimeout = options.valueOf(sendTimeoutOpt) + val queueSize = options.valueOf(queueSizeOpt) + val queueEnqueueTimeoutMs = options.valueOf(queueEnqueueTimeoutMsOpt) + val requestRequiredAcks = options.valueOf(requestRequiredAcksOpt) + val requestTimeoutMs = options.valueOf(requestTimeoutMsOpt) + val messageSendMaxRetries = options.valueOf(messageSendMaxRetriesOpt) + val retryBackoffMs = options.valueOf(retryBackoffMsOpt) + val keyEncoderClass = options.valueOf(keyEncoderOpt) + val valueEncoderClass = options.valueOf(valueEncoderOpt) + val readerClass = options.valueOf(messageReaderOpt) + val socketBuffer = options.valueOf(socketBufferSizeOpt) + val cmdLineProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(propertyOpt)) + val extraProducerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(producerPropertyOpt)) + /* new producer related configs */ + val maxMemoryBytes = options.valueOf(maxMemoryBytesOpt) + val maxPartitionMemoryBytes = options.valueOf(maxPartitionMemoryBytesOpt) + val metadataExpiryMs = options.valueOf(metadataExpiryMsOpt) + val metadataFetchTimeoutMs = options.valueOf(metadataFetchTimeoutMsOpt) + } + + trait MessageReader { + def init(inputStream: InputStream, props: Properties) {} + def readMessage(): KeyedMessage[Array[Byte], Array[Byte]] + def close() {} + } + + class LineMessageReader extends MessageReader { + var topic: String = null + var reader: BufferedReader = null + var parseKey = false + var keySeparator = "\t" + var ignoreError = false + var lineNumber = 0 + + override def init(inputStream: InputStream, props: Properties) { + topic = props.getProperty("topic") + if(props.containsKey("parse.key")) + parseKey = props.getProperty("parse.key").trim.toLowerCase.equals("true") + if(props.containsKey("key.separator")) + keySeparator = props.getProperty("key.separator") + if(props.containsKey("ignore.error")) + ignoreError = props.getProperty("ignore.error").trim.toLowerCase.equals("true") + reader = new BufferedReader(new InputStreamReader(inputStream)) + } + + override def readMessage() = { + lineNumber += 1 + (reader.readLine(), parseKey) match { + case (null, _) => null + case (line, true) => + line.indexOf(keySeparator) match { + case -1 => + if(ignoreError) + new KeyedMessage[Array[Byte], Array[Byte]](topic, line.getBytes()) + else + throw new KafkaException("No key found on line " + lineNumber + ": " + line) + case n => + new KeyedMessage[Array[Byte], Array[Byte]](topic, + line.substring(0, n).getBytes, + (if(n + keySeparator.size > line.size) "" else line.substring(n + keySeparator.size)).getBytes()) + } + case (line, false) => + new KeyedMessage[Array[Byte], Array[Byte]](topic, line.getBytes()) + } + } + } +} diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index 33d7c2c4aacc9..3d52f62c88a50 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -20,16 +20,22 @@ package kafka.tools import joptsimple._ import org.I0Itec.zkclient.ZkClient -import kafka.utils.{Json, ZkUtils, ZKStringSerializer, Logging} +import kafka.utils._ import kafka.consumer.SimpleConsumer -import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} -import kafka.common.{BrokerNotAvailableException, TopicAndPartition} +import kafka.api.{OffsetFetchResponse, OffsetFetchRequest, OffsetRequest} +import kafka.common.{OffsetMetadataAndError, ErrorMapping, BrokerNotAvailableException, TopicAndPartition} +import org.apache.kafka.common.protocol.SecurityProtocol import scala.collection._ - +import kafka.client.ClientUtils +import kafka.network.BlockingChannel +import kafka.api.PartitionOffsetRequestInfo +import org.I0Itec.zkclient.exception.ZkNoNodeException object ConsumerOffsetChecker extends Logging { private val consumerMap: mutable.Map[Int, Option[SimpleConsumer]] = mutable.Map() + private val offsetMap: mutable.Map[TopicAndPartition, Long] = mutable.Map() + private var topicPidMap: immutable.Map[String, Seq[Int]] = immutable.Map() private def getConsumer(zkClient: ZkClient, bid: Int): Option[SimpleConsumer] = { try { @@ -49,18 +55,17 @@ object ConsumerOffsetChecker extends Logging { } } catch { case t: Throwable => - error("Could not parse broker info", t) + println("Could not parse broker info due to " + t.getCause) None } } private def processPartition(zkClient: ZkClient, group: String, topic: String, pid: Int) { - val offset = ZkUtils.readData(zkClient, "/consumers/%s/offsets/%s/%s". - format(group, topic, pid))._1.toLong - val owner = ZkUtils.readDataMaybeNull(zkClient, "/consumers/%s/owners/%s/%s". - format(group, topic, pid))._1 - + val topicPartition = TopicAndPartition(topic, pid) + val offsetOpt = offsetMap.get(topicPartition) + val groupDirs = new ZKGroupTopicDirs(group, topic) + val owner = ZkUtils.readDataMaybeNull(zkClient, groupDirs.consumerOwnerDir + "/%s".format(pid))._1 ZkUtils.getLeaderForPartition(zkClient, topic, pid) match { case Some(bid) => val consumerOpt = consumerMap.getOrElseUpdate(bid, getConsumer(zkClient, bid)) @@ -71,19 +76,18 @@ object ConsumerOffsetChecker extends Logging { OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1))) val logSize = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head - val lag = logSize - offset - println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format(group, topic, pid, offset, logSize, lag, - owner match {case Some(ownerStr) => ownerStr case None => "none"})) + val lagString = offsetOpt.map(o => if (o == -1) "unknown" else (logSize - o).toString) + println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format(group, topic, pid, offsetOpt.getOrElse("unknown"), logSize, lagString.getOrElse("unknown"), + owner match {case Some(ownerStr) => ownerStr case None => "none"})) case None => // ignore } case None => - error("No broker for partition %s - %s".format(topic, pid)) + println("No broker for partition %s - %s".format(topic, pid)) } } private def processTopic(zkClient: ZkClient, group: String, topic: String) { - val pidMap = ZkUtils.getPartitionsForTopics(zkClient, Seq(topic)) - pidMap.get(topic) match { + topicPidMap.get(topic) match { case Some(pids) => pids.sorted.foreach { pid => processPartition(zkClient, group, topic, pid) @@ -105,15 +109,23 @@ object ConsumerOffsetChecker extends Logging { def main(args: Array[String]) { val parser = new OptionParser() - val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string."). - withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]); + val zkConnectOpt = parser.accepts("zookeeper", "ZooKeeper connect string."). + withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]) val topicsOpt = parser.accepts("topic", "Comma-separated list of consumer topics (all topics if absent)."). withRequiredArg().ofType(classOf[String]) val groupOpt = parser.accepts("group", "Consumer group."). withRequiredArg().ofType(classOf[String]) + val channelSocketTimeoutMsOpt = parser.accepts("socket.timeout.ms", "Socket timeout to use when querying for offsets."). + withRequiredArg().ofType(classOf[java.lang.Integer]).defaultsTo(6000) + val channelRetryBackoffMsOpt = parser.accepts("retry.backoff.ms", "Retry back-off to use for failed offset queries."). + withRequiredArg().ofType(classOf[java.lang.Integer]).defaultsTo(3000) + parser.accepts("broker-info", "Print broker info") parser.accepts("help", "Print this message.") + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "Check the offset of your consumers.") val options = parser.parse(args : _*) @@ -122,31 +134,60 @@ object ConsumerOffsetChecker extends Logging { System.exit(0) } - for (opt <- List(groupOpt)) - if (!options.has(opt)) { - System.err.println("Missing required argument: %s".format(opt)) - parser.printHelpOn(System.err) - System.exit(1) - } + CommandLineUtils.checkRequiredArgs(parser, options, groupOpt, zkConnectOpt) val zkConnect = options.valueOf(zkConnectOpt) + val group = options.valueOf(groupOpt) - val topics = if (options.has(topicsOpt)) Some(options.valueOf(topicsOpt)) - else None + val groupDirs = new ZKGroupDirs(group) + + val channelSocketTimeoutMs = options.valueOf(channelSocketTimeoutMsOpt).intValue() + val channelRetryBackoffMs = options.valueOf(channelRetryBackoffMsOpt).intValue() + val topics = if (options.has(topicsOpt)) Some(options.valueOf(topicsOpt)) else None var zkClient: ZkClient = null + var channel: BlockingChannel = null try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) val topicList = topics match { case Some(x) => x.split(",").view.toList - case None => ZkUtils.getChildren( - zkClient, "/consumers/%s/offsets".format(group)).toList + case None => ZkUtils.getChildren(zkClient, groupDirs.consumerGroupDir + "/owners").toList } - debug("zkConnect = %s; topics = %s; group = %s".format( - zkConnect, topicList.toString(), group)) + topicPidMap = immutable.Map(ZkUtils.getPartitionsForTopics(zkClient, topicList).toSeq:_*) + val topicPartitions = topicPidMap.flatMap { case(topic, partitionSeq) => partitionSeq.map(TopicAndPartition(topic, _)) }.toSeq + val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs) + + debug("Sending offset fetch request to coordinator %s:%d.".format(channel.host, channel.port)) + channel.send(OffsetFetchRequest(group, topicPartitions)) + val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload()) + debug("Received offset fetch response %s.".format(offsetFetchResponse)) + + offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => + if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) { + val topicDirs = new ZKGroupTopicDirs(group, topicAndPartition.topic) + // this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool + // (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage) + try { + val offset = ZkUtils.readData(zkClient, topicDirs.consumerOffsetDir + "/%d".format(topicAndPartition.partition))._1.toLong + offsetMap.put(topicAndPartition, offset) + } catch { + case z: ZkNoNodeException => + if(ZkUtils.pathExists(zkClient,topicDirs.consumerOffsetDir)) + offsetMap.put(topicAndPartition,-1) + else + throw z + } + } + else if (offsetAndMetadata.error == ErrorMapping.NoError) + offsetMap.put(topicAndPartition, offsetAndMetadata.offset) + else { + println("Could not fetch offset for %s due to %s.".format(topicAndPartition, ErrorMapping.exceptionFor(offsetAndMetadata.error))) + } + } + channel.disconnect() println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format("Group", "Topic", "Pid", "Offset", "logSize", "Lag", "Owner")) topicList.sorted.foreach { @@ -154,7 +195,7 @@ object ConsumerOffsetChecker extends Logging { } if (options.has("broker-info")) - printBrokerInfo(); + printBrokerInfo() for ((_, consumerOpt) <- consumerMap) consumerOpt match { @@ -162,6 +203,10 @@ object ConsumerOffsetChecker extends Logging { case None => // ignore } } + catch { + case t: Throwable => + println("Exiting due to: %s.".format(t.getMessage)) + } finally { for (consumerOpt <- consumerMap.values) { consumerOpt match { @@ -171,7 +216,9 @@ object ConsumerOffsetChecker extends Logging { } if (zkClient != null) zkClient.close() + + if (channel != null) + channel.disconnect() } } } - diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala new file mode 100644 index 0000000000000..903318d15893a --- /dev/null +++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala @@ -0,0 +1,252 @@ +/** + * 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 scala.collection.JavaConversions._ +import java.util.concurrent.atomic.AtomicLong +import java.nio.channels.ClosedByInterruptException +import org.apache.log4j.Logger +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import kafka.utils.CommandLineUtils +import java.util.{ Random, Properties } +import kafka.consumer.Consumer +import kafka.consumer.ConsumerConnector +import kafka.consumer.KafkaStream +import kafka.consumer.ConsumerTimeoutException +import java.text.SimpleDateFormat + +/** + * Performance test for the full zookeeper consumer + */ +object ConsumerPerformance { + private val logger = Logger.getLogger(getClass()) + + def main(args: Array[String]): Unit = { + + val config = new ConsumerPerfConfig(args) + logger.info("Starting consumer...") + val totalMessagesRead = new AtomicLong(0) + val totalBytesRead = new AtomicLong(0) + + if (!config.hideHeader) { + if (!config.showDetailedStats) + println("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") + else + println("time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") + } + + var startMs, endMs = 0L + if(config.useNewConsumer) { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](config.props) + consumer.subscribe(config.topic) + startMs = System.currentTimeMillis + consume(consumer, config.numMessages, 1000, config, totalMessagesRead, totalBytesRead) + endMs = System.currentTimeMillis + consumer.close() + } else { + import kafka.consumer.ConsumerConfig + val consumerConfig = new ConsumerConfig(config.props) + val consumerConnector: ConsumerConnector = Consumer.create(consumerConfig) + val topicMessageStreams = consumerConnector.createMessageStreams(Map(config.topic -> config.numThreads)) + var threadList = List[ConsumerPerfThread]() + for ((topic, streamList) <- topicMessageStreams) + for (i <- 0 until streamList.length) + threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, totalMessagesRead, totalBytesRead) + + logger.info("Sleeping for 1 second.") + Thread.sleep(1000) + logger.info("starting threads") + startMs = System.currentTimeMillis + for (thread <- threadList) + thread.start + for (thread <- threadList) + thread.join + endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs + consumerConnector.shutdown() + } + val elapsedSecs = (endMs - startMs) / 1000.0 + if (!config.showDetailedStats) { + val totalMBRead = (totalBytesRead.get * 1.0) / (1024 * 1024) + println(("%s, %s, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs), + totalMBRead, totalMBRead / elapsedSecs, totalMessagesRead.get, totalMessagesRead.get / elapsedSecs)) + } + } + + def consume(consumer: KafkaConsumer[Array[Byte], Array[Byte]], count: Long, timeout: Long, config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) { + var bytesRead = 0L + var messagesRead = 0L + val startMs = System.currentTimeMillis + var lastReportTime: Long = startMs + var lastBytesRead = 0L + var lastMessagesRead = 0L + var lastConsumed = System.currentTimeMillis + while(messagesRead < count && lastConsumed >= System.currentTimeMillis - timeout) { + val records = consumer.poll(100) + if(records.count() > 0) + lastConsumed = System.currentTimeMillis + for(record <- records) { + messagesRead += 1 + if(record.key != null) + bytesRead += record.key.size + if(record.value != null) + bytesRead += record.value.size + + if (messagesRead % config.reportingInterval == 0) { + if (config.showDetailedStats) + printProgressMessage(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis, config.dateFormat) + lastReportTime = System.currentTimeMillis + lastMessagesRead = messagesRead + lastBytesRead = bytesRead + } + } + } + totalMessagesRead.set(messagesRead) + totalBytesRead.set(bytesRead) + } + + def printProgressMessage(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, lastMessagesRead: Long, + startMs: Long, endMs: Long, dateFormat: SimpleDateFormat) = { + val elapsedMs: Double = endMs - startMs + val totalMBRead = (bytesRead * 1.0) / (1024 * 1024) + val mbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024) + println(("%s, %d, %.4f, %.4f, %d, %.4f").format(dateFormat.format(endMs), id, totalMBRead, + 1000.0 * (mbRead / elapsedMs), messagesRead, ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0)) + } + + class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) { + val zkConnectOpt = parser.accepts("zookeeper", "The connection string for the zookeeper connection in the form host:port. " + + "Multiple URLS can be given to allow fail-over. This option is only used with the old consumer.") + .withRequiredArg + .describedAs("urls") + .ofType(classOf[String]) + val bootstrapServersOpt = parser.accepts("broker-list", "A broker list to use for connecting if using the new consumer.") + .withRequiredArg() + .describedAs("host") + .ofType(classOf[String]) + val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) + val groupIdOpt = parser.accepts("group", "The group id to consume on.") + .withRequiredArg + .describedAs("gid") + .defaultsTo("perf-consumer-" + new Random().nextInt(100000)) + .ofType(classOf[String]) + val fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.") + .withRequiredArg + .describedAs("size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1024 * 1024) + val resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " + + "offset to consume from, start with the latest message present in the log rather than the earliest message.") + val socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.") + .withRequiredArg + .describedAs("size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(2 * 1024 * 1024) + val numThreadsOpt = parser.accepts("threads", "Number of processing threads.") + .withRequiredArg + .describedAs("count") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(10) + val numFetchersOpt = parser.accepts("num-fetch-threads", "Number of fetcher threads.") + .withRequiredArg + .describedAs("count") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) + val useNewConsumerOpt = parser.accepts("new-consumer", "Use the new consumer implementation.") + + val options = parser.parse(args: _*) + + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt) + + val useNewConsumer = options.has(useNewConsumerOpt) + + val props = new Properties + if(useNewConsumer) { + import org.apache.kafka.clients.consumer.ConsumerConfig + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf(bootstrapServersOpt)) + props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt)) + props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString) + props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString) + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, if (options.has(resetBeginningOffsetOpt)) "latest" else "earliest") + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer]) + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer]) + props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false") + } else { + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt, numMessagesOpt) + props.put("group.id", options.valueOf(groupIdOpt)) + props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) + props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString) + props.put("auto.offset.reset", if (options.has(resetBeginningOffsetOpt)) "largest" else "smallest") + props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) + props.put("consumer.timeout.ms", "1000") + props.put("num.consumer.fetchers", options.valueOf(numFetchersOpt).toString) + } + val numThreads = options.valueOf(numThreadsOpt).intValue + val topic = options.valueOf(topicOpt) + val numMessages = options.valueOf(numMessagesOpt).longValue + val reportingInterval = options.valueOf(reportingIntervalOpt).intValue + val showDetailedStats = options.has(showDetailedStatsOpt) + val dateFormat = new SimpleDateFormat(options.valueOf(dateFormatOpt)) + val hideHeader = options.has(hideHeaderOpt) + } + + class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaStream[Array[Byte], Array[Byte]], + config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) + extends Thread(name) { + + override def run() { + var bytesRead = 0L + var messagesRead = 0L + val startMs = System.currentTimeMillis + var lastReportTime: Long = startMs + var lastBytesRead = 0L + var lastMessagesRead = 0L + + try { + val iter = stream.iterator + while (iter.hasNext && messagesRead < config.numMessages) { + val messageAndMetadata = iter.next + messagesRead += 1 + bytesRead += messageAndMetadata.message.length + + if (messagesRead % config.reportingInterval == 0) { + if (config.showDetailedStats) + printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis, config.dateFormat) + lastReportTime = System.currentTimeMillis + lastMessagesRead = messagesRead + lastBytesRead = bytesRead + } + } + } catch { + case _: InterruptedException => + case _: ClosedByInterruptException => + case _: ConsumerTimeoutException => + case e: Throwable => e.printStackTrace() + } + totalMessagesRead.addAndGet(messagesRead) + totalBytesRead.addAndGet(bytesRead) + if (config.showDetailedStats) + printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis, config.dateFormat) + } + + } + +} diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala old mode 100644 new mode 100755 index 14f44d9305f34..fc11a2ab220a9 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -23,7 +23,9 @@ import kafka.log._ import kafka.utils._ import collection.mutable import joptsimple.OptionParser - +import kafka.serializer.Decoder +import kafka.utils.VerifiableProperties +import org.apache.kafka.common.utils.Utils object DumpLogSegments { @@ -41,19 +43,30 @@ object DumpLogSegments { .ofType(classOf[java.lang.Integer]) .defaultsTo(5 * 1024 * 1024) val deepIterationOpt = parser.accepts("deep-iteration", "if set, uses deep instead of shallow iteration") + val valueDecoderOpt = parser.accepts("value-decoder-class", "if set, used to deserialize the messages. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.") + .withOptionalArg() + .ofType(classOf[java.lang.String]) + .defaultsTo("kafka.serializer.StringDecoder") + val keyDecoderOpt = parser.accepts("key-decoder-class", "if set, used to deserialize the keys. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.") + .withOptionalArg() + .ofType(classOf[java.lang.String]) + .defaultsTo("kafka.serializer.StringDecoder") + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "Parse a log file and dump its contents to the console, useful for debugging a seemingly corrupt log segment.") val options = parser.parse(args : _*) - if(!options.has(filesOpt)) { - System.err.println("Missing required argument \"" + filesOpt + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } + + CommandLineUtils.checkRequiredArgs(parser, options, filesOpt) val print = if(options.has(printOpt)) true else false val verifyOnly = if(options.has(verifyOpt)) true else false val files = options.valueOf(filesOpt).split(",") val maxMessageSize = options.valueOf(maxMessageSizeOpt).intValue() val isDeepIteration = if(options.has(deepIterationOpt)) true else false + + val valueDecoder: Decoder[_] = CoreUtils.createObject[Decoder[_]](options.valueOf(valueDecoderOpt), new VerifiableProperties) + val keyDecoder: Decoder[_] = CoreUtils.createObject[Decoder[_]](options.valueOf(keyDecoderOpt), new VerifiableProperties) val misMatchesForIndexFilesMap = new mutable.HashMap[String, List[(Long, Long)]] val nonConsecutivePairsForLogFilesMap = new mutable.HashMap[String, List[(Long, Long)]] @@ -62,7 +75,7 @@ object DumpLogSegments { val file = new File(arg) if(file.getName.endsWith(Log.LogFileSuffix)) { println("Dumping " + file) - dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration) + dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration, maxMessageSize , valueDecoder, keyDecoder) } else if(file.getName.endsWith(Log.IndexFileSuffix)) { println("Dumping " + file) dumpIndex(file, verifyOnly, misMatchesForIndexFilesMap, maxMessageSize) @@ -92,8 +105,7 @@ object DumpLogSegments { misMatchesForIndexFilesMap: mutable.HashMap[String, List[(Long, Long)]], maxMessageSize: Int) { val startOffset = file.getName().split("\\.")(0).toLong - val logFileName = file.getAbsolutePath.split("\\.")(0) + Log.LogFileSuffix - val logFile = new File(logFileName) + val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix) val messageSet = new FileMessageSet(logFile, false) val index = new OffsetIndex(file = file, baseOffset = startOffset) for(i <- 0 until index.entries) { @@ -117,13 +129,17 @@ object DumpLogSegments { private def dumpLog(file: File, printContents: Boolean, nonConsecutivePairsForLogFilesMap: mutable.HashMap[String, List[(Long, Long)]], - isDeepIteration: Boolean) { + isDeepIteration: Boolean, + maxMessageSize: Int, + valueDecoder: Decoder[_], + keyDecoder: Decoder[_]) { val startOffset = file.getName().split("\\.")(0).toLong println("Starting offset: " + startOffset) val messageSet = new FileMessageSet(file, false) var validBytes = 0L var lastOffset = -1l - for(shallowMessageAndOffset <- messageSet) { // this only does shallow iteration + val shallowIterator = messageSet.iterator(maxMessageSize) + for(shallowMessageAndOffset <- shallowIterator) { // this only does shallow iteration val itr = getIterator(shallowMessageAndOffset, isDeepIteration) for (messageAndOffset <- itr) { val msg = messageAndOffset.message @@ -145,8 +161,8 @@ object DumpLogSegments { print(" keysize: " + msg.keySize) if(printContents) { if(msg.hasKey) - print(" key: " + Utils.readString(messageAndOffset.message.key, "UTF-8")) - val payload = if(messageAndOffset.message.isNull) null else Utils.readString(messageAndOffset.message.payload, "UTF-8") + print(" key: " + keyDecoder.fromBytes(Utils.readBytes(messageAndOffset.message.key))) + val payload = if(messageAndOffset.message.isNull) null else valueDecoder.fromBytes(Utils.readBytes(messageAndOffset.message.payload)) print(" payload: " + payload) } println() @@ -165,7 +181,7 @@ object DumpLogSegments { case NoCompressionCodec => getSingleMessageIterator(messageAndOffset) case _ => - ByteBufferMessageSet.decompress(message).iterator + ByteBufferMessageSet.deepIterator(message) } } else getSingleMessageIterator(messageAndOffset) @@ -184,4 +200,5 @@ object DumpLogSegments { } } } + } diff --git a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala index 005231f38dd9c..7b52fe46e1f53 100644 --- a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala +++ b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala @@ -19,7 +19,7 @@ package kafka.tools import java.io.FileWriter import joptsimple._ -import kafka.utils.{Logging, ZkUtils, ZKStringSerializer,ZKGroupTopicDirs} +import kafka.utils.{Logging, ZkUtils, ZKGroupTopicDirs, CommandLineUtils} import org.I0Itec.zkclient.ZkClient @@ -55,6 +55,9 @@ object ExportZkOffsets extends Logging { .withRequiredArg() .ofType(classOf[String]) parser.accepts("help", "Print this message.") + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "Export consumer offsets to an output file.") val options = parser.parse(args : _*) @@ -63,13 +66,7 @@ object ExportZkOffsets extends Logging { System.exit(0) } - for (opt <- List(zkConnectOpt, outFileOpt)) { - if (!options.has(opt)) { - System.err.println("Missing required argument: %s".format(opt)) - parser.printHelpOn(System.err) - System.exit(1) - } - } + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt, outFileOpt) val zkConnect = options.valueOf(zkConnectOpt) val groups = options.valuesOf(groupOpt) @@ -79,7 +76,7 @@ object ExportZkOffsets extends Logging { val fileWriter : FileWriter = new FileWriter(outfile) try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) var consumerGroups: Seq[String] = null @@ -117,11 +114,10 @@ object ExportZkOffsets extends Logging { } } - private def getBrokeridPartition(zkClient: ZkClient, consumerGroup: String, topic: String): List[String] = { - return ZkUtils.getChildrenParentMayNotExist(zkClient, "/consumers/%s/offsets/%s".format(consumerGroup, topic)).toList - } + private def getBrokeridPartition(zkClient: ZkClient, consumerGroup: String, topic: String): List[String] = + ZkUtils.getChildrenParentMayNotExist(zkClient, "/consumers/%s/offsets/%s".format(consumerGroup, topic)).toList - private def getTopicsList(zkClient: ZkClient, consumerGroup: String): List[String] = { - return ZkUtils.getChildren(zkClient, "/consumers/%s/offsets".format(consumerGroup)).toList - } + private def getTopicsList(zkClient: ZkClient, consumerGroup: String): List[String] = + ZkUtils.getChildren(zkClient, "/consumers/%s/offsets".format(consumerGroup)).toList + } diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index fba652e3716a6..3d9293e4abbe3 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -23,7 +23,7 @@ import joptsimple._ import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} import kafka.common.TopicAndPartition import kafka.client.ClientUtils -import kafka.utils.CommandLineUtils +import kafka.utils.{ToolsUtils, CommandLineUtils} object GetOffsetShell { @@ -57,13 +57,18 @@ object GetOffsetShell { .describedAs("ms") .ofType(classOf[java.lang.Integer]) .defaultsTo(1000) + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "An interactive shell for getting consumer offsets.") val options = parser.parse(args : _*) CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt, topicOpt, timeOpt) val clientId = "GetOffsetShell" - val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt)) + val brokerList = options.valueOf(brokerListOpt) + ToolsUtils.validatePortOrDie(parser, brokerList) + val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) val topic = options.valueOf(topicOpt) var partitionList = options.valueOf(partitionOpt) var time = options.valueOf(timeOpt).longValue diff --git a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala index c8023ee60c07b..b56f587c5b06e 100644 --- a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala +++ b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala @@ -20,7 +20,7 @@ package kafka.tools import java.io.BufferedReader import java.io.FileReader import joptsimple._ -import kafka.utils.{Logging, ZkUtils,ZKStringSerializer} +import kafka.utils.{Logging, ZkUtils, CommandLineUtils} import org.I0Itec.zkclient.ZkClient @@ -52,6 +52,9 @@ object ImportZkOffsets extends Logging { .withRequiredArg() .ofType(classOf[String]) parser.accepts("help", "Print this message.") + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "Import offsets to zookeeper from files.") val options = parser.parse(args : _*) @@ -60,18 +63,12 @@ object ImportZkOffsets extends Logging { System.exit(0) } - for (opt <- List(inFileOpt)) { - if (!options.has(opt)) { - System.err.println("Missing required argument: %s".format(opt)) - parser.printHelpOn(System.err) - System.exit(1) - } - } + CommandLineUtils.checkRequiredArgs(parser, options, inFileOpt) val zkConnect = options.valueOf(zkConnectOpt) val partitionOffsetFile = options.valueOf(inFileOpt) - val zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) val partitionOffsets: Map[String,String] = getPartitionOffsetsFromFile(partitionOffsetFile) updateZkOffsets(zkClient, partitionOffsets) @@ -92,7 +89,7 @@ object ImportZkOffsets extends Logging { s = br.readLine() } - return partOffsetsMap + partOffsetsMap } private def updateZkOffsets(zkClient: ZkClient, partitionOffsets: Map[String,String]): Unit = { diff --git a/core/src/main/scala/kafka/tools/JmxTool.scala b/core/src/main/scala/kafka/tools/JmxTool.scala index 747a675455e9a..c2b2030ccfc16 100644 --- a/core/src/main/scala/kafka/tools/JmxTool.scala +++ b/core/src/main/scala/kafka/tools/JmxTool.scala @@ -26,7 +26,7 @@ import joptsimple.OptionParser import scala.collection.JavaConversions._ import scala.collection.mutable import scala.math._ -import kafka.utils.Logging +import kafka.utils.{CommandLineUtils, Logging} object JmxTool extends Logging { @@ -63,6 +63,9 @@ object JmxTool extends Logging { .describedAs("service-url") .ofType(classOf[String]) .defaultsTo("service:jmx:rmi:///jndi/rmi://:9999/jmxrmi") + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "Dump JMX values to standard output.") val options = parser.parse(args : _*) @@ -100,7 +103,7 @@ object JmxTool extends Logging { // print csv header val keys = List("time") ++ queryAttributes(mbsc, names, attributesWhitelist).keys.toArray.sorted - if(keys.size == numExpectedAttributes.map(_._2).foldLeft(0)(_ + _) + 1) + if(keys.size == numExpectedAttributes.map(_._2).sum + 1) println(keys.map("\"" + _ + "\"").mkString(",")) while(true) { @@ -110,7 +113,7 @@ object JmxTool extends Logging { case Some(dFormat) => dFormat.format(new Date) case None => System.currentTimeMillis().toString } - if(attributes.keySet.size == numExpectedAttributes.map(_._2).foldLeft(0)(_ + _) + 1) + if(attributes.keySet.size == numExpectedAttributes.map(_._2).sum + 1) println(keys.map(attributes(_)).mkString(",")) val sleep = max(0, interval - (System.currentTimeMillis - start)) Thread.sleep(sleep) @@ -134,4 +137,4 @@ object JmxTool extends Logging { attributes } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java old mode 100644 new mode 100755 index 7909d255ff47a..f19df0cf5edd3 --- a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java +++ b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java @@ -21,7 +21,6 @@ import kafka.javaapi.producer.Producer; import kafka.producer.KeyedMessage; import kafka.producer.ProducerConfig; -import kafka.utils.Utils; import java.io.File; import java.io.FileInputStream; @@ -39,6 +38,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.kafka.common.utils.Utils; /** @@ -60,7 +60,7 @@ @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 org.apache.log4j.Logger log = 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"; private static final String KAFKA_07_CONSUMER_STREAM_CLASS_NAME = "kafka.consumer.KafkaStream"; @@ -194,7 +194,7 @@ public static void main(String[] args) throws InterruptedException, IOException kafkaConsumerProperties_07.load(new FileInputStream(consumerConfigFile_07)); /** Disable shallow iteration because the message format is different between 07 and 08, we have to get each individual message **/ if(kafkaConsumerProperties_07.getProperty("shallow.iterator.enable", "").equals("true")) { - logger.warn("Shallow iterator should not be used in the migration tool"); + log.warn("Shallow iterator should not be used in the migration tool"); kafkaConsumerProperties_07.setProperty("shallow.iterator.enable", "false"); } Object consumerConfig_07 = ConsumerConfigConstructor_07.newInstance(kafkaConsumerProperties_07); @@ -230,7 +230,7 @@ public void run() { try { ConsumerConnectorShutdownMethod_07.invoke(consumerConnector_07); } catch(Exception e) { - logger.error("Error while shutting down Kafka consumer", e); + log.error("Error while shutting down Kafka consumer", e); } for(MigrationThread migrationThread : migrationThreads) { migrationThread.shutdown(); @@ -241,7 +241,7 @@ public void run() { for(ProducerThread producerThread : producerThreads) { producerThread.awaitShutdown(); } - logger.info("Kafka migration tool shutdown successfully"); + log.info("Kafka migration tool shutdown successfully"); } }); @@ -266,7 +266,7 @@ public void run() { } catch (Throwable e){ System.out.println("Kafka migration tool failed due to: " + Utils.stackTrace(e)); - logger.error("Kafka migration tool failed: ", e); + log.error("Kafka migration tool failed: ", e); } } @@ -388,7 +388,7 @@ public void run() { KeyedMessage data = producerDataChannel.receiveRequest(); if(!data.equals(shutdownMessage)) { producer.send(data); - if(logger.isDebugEnabled()) logger.debug("Sending message %s".format(new String(data.message()))); + if(logger.isDebugEnabled()) logger.debug(String.format("Sending message %s", new String(data.message()))); } else break; diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala old mode 100644 new mode 100755 index f0f871c22dea3..797b4bbbf6f6f --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -17,32 +17,71 @@ package kafka.tools +import java.util +import java.util.concurrent.{TimeUnit, CountDownLatch} +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.{Collections, Properties} + +import com.yammer.metrics.core.Gauge import joptsimple.OptionParser -import kafka.utils.{Utils, CommandLineUtils, Logging} -import kafka.producer.{KeyedMessage, ProducerConfig, Producer} +import kafka.consumer.{Blacklist, ConsumerConfig, ConsumerThreadId, ConsumerTimeoutException, TopicFilter, Whitelist, ZookeeperConsumerConnector} +import kafka.javaapi.consumer.ConsumerRebalanceListener +import kafka.message.MessageAndMetadata +import kafka.metrics.KafkaMetricsGroup +import kafka.serializer.DefaultDecoder +import kafka.utils.{CommandLineUtils, Logging, CoreUtils} +import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata} +import org.apache.kafka.common.utils.Utils + import scala.collection.JavaConversions._ -import java.util.concurrent.CountDownLatch -import java.nio.ByteBuffer -import kafka.consumer._ -import kafka.serializer._ -import collection.mutable.ListBuffer -import kafka.tools.KafkaMigrationTool.{ProducerThread, ProducerDataChannel} -import kafka.javaapi -object MirrorMaker extends Logging { + +/** + * The mirror maker has the following architecture: + * - There are N mirror maker thread shares one ZookeeperConsumerConnector and each owns a Kafka stream. + * - All the mirror maker threads share one producer. + * - Each mirror maker thread periodically flushes the producer and then commits all offsets. + * + * @note For mirror maker, the following settings are set by default to make sure there is no data loss: + * 1. use new producer with following settings + * acks=all + * retries=max integer + * block.on.buffer.full=true + * max.in.flight.requests.per.connection=1 + * 2. Consumer Settings + * auto.commit.enable=false + * 3. Mirror Maker Setting: + * abort.on.send.failure=true + */ +object MirrorMaker extends Logging with KafkaMetricsGroup { private var connectors: Seq[ZookeeperConsumerConnector] = null - private var consumerThreads: Seq[MirrorMakerThread] = null - private var producerThreads: ListBuffer[ProducerThread] = null + private var producer: MirrorMakerProducer = null + private var mirrorMakerThreads: Seq[MirrorMakerThread] = null + private val isShuttingdown: AtomicBoolean = new AtomicBoolean(false) + // Track the messages not successfully sent by mirror maker. + private var numDroppedMessages: AtomicInteger = new AtomicInteger(0) + private var messageHandler: MirrorMakerMessageHandler = null + private var offsetCommitIntervalMs = 0 + private var abortOnSendFailure: Boolean = true + @volatile private var exitingOnSendFailure: Boolean = false + + // If a message send failed after retries are exhausted. The offset of the messages will also be removed from + // the unacked offset list to avoid offset commit being stuck on that offset. In this case, the offset of that + // message was not really acked, but was skipped. This metric records the number of skipped offsets. + newGauge("MirrorMaker-numDroppedMessages", + new Gauge[Int] { + def value = numDroppedMessages.get() + }) def main(args: Array[String]) { - - info ("Starting mirror maker") + + info("Starting mirror maker") val parser = new OptionParser val consumerConfigOpt = parser.accepts("consumer.config", - "Consumer config to consume from a source cluster. " + - "You may specify multiple of these.") + "Embedded consumer config for consuming from the source cluster.") .withRequiredArg() .describedAs("config file") .ofType(classOf[String]) @@ -53,13 +92,6 @@ object MirrorMaker extends Logging { .describedAs("config file") .ofType(classOf[String]) - val numProducersOpt = parser.accepts("num.producers", - "Number of producer instances") - .withRequiredArg() - .describedAs("Number of producers") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) - val numStreamsOpt = parser.accepts("num.streams", "Number of consumption streams.") .withRequiredArg() @@ -67,12 +99,6 @@ object MirrorMaker extends Logging { .ofType(classOf[java.lang.Integer]) .defaultsTo(1) - val bufferSizeOpt = parser.accepts("queue.size", "Number of messages that are buffered between the consumer and producer") - .withRequiredArg() - .describedAs("Queue size in terms of number of messages") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(10000); - val whitelistOpt = parser.accepts("whitelist", "Whitelist of topics to mirror.") .withRequiredArg() @@ -80,14 +106,56 @@ object MirrorMaker extends Logging { .ofType(classOf[String]) val blacklistOpt = parser.accepts("blacklist", - "Blacklist of topics to mirror.") - .withRequiredArg() - .describedAs("Java regex (String)") - .ofType(classOf[String]) + "Blacklist of topics to mirror.") + .withRequiredArg() + .describedAs("Java regex (String)") + .ofType(classOf[String]) + + val offsetCommitIntervalMsOpt = parser.accepts("offset.commit.interval.ms", + "Offset commit interval in ms") + .withRequiredArg() + .describedAs("offset commit interval in millisecond") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(60000) + + val consumerRebalanceListenerOpt = parser.accepts("consumer.rebalance.listener", + "The consumer rebalance listener to use for mirror maker consumer.") + .withRequiredArg() + .describedAs("A custom rebalance listener of type ConsumerRebalanceListener") + .ofType(classOf[String]) + + val rebalanceListenerArgsOpt = parser.accepts("rebalance.listener.args", + "Arguments used by custom rebalance listener for mirror maker consumer") + .withRequiredArg() + .describedAs("Arguments passed to custom rebalance listener constructor as a string.") + .ofType(classOf[String]) + + val messageHandlerOpt = parser.accepts("message.handler", + "Message handler which will process every record in-between consumer and producer.") + .withRequiredArg() + .describedAs("A custom message handler of type MirrorMakerMessageHandler") + .ofType(classOf[String]) + + val messageHandlerArgsOpt = parser.accepts("message.handler.args", + "Arguments used by custom rebalance listener for mirror maker consumer") + .withRequiredArg() + .describedAs("Arguments passed to message handler constructor.") + .ofType(classOf[String]) + + val abortOnSendFailureOpt = parser.accepts("abort.on.send.failure", + "Configure the mirror maker to exit on a failed send.") + .withRequiredArg() + .describedAs("Stop the entire mirror maker when a send failure occurs") + .ofType(classOf[String]) + .defaultsTo("true") val helpOpt = parser.accepts("help", "Print this message.") - val options = parser.parse(args : _*) + if (args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "Continuously copy data between two Kafka clusters.") + + + val options = parser.parse(args: _*) if (options.has(helpOpt)) { parser.printHelpOn(System.out) @@ -100,125 +168,282 @@ object MirrorMaker extends Logging { System.exit(1) } - val numStreams = options.valueOf(numStreamsOpt) - val bufferSize = options.valueOf(bufferSizeOpt).intValue() + abortOnSendFailure = options.valueOf(abortOnSendFailureOpt).toBoolean + offsetCommitIntervalMs = options.valueOf(offsetCommitIntervalMsOpt).intValue() + val numStreams = options.valueOf(numStreamsOpt).intValue() - val producers = (1 to options.valueOf(numProducersOpt).intValue()).map(_ => { - val props = Utils.loadProps(options.valueOf(producerConfigOpt)) - val config = props.getProperty("partitioner.class") match { - case null => - new ProducerConfig(props) { - override val partitionerClass = "kafka.producer.ByteArrayPartitioner" - } - case pClass : String => - new ProducerConfig(props) + Runtime.getRuntime.addShutdownHook(new Thread("MirrorMakerShutdownHook") { + override def run() { + cleanShutdown() } - new Producer[Array[Byte], Array[Byte]](config) }) + + // create producer + val producerProps = Utils.loadProps(options.valueOf(producerConfigOpt)) + // Defaults to no data loss settings. + maybeSetDefaultProperty(producerProps, ProducerConfig.RETRIES_CONFIG, Int.MaxValue.toString) + maybeSetDefaultProperty(producerProps, ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") + maybeSetDefaultProperty(producerProps, ProducerConfig.ACKS_CONFIG, "all") + maybeSetDefaultProperty(producerProps, ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1") + producer = new MirrorMakerProducer(producerProps) + + // Create consumer connector + val consumerConfigProps = Utils.loadProps(options.valueOf(consumerConfigOpt)) + // Disable consumer auto offsets commit to prevent data loss. + maybeSetDefaultProperty(consumerConfigProps, "auto.commit.enable", "false") + // Set the consumer timeout so we will not block for low volume pipeline. The timeout is necessary to make sure + // Offsets are still committed for those low volume pipelines. + maybeSetDefaultProperty(consumerConfigProps, "consumer.timeout.ms", "10000") + // The default client id is group id, we manually set client id to groupId-index to avoid metric collision + val groupIdString = consumerConfigProps.getProperty("group.id") + connectors = (0 until numStreams) map { i => + consumerConfigProps.setProperty("client.id", groupIdString + "-" + i.toString) + val consumerConfig = new ConsumerConfig(consumerConfigProps) + new ZookeeperConsumerConnector(consumerConfig) + } - connectors = options.valuesOf(consumerConfigOpt).toList - .map(cfg => new ConsumerConfig(Utils.loadProps(cfg.toString))) - .map(new ZookeeperConsumerConnector(_)) + // Set consumer rebalance listener. + // Custom rebalance listener will be invoked after internal listener finishes its work. + val customRebalanceListenerClass = options.valueOf(consumerRebalanceListenerOpt) + val rebalanceListenerArgs = options.valueOf(rebalanceListenerArgsOpt) + val customRebalanceListener = { + if (customRebalanceListenerClass != null) { + if (rebalanceListenerArgs != null) + Some(CoreUtils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass, rebalanceListenerArgs)) + else + Some(CoreUtils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass)) + } else { + None + } + } + connectors.foreach { + connector => + val consumerRebalanceListener = new InternalRebalanceListener(connector, customRebalanceListener) + connector.setConsumerRebalanceListener(consumerRebalanceListener) + } + // create filters val filterSpec = if (options.has(whitelistOpt)) new Whitelist(options.valueOf(whitelistOpt)) else new Blacklist(options.valueOf(blacklistOpt)) - var streams: Seq[KafkaStream[Array[Byte], Array[Byte]]] = Nil - try { - streams = connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue(), new DefaultDecoder(), new DefaultDecoder())).flatten - } catch { - case t: Throwable => - fatal("Unable to create stream - shutting down mirror maker.") - connectors.foreach(_.shutdown) - } - - val producerDataChannel = new ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]](bufferSize); - - consumerThreads = streams.zipWithIndex.map(streamAndIndex => new MirrorMakerThread(streamAndIndex._1, producerDataChannel, producers, streamAndIndex._2)) - - producerThreads = new ListBuffer[ProducerThread]() - - Runtime.getRuntime.addShutdownHook(new Thread() { - override def run() { - cleanShutdown() + // Create mirror maker threads + mirrorMakerThreads = (0 until numStreams) map ( i => + new MirrorMakerThread(connectors(i), filterSpec, i) + ) + + // Create and initialize message handler + val customMessageHandlerClass = options.valueOf(messageHandlerOpt) + val messageHandlerArgs = options.valueOf(messageHandlerArgsOpt) + messageHandler = { + if (customMessageHandlerClass != null) { + if (messageHandlerArgs != null) + CoreUtils.createObject[MirrorMakerMessageHandler](customMessageHandlerClass, messageHandlerArgs) + else + CoreUtils.createObject[MirrorMakerMessageHandler](customMessageHandlerClass) + } else { + defaultMirrorMakerMessageHandler } - }) - - // create producer threads - var i: Int = 1 - for(producer <- producers) { - val producerThread: KafkaMigrationTool.ProducerThread = new KafkaMigrationTool.ProducerThread(producerDataChannel, - new javaapi.producer.Producer[Array[Byte], Array[Byte]](producer), i) - producerThreads += producerThread - i += 1 } - consumerThreads.foreach(_.start) - producerThreads.foreach(_.start) + mirrorMakerThreads.foreach(_.start()) + mirrorMakerThreads.foreach(_.awaitShutdown()) + } - // in case the consumer threads hit a timeout/other exception - consumerThreads.foreach(_.awaitShutdown) - cleanShutdown() + def commitOffsets(connector: ZookeeperConsumerConnector) { + if (!exitingOnSendFailure) { + trace("Committing offsets.") + connector.commitOffsets + } else { + info("Exiting on send failure, skip committing offsets.") + } } def cleanShutdown() { - if (connectors != null) connectors.foreach(_.shutdown) - if (consumerThreads != null) consumerThreads.foreach(_.awaitShutdown) - if (producerThreads != null) { - producerThreads.foreach(_.shutdown) - producerThreads.foreach(_.awaitShutdown) + if (isShuttingdown.compareAndSet(false, true)) { + info("Start clean shutdown.") + // Shutdown consumer threads. + info("Shutting down consumer threads.") + if (mirrorMakerThreads != null) { + mirrorMakerThreads.foreach(_.shutdown()) + mirrorMakerThreads.foreach(_.awaitShutdown()) + } + info("Closing producer.") + producer.close() + info("Kafka mirror maker shutdown successfully") } - info("Kafka mirror maker shutdown successfully") } - class MirrorMakerThread(stream: KafkaStream[Array[Byte], Array[Byte]], - producerDataChannel: ProducerDataChannel[KeyedMessage[Array[Byte], Array[Byte]]], - producers: Seq[Producer[Array[Byte], Array[Byte]]], - threadId: Int) - extends Thread with Logging { + private def maybeSetDefaultProperty(properties: Properties, propertyName: String, defaultValue: String) { + val propertyValue = properties.getProperty(propertyName) + properties.setProperty(propertyName, Option(propertyValue).getOrElse(defaultValue)) + if (properties.getProperty(propertyName) != defaultValue) + info("Property %s is overridden to %s - data loss or message reordering is possible.".format(propertyName, propertyValue)) + } - private val shutdownLatch = new CountDownLatch(1) - private val threadName = "mirrormaker-" + threadId + class MirrorMakerThread(connector: ZookeeperConsumerConnector, + filterSpec: TopicFilter, + val threadId: Int) extends Thread with Logging with KafkaMetricsGroup { + private val threadName = "mirrormaker-thread-" + threadId + private val shutdownLatch: CountDownLatch = new CountDownLatch(1) + private var lastOffsetCommitMs = System.currentTimeMillis() + @volatile private var shuttingDown: Boolean = false this.logIdent = "[%s] ".format(threadName) - this.setName(threadName) + setName(threadName) override def run() { info("Starting mirror maker thread " + threadName) try { - for (msgAndMetadata <- stream) { - // If the key of the message is empty, put it into the universal channel - // Otherwise use a pre-assigned producer to send the message - if (msgAndMetadata.key == null) { - trace("Send the non-keyed message the producer channel.") - val pd = new KeyedMessage[Array[Byte], Array[Byte]](msgAndMetadata.topic, msgAndMetadata.message) - producerDataChannel.sendRequest(pd) - } else { - val producerId = Utils.abs(java.util.Arrays.hashCode(msgAndMetadata.key)) % producers.size() - trace("Send message with key %s to producer %d.".format(java.util.Arrays.toString(msgAndMetadata.key), producerId)) - val producer = producers(producerId) - val pd = new KeyedMessage[Array[Byte], Array[Byte]](msgAndMetadata.topic, msgAndMetadata.key, msgAndMetadata.message) - producer.send(pd) + // Creating one stream per each connector instance + val streams = connector.createMessageStreamsByFilter(filterSpec, 1, new DefaultDecoder(), new DefaultDecoder()) + require(streams.size == 1) + val stream = streams(0) + val iter = stream.iterator() + + while (!exitingOnSendFailure && !shuttingDown) { + try { + while (!exitingOnSendFailure && !shuttingDown && iter.hasNext()) { + val data = iter.next() + trace("Sending message with value size %d".format(data.message().size)) + val records = messageHandler.handle(data) + records.foreach(producer.send) + maybeFlushAndCommitOffsets() + } + } catch { + case e: ConsumerTimeoutException => + trace("Caught ConsumerTimeoutException, continue iteration.") } + maybeFlushAndCommitOffsets() } } catch { - case e: Throwable => - fatal("Stream unexpectedly exited.", e) + case t: Throwable => + fatal("Mirror maker thread failure due to ", t) } finally { + info("Flushing producer.") + producer.flush() + info("Committing consumer offsets.") + commitOffsets(connector) + info("Shutting down consumer connectors.") + connector.shutdown() shutdownLatch.countDown() - info("Stopped thread.") + info("Mirror maker thread stopped") + // if it exits accidentally, stop the entire mirror maker + if (!isShuttingdown.get()) { + fatal("Mirror maker thread exited abnormally, stopping the whole mirror maker.") + System.exit(-1) + } + } + } + + def maybeFlushAndCommitOffsets() { + if (System.currentTimeMillis() - lastOffsetCommitMs > offsetCommitIntervalMs) { + producer.flush() + commitOffsets(connector) + lastOffsetCommitMs = System.currentTimeMillis() + } + } + + def shutdown() { + try { + info(threadName + " shutting down") + shuttingDown = true + } + catch { + case ie: InterruptedException => + warn("Interrupt during shutdown of the mirror maker thread") } } def awaitShutdown() { try { shutdownLatch.await() + info("Mirror maker thread shutdown complete") } catch { - case e: InterruptedException => fatal("Shutdown of thread %s interrupted. This might leak data!".format(threadName)) + case ie: InterruptedException => + warn("Shutdown of the mirror maker thread interrupted") + } + } + } + + private class MirrorMakerProducer(val producerProps: Properties) { + + val sync = producerProps.getProperty("producer.type", "async").equals("sync") + + val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps) + + def send(record: ProducerRecord[Array[Byte], Array[Byte]]) { + if (sync) { + this.producer.send(record).get() + } else { + this.producer.send(record, + new MirrorMakerProducerCallback(record.topic(), record.key(), record.value())) + } + } + + def flush() { + this.producer.flush() + } + + def close() { + this.producer.close() + } + + def close(timeout: Long) { + this.producer.close(timeout, TimeUnit.MILLISECONDS) + } + } + + private class MirrorMakerProducerCallback (topic: String, key: Array[Byte], value: Array[Byte]) + extends ErrorLoggingCallback(topic, key, value, false) { + + override def onCompletion(metadata: RecordMetadata, exception: Exception) { + if (exception != null) { + // Use default call back to log error. This means the max retries of producer has reached and message + // still could not be sent. + super.onCompletion(metadata, exception) + // If abort.on.send.failure is set, stop the mirror maker. Otherwise log skipped message and move on. + if (abortOnSendFailure) { + info("Closing producer due to send failure.") + exitingOnSendFailure = true + producer.close(0) + } + numDroppedMessages.incrementAndGet() } } } -} + private class InternalRebalanceListener(connector: ZookeeperConsumerConnector, + customRebalanceListener: Option[ConsumerRebalanceListener]) + extends ConsumerRebalanceListener { + + override def beforeReleasingPartitions(partitionOwnership: java.util.Map[String, java.util.Set[java.lang.Integer]]) { + producer.flush() + commitOffsets(connector) + // invoke custom consumer rebalance listener + if (customRebalanceListener.isDefined) + customRebalanceListener.get.beforeReleasingPartitions(partitionOwnership) + } + + override def beforeStartingFetchers(consumerId: String, + partitionAssignment: java.util.Map[String, java.util.Map[java.lang.Integer, ConsumerThreadId]]) { + if (customRebalanceListener.isDefined) + customRebalanceListener.get.beforeStartingFetchers(consumerId, partitionAssignment) + } + } + + /** + * If message.handler.args is specified. A constructor that takes in a String as argument must exist. + */ + trait MirrorMakerMessageHandler { + def handle(record: MessageAndMetadata[Array[Byte], Array[Byte]]): util.List[ProducerRecord[Array[Byte], Array[Byte]]] + } + + private object defaultMirrorMakerMessageHandler extends MirrorMakerMessageHandler { + override def handle(record: MessageAndMetadata[Array[Byte], Array[Byte]]): util.List[ProducerRecord[Array[Byte], Array[Byte]]] = { + Collections.singletonList(new ProducerRecord[Array[Byte], Array[Byte]](record.topic, record.key(), record.message())) + } + } + +} diff --git a/perf/src/main/scala/kafka/perf/PerfConfig.scala b/core/src/main/scala/kafka/tools/PerfConfig.scala similarity index 92% rename from perf/src/main/scala/kafka/perf/PerfConfig.scala rename to core/src/main/scala/kafka/tools/PerfConfig.scala index a8fc6b9ec81c4..298bb29fe797f 100644 --- a/perf/src/main/scala/kafka/perf/PerfConfig.scala +++ b/core/src/main/scala/kafka/tools/PerfConfig.scala @@ -15,18 +15,17 @@ * limitations under the License. */ -package kafka.perf +package kafka.tools import joptsimple.OptionParser class PerfConfig(args: Array[String]) { val parser = new OptionParser - val numMessagesOpt = parser.accepts("messages", "The number of messages to send or consume") + val numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume") .withRequiredArg .describedAs("count") .ofType(classOf[java.lang.Long]) - .defaultsTo(Long.MaxValue) val reportingIntervalOpt = parser.accepts("reporting-interval", "Interval at which to print progress info.") .withRequiredArg .describedAs("size") @@ -53,7 +52,7 @@ class PerfConfig(args: Array[String]) { .defaultsTo(200) val compressionCodecOpt = parser.accepts("compression-codec", "If set, messages are sent compressed") .withRequiredArg - .describedAs("supported codec: NoCompressionCodec as 0, GZIPCompressionCodec as 1, SnappyCompressionCodec as 2") + .describedAs("supported codec: NoCompressionCodec as 0, GZIPCompressionCodec as 1, SnappyCompressionCodec as 2, LZ4CompressionCodec as 3") .ofType(classOf[java.lang.Integer]) .defaultsTo(0) val helpOpt = parser.accepts("help", "Print usage.") diff --git a/perf/src/main/scala/kafka/perf/ProducerPerformance.scala b/core/src/main/scala/kafka/tools/ProducerPerformance.scala similarity index 53% rename from perf/src/main/scala/kafka/perf/ProducerPerformance.scala rename to core/src/main/scala/kafka/tools/ProducerPerformance.scala index ad2ac26411de8..0ebfa59090a82 100644 --- a/perf/src/main/scala/kafka/perf/ProducerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ProducerPerformance.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,21 @@ * limitations under the License. */ -package kafka.perf +package kafka.tools + +import kafka.metrics.KafkaMetricsReporter +import kafka.producer.{OldProducer, NewShinyProducer} +import kafka.utils.{ToolsUtils, VerifiableProperties, Logging, CommandLineUtils} +import kafka.message.CompressionCodec +import kafka.serializer._ import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong -import kafka.producer._ -import org.apache.log4j.Logger -import kafka.message.{CompressionCodec, Message} -import java.text.SimpleDateFormat -import kafka.serializer._ import java.util._ -import collection.immutable.List -import kafka.utils.{VerifiableProperties, Logging} -import kafka.metrics.KafkaMetricsReporter +import java.text.SimpleDateFormat +import java.math.BigInteger +import org.apache.log4j.Logger /** * Load test for the producer @@ -39,7 +40,7 @@ object ProducerPerformance extends Logging { val logger = Logger.getLogger(getClass) val config = new ProducerPerfConfig(args) - if(!config.isFixSize) + if (!config.isFixedSize) logger.info("WARN: Throughput will be slower due to changing message size per request") val totalBytesSent = new AtomicLong(0) @@ -49,84 +50,80 @@ object ProducerPerformance extends Logging { val startMs = System.currentTimeMillis val rand = new java.util.Random - if(!config.hideHeader) - println("start.time, end.time, compression, message.size, batch.size, total.data.sent.in.MB, MB.sec, " + - "total.data.sent.in.nMsg, nMsg.sec") + if (!config.hideHeader) + println("start.time, end.time, compression, message.size, batch.size, total.data.sent.in.MB, MB.sec, " + + "total.data.sent.in.nMsg, nMsg.sec") - for(i <- 0 until config.numThreads) { + for (i <- 0 until config.numThreads) { executor.execute(new ProducerThread(i, config, totalBytesSent, totalMessagesSent, allDone, rand)) } allDone.await() val endMs = System.currentTimeMillis val elapsedSecs = (endMs - startMs) / 1000.0 - val totalMBSent = (totalBytesSent.get * 1.0)/ (1024 * 1024) + val totalMBSent = (totalBytesSent.get * 1.0) / (1024 * 1024) println(("%s, %s, %d, %d, %d, %.2f, %.4f, %d, %.4f").format( config.dateFormat.format(startMs), config.dateFormat.format(endMs), config.compressionCodec.codec, config.messageSize, config.batchSize, totalMBSent, - totalMBSent/elapsedSecs, totalMessagesSent.get, totalMessagesSent.get/elapsedSecs)) + totalMBSent / elapsedSecs, totalMessagesSent.get, totalMessagesSent.get / elapsedSecs)) System.exit(0) } class ProducerPerfConfig(args: Array[String]) extends PerfConfig(args) { - val brokerListOpt = parser.accepts("broker-list", "REQUIRED: broker info (the list of broker host and port for bootstrap.") - .withRequiredArg - .describedAs("hostname:port,..,hostname:port") - .ofType(classOf[String]) + val brokerListOpt = parser.accepts("broker-list", "REQUIRED: broker info the list of broker host and port for bootstrap.") + .withRequiredArg + .describedAs("hostname:port,..,hostname:port") + .ofType(classOf[String]) val topicsOpt = parser.accepts("topics", "REQUIRED: The comma separated list of topics to produce to") .withRequiredArg .describedAs("topic1,topic2..") .ofType(classOf[String]) - val producerRequestTimeoutMsOpt = parser.accepts("request-timeout-ms", "The produce request timeout in ms") - .withRequiredArg() - .ofType(classOf[java.lang.Integer]) - .defaultsTo(3000) + val producerRequestTimeoutMsOpt = parser.accepts("request-timeout-ms", "The producer request timeout in ms") + .withRequiredArg() + .ofType(classOf[java.lang.Integer]) + .defaultsTo(3000) val producerNumRetriesOpt = parser.accepts("producer-num-retries", "The producer retries number") - .withRequiredArg() - .ofType(classOf[java.lang.Integer]) - .defaultsTo(3) + .withRequiredArg() + .ofType(classOf[java.lang.Integer]) + .defaultsTo(3) val producerRetryBackOffMsOpt = parser.accepts("producer-retry-backoff-ms", "The producer retry backoff time in milliseconds") - .withRequiredArg() - .ofType(classOf[java.lang.Integer]) - .defaultsTo(100) + .withRequiredArg() + .ofType(classOf[java.lang.Integer]) + .defaultsTo(100) val producerRequestRequiredAcksOpt = parser.accepts("request-num-acks", "Number of acks required for producer request " + - "to complete") - .withRequiredArg() - .ofType(classOf[java.lang.Integer]) - .defaultsTo(-1) + "to complete") + .withRequiredArg() + .ofType(classOf[java.lang.Integer]) + .defaultsTo(-1) val varyMessageSizeOpt = parser.accepts("vary-message-size", "If set, message size will vary up to the given maximum.") val syncOpt = parser.accepts("sync", "If set, messages are sent synchronously.") val numThreadsOpt = parser.accepts("threads", "Number of sending threads.") - .withRequiredArg - .describedAs("number of threads") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) + .withRequiredArg + .describedAs("number of threads") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) val initialMessageIdOpt = parser.accepts("initial-message-id", "The is used for generating test data, If set, messages will be tagged with an " + - "ID and sent by producer starting from this ID sequentially. Message content will be String type and " + - "in the form of 'Message:000...1:xxx...'") - .withRequiredArg() - .describedAs("initial message id") - .ofType(classOf[java.lang.Integer]) + "ID and sent by producer starting from this ID sequentially. Message content will be String type and " + + "in the form of 'Message:000...1:xxx...'") + .withRequiredArg() + .describedAs("initial message id") + .ofType(classOf[java.lang.Integer]) val messageSendGapMsOpt = parser.accepts("message-send-gap-ms", "If set, the send thread will wait for specified time between two sends") - .withRequiredArg() - .describedAs("message send time gap") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) + .withRequiredArg() + .describedAs("message send time gap") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(0) val csvMetricsReporterEnabledOpt = parser.accepts("csv-reporter-enabled", "If set, the CSV metrics reporter will be enabled") val metricsDirectoryOpt = parser.accepts("metrics-dir", "If csv-reporter-enable is set, and this parameter is" + - "set, the csv metrics will be outputed here") + "set, the csv metrics will be outputed here") .withRequiredArg .describedAs("metrics dictory") .ofType(classOf[java.lang.String]) + val useNewProducerOpt = parser.accepts("new-producer", "Use the new producer implementation.") + + val options = parser.parse(args: _*) + CommandLineUtils.checkRequiredArgs(parser, options, topicsOpt, brokerListOpt, numMessagesOpt) - val options = parser.parse(args : _*) - for(arg <- List(topicsOpt, brokerListOpt, numMessagesOpt)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } val topicsStr = options.valueOf(topicsOpt) val topics = topicsStr.split(",") val numMessages = options.valueOf(numMessagesOpt).longValue @@ -134,20 +131,22 @@ object ProducerPerformance extends Logging { val dateFormat = new SimpleDateFormat(options.valueOf(dateFormatOpt)) val hideHeader = options.has(hideHeaderOpt) val brokerList = options.valueOf(brokerListOpt) + ToolsUtils.validatePortOrDie(parser,brokerList) val messageSize = options.valueOf(messageSizeOpt).intValue - var isFixSize = !options.has(varyMessageSizeOpt) + var isFixedSize = !options.has(varyMessageSizeOpt) var isSync = options.has(syncOpt) var batchSize = options.valueOf(batchSizeOpt).intValue var numThreads = options.valueOf(numThreadsOpt).intValue val compressionCodec = CompressionCodec.getCompressionCodec(options.valueOf(compressionCodecOpt).intValue) val seqIdMode = options.has(initialMessageIdOpt) var initialMessageId: Int = 0 - if(seqIdMode) + if (seqIdMode) initialMessageId = options.valueOf(initialMessageIdOpt).intValue() val producerRequestTimeoutMs = options.valueOf(producerRequestTimeoutMsOpt).intValue() val producerRequestRequiredAcks = options.valueOf(producerRequestRequiredAcksOpt).intValue() val producerNumRetries = options.valueOf(producerNumRetriesOpt).intValue() val producerRetryBackoffMs = options.valueOf(producerRetryBackOffMsOpt).intValue() + val useNewProducer = options.has(useNewProducerOpt) val csvMetricsReporterEnabled = options.has(csvMetricsReporterEnabledOpt) @@ -168,43 +167,55 @@ object ProducerPerformance extends Logging { } class ProducerThread(val threadId: Int, - val config: ProducerPerfConfig, - val totalBytesSent: AtomicLong, - val totalMessagesSent: AtomicLong, - val allDone: CountDownLatch, - val rand: Random) extends Runnable { - val props = new Properties() - props.put("metadata.broker.list", config.brokerList) - props.put("compression.codec", config.compressionCodec.codec.toString) - props.put("reconnect.interval", Integer.MAX_VALUE.toString) - props.put("send.buffer.bytes", (64*1024).toString) - if(!config.isSync) { - props.put("producer.type","async") - props.put("batch.num.messages", config.batchSize.toString) - props.put("queue.enqueue.timeout.ms", "-1") - } - props.put("client.id", "ProducerPerformance") - props.put("request.required.acks", config.producerRequestRequiredAcks.toString) - props.put("request.timeout.ms", config.producerRequestTimeoutMs.toString) - props.put("message.send.max.retries", config.producerNumRetries.toString) - props.put("retry.backoff.ms", config.producerRetryBackoffMs.toString) - props.put("serializer.class", classOf[DefaultEncoder].getName.toString) - props.put("key.serializer.class", classOf[NullEncoder[Long]].getName.toString) - - - val producerConfig = new ProducerConfig(props) - val producer = new Producer[Long, Array[Byte]](producerConfig) - val seqIdNumDigit = 10 // no. of digits for max int value + val config: ProducerPerfConfig, + val totalBytesSent: AtomicLong, + val totalMessagesSent: AtomicLong, + val allDone: CountDownLatch, + val rand: Random) extends Runnable { + val seqIdNumDigit = 10 // no. of digits for max int value val messagesPerThread = config.numMessages / config.numThreads debug("Messages per thread = " + messagesPerThread) + val props = new Properties() + val producer = + if (config.useNewProducer) { + import org.apache.kafka.clients.producer.ProducerConfig + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.brokerList) + props.put(ProducerConfig.SEND_BUFFER_CONFIG, (64 * 1024).toString) + props.put(ProducerConfig.CLIENT_ID_CONFIG, "producer-performance") + props.put(ProducerConfig.ACKS_CONFIG, config.producerRequestRequiredAcks.toString) + props.put(ProducerConfig.TIMEOUT_CONFIG, config.producerRequestTimeoutMs.toString) + props.put(ProducerConfig.RETRIES_CONFIG, config.producerNumRetries.toString) + props.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, config.producerRetryBackoffMs.toString) + props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, config.compressionCodec.name) + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + new NewShinyProducer(props) + } else { + props.put("metadata.broker.list", config.brokerList) + props.put("compression.codec", config.compressionCodec.codec.toString) + props.put("send.buffer.bytes", (64 * 1024).toString) + if (!config.isSync) { + props.put("producer.type", "async") + props.put("batch.num.messages", config.batchSize.toString) + props.put("queue.enqueue.timeout.ms", "-1") + } + props.put("client.id", "producer-performance") + props.put("request.required.acks", config.producerRequestRequiredAcks.toString) + props.put("request.timeout.ms", config.producerRequestTimeoutMs.toString) + props.put("message.send.max.retries", config.producerNumRetries.toString) + props.put("retry.backoff.ms", config.producerRetryBackoffMs.toString) + props.put("serializer.class", classOf[DefaultEncoder].getName) + props.put("key.serializer.class", classOf[NullEncoder[Long]].getName) + new OldProducer(props) + } // generate the sequential message ID - private val SEP = ":" // message field separator + private val SEP = ":" // message field separator private val messageIdLabel = "MessageID" - private val threadIdLabel = "ThreadID" - private val topicLabel = "Topic" - private var leftPaddedSeqId : String = "" + private val threadIdLabel = "ThreadID" + private val topicLabel = "Topic" + private var leftPaddedSeqId: String = "" private def generateMessageWithSeqId(topic: String, msgId: Long, msgSize: Int): Array[Byte] = { // Each thread gets a unique range of sequential no. for its ids. @@ -213,54 +224,57 @@ object ProducerPerformance extends Logging { // thread 1 IDs : 100 ~ 199 // thread 2 IDs : 200 ~ 299 // . . . - leftPaddedSeqId = String.format("%0"+seqIdNumDigit+"d", long2Long(msgId)) + leftPaddedSeqId = String.format("%0" + seqIdNumDigit + "d", long2Long(msgId)) - val msgHeader = topicLabel + SEP + - topic + SEP + - threadIdLabel + SEP + - threadId + SEP + - messageIdLabel + SEP + - leftPaddedSeqId + SEP + val msgHeader = topicLabel + SEP + + topic + SEP + + threadIdLabel + SEP + + threadId + SEP + + messageIdLabel + SEP + + leftPaddedSeqId + SEP - val seqMsgString = String.format("%1$-"+msgSize+"s", msgHeader).replace(' ', 'x') + val seqMsgString = String.format("%1$-" + msgSize + "s", msgHeader).replace(' ', 'x') debug(seqMsgString) - return seqMsgString.getBytes() + seqMsgString.getBytes() } - private def generateProducerData(topic: String, messageId: Long): (KeyedMessage[Long, Array[Byte]], Int) = { - val msgSize = if(config.isFixSize) config.messageSize else 1 + rand.nextInt(config.messageSize) - val message = - if(config.seqIdMode) { - val seqId = config.initialMessageId + (messagesPerThread * threadId) + messageId - generateMessageWithSeqId(topic, seqId, msgSize) - } else { - new Array[Byte](msgSize) - } - (new KeyedMessage[Long, Array[Byte]](topic, messageId, message), message.length) + private def generateProducerData(topic: String, messageId: Long): Array[Byte] = { + val msgSize = if (config.isFixedSize) config.messageSize else 1 + rand.nextInt(config.messageSize) + if (config.seqIdMode) { + val seqId = config.initialMessageId + (messagesPerThread * threadId) + messageId + generateMessageWithSeqId(topic, seqId, msgSize) + } else { + new Array[Byte](msgSize) + } } override def run { var bytesSent = 0L var nSends = 0 - var j: Long = 0L - while(j < messagesPerThread) { + var i: Long = 0L + var message: Array[Byte] = null + + while (i < messagesPerThread) { try { config.topics.foreach( topic => { - val (producerData, bytesSent_) = generateProducerData(topic, j) - bytesSent += bytesSent_ - producer.send(producerData) + message = generateProducerData(topic, i) + producer.send(topic, BigInteger.valueOf(i).toByteArray, message) + bytesSent += message.size nSends += 1 - if(config.messageSendGapMs > 0) + if (config.messageSendGapMs > 0) Thread.sleep(config.messageSendGapMs) - } - ) + }) } catch { - case e: Exception => error("Error sending messages", e) + case e: Throwable => error("Error when sending message " + new String(message), e) } - j += 1 + i += 1 + } + try { + producer.close() + } catch { + case e: Throwable => error("Error when closing producer", e) } - producer.close() totalBytesSent.addAndGet(bytesSent) totalMessagesSent.addAndGet(nSends) allDone.countDown() diff --git a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala index 814d61ae477ce..2b8537b89fafa 100644 --- a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala +++ b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala @@ -20,11 +20,10 @@ package kafka.tools import joptsimple.OptionParser import java.util.concurrent.{Executors, CountDownLatch} import java.util.Properties -import kafka.producer.{KeyedMessage, ProducerConfig, Producer} import kafka.consumer._ -import kafka.utils.{Logging, ZkUtils} +import kafka.utils.{ToolsUtils, CommandLineUtils, Logging, ZkUtils} import kafka.api.OffsetRequest -import kafka.message.CompressionCodec +import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer, ProducerConfig} object ReplayLogProducer extends Logging { @@ -88,17 +87,6 @@ object ReplayLogProducer extends Logging { .describedAs("count") .ofType(classOf[java.lang.Integer]) .defaultsTo(-1) - val asyncOpt = parser.accepts("async", "If set, messages are sent asynchronously.") - val delayMSBtwBatchOpt = parser.accepts("delay-btw-batch-ms", "Delay in ms between 2 batch sends.") - .withRequiredArg - .describedAs("ms") - .ofType(classOf[java.lang.Long]) - .defaultsTo(0) - val batchSizeOpt = parser.accepts("batch-size", "Number of messages to send in a single batch.") - .withRequiredArg - .describedAs("batch size") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(200) val numThreadsOpt = parser.accepts("threads", "Number of sending threads.") .withRequiredArg .describedAs("threads") @@ -109,48 +97,36 @@ object ReplayLogProducer extends Logging { .describedAs("size") .ofType(classOf[java.lang.Integer]) .defaultsTo(5000) - val compressionCodecOption = parser.accepts("compression-codec", "If set, messages are sent compressed") + val propertyOpt = parser.accepts("property", "A mechanism to pass properties in the form key=value to the producer. " + + "This allows the user to override producer properties that are not exposed by the existing command line arguments") .withRequiredArg - .describedAs("compression codec ") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) + .describedAs("producer properties") + .ofType(classOf[String]) + val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") val options = parser.parse(args : _*) - for(arg <- List(brokerListOpt, inputTopicOpt)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } + + CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt, inputTopicOpt) + val zkConnect = options.valueOf(zkConnectOpt) val brokerList = options.valueOf(brokerListOpt) + ToolsUtils.validatePortOrDie(parser,brokerList) val numMessages = options.valueOf(numMessagesOpt).intValue - val isAsync = options.has(asyncOpt) - val delayedMSBtwSend = options.valueOf(delayMSBtwBatchOpt).longValue - var batchSize = options.valueOf(batchSizeOpt).intValue val numThreads = options.valueOf(numThreadsOpt).intValue val inputTopic = options.valueOf(inputTopicOpt) val outputTopic = options.valueOf(outputTopicOpt) val reportingInterval = options.valueOf(reportingIntervalOpt).intValue - val compressionCodec = CompressionCodec.getCompressionCodec(options.valueOf(compressionCodecOption).intValue) + val isSync = options.has(syncOpt) + import scala.collection.JavaConversions._ + val producerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(propertyOpt)) + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") } class ZKConsumerThread(config: Config, stream: KafkaStream[Array[Byte], Array[Byte]]) extends Thread with Logging { val shutdownLatch = new CountDownLatch(1) - val props = new Properties() - props.put("metadata.broker.list", config.brokerList) - props.put("reconnect.interval", Integer.MAX_VALUE.toString) - props.put("send.buffer.bytes", (64*1024).toString) - props.put("compression.codec", config.compressionCodec.codec.toString) - props.put("batch.num.messages", config.batchSize.toString) - props.put("queue.enqueue.timeout.ms", "-1") - - if(config.isAsync) - props.put("producer.type", "async") - - val producerConfig = new ProducerConfig(props) - val producer = new Producer[Array[Byte], Array[Byte]](producerConfig) + val producer = new KafkaProducer[Array[Byte],Array[Byte]](config.producerProps) override def run() { info("Starting consumer thread..") @@ -163,9 +139,11 @@ object ReplayLogProducer extends Logging { stream for (messageAndMetadata <- iter) { try { - producer.send(new KeyedMessage[Array[Byte], Array[Byte]](config.outputTopic, messageAndMetadata.message)) - if (config.delayedMSBtwSend > 0 && (messageCount + 1) % config.batchSize == 0) - Thread.sleep(config.delayedMSBtwSend) + val response = producer.send(new ProducerRecord[Array[Byte],Array[Byte]](config.outputTopic, + messageAndMetadata.key(), messageAndMetadata.message())) + if(config.isSync) { + response.get() + } messageCount += 1 }catch { case ie: Exception => error("Skipping this message", ie) diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index 5e8c56d963174..1366172243b41 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -18,7 +18,7 @@ package kafka.tools import joptsimple.OptionParser -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import kafka.message.{MessageSet, MessageAndOffset, ByteBufferMessageSet} import java.util.concurrent.CountDownLatch import java.util.concurrent.atomic.AtomicReference @@ -92,7 +92,9 @@ object ReplicaVerificationTool extends Logging { .describedAs("ms") .ofType(classOf[java.lang.Long]) .defaultsTo(30 * 1000L) - + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "Validate that all replicas for a set of topics have the same data.") val options = parser.parse(args : _*) CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt) @@ -114,11 +116,16 @@ object ReplicaVerificationTool extends Logging { val reportInterval = options.valueOf(reportIntervalOpt).longValue // getting topic metadata info("Getting topic metatdata...") - val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt)) + val brokerList = options.valueOf(brokerListOpt) + ToolsUtils.validatePortOrDie(parser,brokerList) + val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, clientId, maxWaitMs) - val brokerMap = topicsMetadataResponse.extractBrokers(topicsMetadataResponse.topicsMetadata) + val brokerMap = topicsMetadataResponse.brokers.map(b => (b.id, b)).toMap val filteredTopicMetadata = topicsMetadataResponse.topicsMetadata.filter( - topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic)) true else false + topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic, excludeInternalTopics = false)) + true + else + false ) val topicPartitionReplicaList: Seq[TopicPartitionReplica] = filteredTopicMetadata.flatMap( topicMetadataResponse => @@ -190,7 +197,7 @@ private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, c private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int], leadersPerBroker: Map[Int, Seq[TopicAndPartition]], expectedNumFetchers: Int, - brokerMap: Map[Int, Broker], + brokerMap: Map[Int, BrokerEndPoint], initialOffsetTime: Long, reportInterval: Long) extends Logging { private val fetchOffsetMap = new Pool[TopicAndPartition, Long] @@ -299,7 +306,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa } else isMessageInAllReplicas = false } catch { - case t => + case t: Throwable => throw new RuntimeException("Error in processing replica %d in partition %s at offset %d." .format(replicaId, topicAndPartition, fetchOffsetMap.get(topicAndPartition)), t) } @@ -328,7 +335,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa } } -private class ReplicaFetcher(name: String, sourceBroker: Broker, topicAndPartitions: Iterable[TopicAndPartition], +private class ReplicaFetcher(name: String, sourceBroker: BrokerEndPoint, topicAndPartitions: Iterable[TopicAndPartition], replicaBuffer: ReplicaBuffer, socketTimeout: Int, socketBufferSize: Int, fetchSize: Int, maxWait: Int, minBytes: Int, doVerification: Boolean) extends ShutdownableThread(name) { diff --git a/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala similarity index 95% rename from perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala rename to core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala index c52ada0a30ae4..5e3c605248fd7 100644 --- a/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.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,7 +15,7 @@ * limitations under the License. */ -package kafka.perf +package kafka.tools import java.net.URI import java.text.SimpleDateFormat @@ -31,9 +31,11 @@ import kafka.common.TopicAndPartition */ object SimpleConsumerPerformance { + private val logger = Logger.getLogger(getClass()) + def main(args: Array[String]) { - val logger = Logger.getLogger(getClass) val config = new ConsumerPerfConfig(args) + logger.info("Starting SimpleConsumer...") if(!config.hideHeader) { if(!config.showDetailedStats) @@ -74,17 +76,17 @@ object SimpleConsumerPerformance { messagesRead += 1 bytesRead += message.message.payloadSize } - + if(messagesRead == 0 || totalMessagesRead > config.numMessages) done = true else // we only did one fetch so we find the offset for the first (head) messageset - offset += messageSet.validBytes - + offset = messageSet.last.nextOffset + totalBytesRead += bytesRead totalMessagesRead += messagesRead consumedInterval += messagesRead - + if(consumedInterval > config.reportingInterval) { if(config.showDetailedStats) { val reportTime = System.currentTimeMillis @@ -141,13 +143,8 @@ object SimpleConsumerPerformance { val options = parser.parse(args : _*) - for(arg <- List(topicOpt, urlOpt)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, urlOpt, numMessagesOpt) + val url = new URI(options.valueOf(urlOpt)) val fetchSize = options.valueOf(fetchSizeOpt).intValue val fromLatest = options.has(resetBeginningOffsetOpt) diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala old mode 100644 new mode 100755 index 747e07280cce7..1c2023c0a7e01 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -22,9 +22,10 @@ import kafka.utils._ import kafka.consumer._ import kafka.client.ClientUtils import kafka.api.{OffsetRequest, FetchRequestBuilder, Request} -import kafka.cluster.Broker +import kafka.cluster.BrokerEndPoint import scala.collection.JavaConversions._ import kafka.common.TopicAndPartition +import org.apache.kafka.common.utils.Utils /** * Command line program to dump out messages to standard out using the simple consumer @@ -93,15 +94,12 @@ object SimpleConsumerShell extends Logging { "skip it instead of halt.") val noWaitAtEndOfLogOpt = parser.accepts("no-wait-at-logend", "If set, when the simple consumer reaches the end of the Log, it will stop, not waiting for new produced messages") + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "A low-level tool for fetching data directly from a particular replica.") val options = parser.parse(args : _*) - for(arg <- List(brokerListOpt, topicOpt, partitionIdOpt)) { - if(!options.has(arg)) { - error("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } + CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt, topicOpt, partitionIdOpt) val topic = options.valueOf(topicOpt) val partitionId = options.valueOf(partitionIdOpt).intValue() @@ -117,7 +115,7 @@ object SimpleConsumerShell extends Logging { val noWaitAtEndOfLog = options.has(noWaitAtEndOfLogOpt) val messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)) - val formatterArgs = MessageFormatter.tryParseFormatterArgs(options.valuesOf(messageFormatterArgOpt)) + val formatterArgs = CommandLineUtils.parseKeyValueArgs(options.valuesOf(messageFormatterArgOpt)) val fetchRequestBuilder = new FetchRequestBuilder() .clientId(clientId) @@ -127,7 +125,9 @@ object SimpleConsumerShell extends Logging { // getting topic metadata info("Getting topic metatdata...") - val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt)) + val brokerList = options.valueOf(brokerListOpt) + ToolsUtils.validatePortOrDie(parser,brokerList) + val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) 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, " + "what we get from server is only: %s").format(topic, topicsMetadata)) @@ -143,12 +143,12 @@ object SimpleConsumerShell extends Logging { } // validating replica id and initializing target broker - var fetchTargetBroker: Broker = null - var replicaOpt: Option[Broker] = null + var fetchTargetBroker: BrokerEndPoint = null + var replicaOpt: Option[BrokerEndPoint] = null if(replicaId == UseLeaderReplica) { replicaOpt = partitionMetadataOpt.get.leader if(!replicaOpt.isDefined) { - System.err.println("Error: user speicifies to fetch from leader for partition (%s, %d) which has not been elected yet".format(replicaId, topic, partitionId)) + System.err.println("Error: user specifies to fetch from leader for partition (%s, %d) which has not been elected yet".format(topic, partitionId)) System.exit(1) } } @@ -168,7 +168,9 @@ object SimpleConsumerShell extends Logging { System.exit(1) } if (startingOffset < 0) { - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, ConsumerConfig.SocketTimeout, + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, + fetchTargetBroker.port, + ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize, clientId) try { startingOffset = simpleConsumer.earliestOrLatestOffset(TopicAndPartition(topic, partitionId), startingOffset, @@ -189,8 +191,12 @@ object SimpleConsumerShell extends Logging { val replicaString = if(replicaId > 0) "leader" else "replica" info("Starting simple consumer shell to partition [%s, %d], %s [%d], host and port: [%s, %d], from offset [%d]" - .format(topic, partitionId, replicaString, replicaId, fetchTargetBroker.host, fetchTargetBroker.port, startingOffset)) - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, 10000, 64*1024, clientId) + .format(topic, partitionId, replicaString, replicaId, + fetchTargetBroker.host, + fetchTargetBroker.port, startingOffset)) + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, + fetchTargetBroker.port, + 10000, 64*1024, clientId) val thread = Utils.newThread("kafka-simpleconsumer-shell", new Runnable() { def run() { var offset = startingOffset diff --git a/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala b/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala old mode 100644 new mode 100755 index 97970fb941faf..8b523e7518fde --- a/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala +++ b/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala @@ -22,7 +22,7 @@ import scala.util.matching.Regex import collection.mutable import java.util.Date import java.text.SimpleDateFormat -import kafka.utils.Logging +import kafka.utils.{CoreUtils, Logging, CommandLineUtils} import kafka.common.Topic import java.io.{BufferedOutputStream, OutputStream} @@ -83,6 +83,9 @@ object StateChangeLogMerger extends Logging { .describedAs("end timestamp in the format " + dateFormat) .ofType(classOf[String]) .defaultsTo("9999-12-31 23:59:59,999") + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "A tool for merging the log files from several brokers to reconnstruct a unified history of what happened.") val options = parser.parse(args : _*) @@ -112,6 +115,11 @@ object StateChangeLogMerger extends Logging { } if (options.has(partitionsOpt)) { partitions = options.valueOf(partitionsOpt).split(",").toList.map(_.toInt) + val duplicatePartitions = CoreUtils.duplicates(partitions) + if (duplicatePartitions.nonEmpty) { + System.err.println("The list of partitions contains repeated entries: %s".format(duplicatePartitions.mkString(","))) + System.exit(1) + } } startDate = dateFormat.parse(options.valueOf(startTimeOpt).replace('\"', ' ').trim) endDate = dateFormat.parse(options.valueOf(endTimeOpt).replace('\"', ' ').trim) diff --git a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala old mode 100644 new mode 100755 index 111c9a8b94ce4..994268662f30c --- a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala +++ b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala @@ -21,8 +21,9 @@ import org.I0Itec.zkclient.ZkClient import kafka.consumer.{SimpleConsumer, ConsumerConfig} import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} import kafka.common.{TopicAndPartition, KafkaException} -import kafka.utils.{ZKGroupTopicDirs, ZkUtils, ZKStringSerializer, Utils} - +import kafka.utils.{ZKGroupTopicDirs, ZkUtils, CoreUtils} +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.utils.Utils /** * A utility that updates the offset of every broker partition to the offset of earliest or latest log segment file, in ZK. @@ -35,8 +36,8 @@ object UpdateOffsetsInZK { if(args.length < 3) usage val config = new ConsumerConfig(Utils.loadProps(args(1))) - val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, - config.zkConnectionTimeoutMs, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) args(0) match { case Earliest => getAndSetOffsets(zkClient, OffsetRequest.EarliestTime, config, args(2)) case Latest => getAndSetOffsets(zkClient, OffsetRequest.LatestTime, config, args(2)) @@ -65,7 +66,9 @@ object UpdateOffsetsInZK { ZkUtils.getBrokerInfo(zkClient, broker) match { case Some(brokerInfo) => - val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024, "UpdateOffsetsInZk") + val consumer = new SimpleConsumer(brokerInfo.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host, + brokerInfo.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port, + 10000, 100 * 1024, "UpdateOffsetsInZk") val topicAndPartition = TopicAndPartition(topic, partition) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1))) val offset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head diff --git a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala index eac9af2e1d459..db2721fec60cf 100644 --- a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala +++ b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala @@ -19,17 +19,20 @@ package kafka.tools import joptsimple.OptionParser import org.I0Itec.zkclient.ZkClient -import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils, ZKStringSerializer} +import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils, CommandLineUtils} object VerifyConsumerRebalance extends Logging { def main(args: Array[String]) { val parser = new OptionParser() val zkConnectOpt = parser.accepts("zookeeper.connect", "ZooKeeper connect string."). - withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]); + withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]) val groupOpt = parser.accepts("group", "Consumer group."). withRequiredArg().ofType(classOf[String]) parser.accepts("help", "Print this message.") + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "Validate that all partitions have a consumer for a given consumer group.") val options = parser.parse(args : _*) @@ -38,19 +41,14 @@ object VerifyConsumerRebalance extends Logging { System.exit(0) } - for (opt <- List(groupOpt)) - if (!options.has(opt)) { - System.err.println("Missing required argument: %s".format(opt)) - parser.printHelpOn(System.err) - System.exit(1) - } + CommandLineUtils.checkRequiredArgs(parser, options, groupOpt) val zkConnect = options.valueOf(zkConnectOpt) val group = options.valueOf(groupOpt) var zkClient: ZkClient = null try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000) debug("zkConnect = %s; group = %s".format(zkConnect, group)) @@ -78,12 +76,10 @@ object VerifyConsumerRebalance extends Logging { * This means that for each partition registered under /brokers/topics/[topic]/[broker-id], an owner exists * under /consumers/[consumer_group]/owners/[topic]/[broker_id-partition_id] */ - val consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group) + val consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group, excludeInternalTopics = false) val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, consumersPerTopicMap.keySet.toSeq) - partitionsPerTopicMap.foreach { partitionsForTopic => - val topic = partitionsForTopic._1 - val partitions = partitionsForTopic._2 + partitionsPerTopicMap.foreach { case (topic, partitions) => val topicDirs = new ZKGroupTopicDirs(group, topic) info("Alive partitions for topic %s are %s ".format(topic, partitions.toString)) info("Alive consumers for topic %s => %s ".format(topic, consumersPerTopicMap.get(topic))) @@ -97,8 +93,8 @@ object VerifyConsumerRebalance extends Logging { // for each available partition for topic, check if an owner exists partitions.foreach { partition => - // check if there is a node for [partition] - if(!partitionsWithOwners.exists(p => p.equals(partition))) { + // check if there is a node for [partition] + if(!partitionsWithOwners.contains(partition.toString)) { error("No owner for partition [%s,%d]".format(topic, partition)) rebalanceSucceeded = false } diff --git a/core/src/main/scala/kafka/tools/ZooKeeperMainWrapper.scala b/core/src/main/scala/kafka/tools/ZooKeeperMainWrapper.scala new file mode 100644 index 0000000000000..4c51f312e9ee8 --- /dev/null +++ b/core/src/main/scala/kafka/tools/ZooKeeperMainWrapper.scala @@ -0,0 +1,39 @@ +/* + * 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 org.apache.zookeeper.ZooKeeperMain + +class ZooKeeperMainWrapper(args: Array[String]) extends ZooKeeperMain(args) { + def runCmd(): Unit = { + processCmd(this.cl) + System.exit(0) + } +} + +/** + * ZooKeeper 3.4.6 broke being able to pass commands on command line. + * See ZOOKEEPER-1897. This class is a hack to restore this faclity. + */ +object ZooKeeperMainWrapper { + + def main(args: Array[String]): Unit = { + val main: ZooKeeperMainWrapper = new ZooKeeperMainWrapper(args) + main.runCmd() + } +} diff --git a/core/src/main/scala/kafka/utils/Annotations_2.9+.scala b/core/src/main/scala/kafka/utils/Annotations.scala similarity index 100% rename from core/src/main/scala/kafka/utils/Annotations_2.9+.scala rename to core/src/main/scala/kafka/utils/Annotations.scala diff --git a/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala b/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala new file mode 100644 index 0000000000000..26149af943c1d --- /dev/null +++ b/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala @@ -0,0 +1,230 @@ +/** + * 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.concurrent.atomic.AtomicInteger +import java.util.concurrent.{TimeUnit, LinkedBlockingQueue} + +/** + * A blocking queue that have size limits on both number of elements and number of bytes. + */ +class ByteBoundedBlockingQueue[E] (val queueNumMessageCapacity: Int, val queueByteCapacity: Int, sizeFunction: Option[(E) => Int]) + extends Iterable[E] { + private val queue = new LinkedBlockingQueue[E] (queueNumMessageCapacity) + private var currentByteSize = new AtomicInteger() + private val putLock = new Object + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#offer]] + * An element can be enqueued provided the current size (in number of elements) is within the configured + * capacity and the current size in bytes of the queue is within the configured byte capacity. i.e., the + * element may be enqueued even if adding it causes the queue's size in bytes to exceed the byte capacity. + * @param e the element to put into the queue + * @param timeout the amount of time to wait before the expire the operation + * @param unit the time unit of timeout parameter, default to millisecond + * @return true if the element is put into queue, false if it is not + * @throws NullPointerException if element is null + * @throws InterruptedException if interrupted during waiting + */ + def offer(e: E, timeout: Long, unit: TimeUnit = TimeUnit.MICROSECONDS): Boolean = { + if (e == null) throw new NullPointerException("Putting null element into queue.") + val startTime = SystemTime.nanoseconds + val expireTime = startTime + unit.toNanos(timeout) + putLock synchronized { + var timeoutNanos = expireTime - SystemTime.nanoseconds + while (currentByteSize.get() >= queueByteCapacity && timeoutNanos > 0) { + // ensure that timeoutNanos > 0, otherwise (per javadoc) we have to wait until the next notify + putLock.wait(timeoutNanos / 1000000, (timeoutNanos % 1000000).toInt) + timeoutNanos = expireTime - SystemTime.nanoseconds + } + // only proceed if queue has capacity and not timeout + timeoutNanos = expireTime - SystemTime.nanoseconds + if (currentByteSize.get() < queueByteCapacity && timeoutNanos > 0) { + val success = queue.offer(e, timeoutNanos, TimeUnit.NANOSECONDS) + // only increase queue byte size if put succeeds + if (success) + currentByteSize.addAndGet(sizeFunction.get(e)) + // wake up another thread in case multiple threads are waiting + if (currentByteSize.get() < queueByteCapacity) + putLock.notify() + success + } else { + false + } + } + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#offer]]. + * Put an element to the tail of the queue, return false immediately if queue is full + * @param e The element to put into queue + * @return true on succeed, false on failure + * @throws NullPointerException if element is null + * @throws InterruptedException if interrupted during waiting + */ + def offer(e: E): Boolean = { + if (e == null) throw new NullPointerException("Putting null element into queue.") + putLock synchronized { + if (currentByteSize.get() >= queueByteCapacity) { + false + } else { + val success = queue.offer(e) + if (success) + currentByteSize.addAndGet(sizeFunction.get(e)) + // wake up another thread in case multiple threads are waiting + if (currentByteSize.get() < queueByteCapacity) + putLock.notify() + success + } + } + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#put]]. + * Put an element to the tail of the queue, block if queue is full + * @param e The element to put into queue + * @return true on succeed, false on failure + * @throws NullPointerException if element is null + * @throws InterruptedException if interrupted during waiting + */ + def put(e: E): Boolean = { + if (e == null) throw new NullPointerException("Putting null element into queue.") + putLock synchronized { + if (currentByteSize.get() >= queueByteCapacity) + putLock.wait() + val success = queue.offer(e) + if (success) + currentByteSize.addAndGet(sizeFunction.get(e)) + // wake up another thread in case multiple threads are waiting + if (currentByteSize.get() < queueByteCapacity) + putLock.notify() + success + } + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#poll]] + * Get an element from the head of queue. Wait for some time if the queue is empty. + * @param timeout the amount of time to wait if the queue is empty + * @param unit the unit type + * @return the first element in the queue, null if queue is empty + */ + def poll(timeout: Long, unit: TimeUnit): E = { + val e = queue.poll(timeout, unit) + // only wake up waiting threads if the queue size drop under queueByteCapacity + if (e != null && + currentByteSize.getAndAdd(-sizeFunction.get(e)) > queueByteCapacity && + currentByteSize.get() < queueByteCapacity) + putLock.synchronized(putLock.notify()) + e + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#poll]] + * Get an element from the head of queue. + * @return the first element in the queue, null if queue is empty + */ + def poll(): E = { + val e = queue.poll() + // only wake up waiting threads if the queue size drop under queueByteCapacity + if (e != null && + currentByteSize.getAndAdd(-sizeFunction.get(e)) > queueByteCapacity && + currentByteSize.get() < queueByteCapacity) + putLock.synchronized(putLock.notify()) + e + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#take]] + * Get an element from the head of the queue, block if the queue is empty + * @return the first element in the queue, null if queue is empty + */ + def take(): E = { + val e = queue.take() + // only wake up waiting threads if the queue size drop under queueByteCapacity + if (currentByteSize.getAndAdd(-sizeFunction.get(e)) > queueByteCapacity && + currentByteSize.get() < queueByteCapacity) + putLock.synchronized(putLock.notify()) + e + } + + /** + * Iterator for the queue + * @return Iterator for the queue + */ + override def iterator = new Iterator[E] () { + private val iter = queue.iterator() + private var curr: E = null.asInstanceOf[E] + + def hasNext: Boolean = iter.hasNext + + def next(): E = { + curr = iter.next() + curr + } + + def remove() { + if (curr == null) + throw new IllegalStateException("Iterator does not have a current element.") + iter.remove() + if (currentByteSize.addAndGet(-sizeFunction.get(curr)) < queueByteCapacity) + putLock.synchronized(putLock.notify()) + } + } + + /** + * get the number of elements in the queue + * @return number of elements in the queue + */ + override def size() = queue.size() + + /** + * get the current byte size in the queue + * @return current queue size in bytes + */ + def byteSize() = { + val currSize = currentByteSize.get() + // There is a potential race where after an element is put into the queue and before the size is added to + // currentByteSize, it was taken out of the queue and the size was deducted from the currentByteSize, + // in that case, currentByteSize would become negative, in that case, just put the queue size to be 0. + if (currSize > 0) currSize else 0 + } + + /** + * get the number of unused slots in the queue + * @return the number of unused slots in the queue + */ + def remainingSize = queue.remainingCapacity() + + /** + * get the remaining bytes capacity of the queue + * @return the remaining bytes capacity of the queue + */ + def remainingByteSize = math.max(0, queueByteCapacity - currentByteSize.get()) + + /** + * remove all the items in the queue + */ + def clear() { + putLock synchronized { + queue.clear() + currentByteSize.set(0) + putLock.notify() + } + } +} diff --git a/core/src/main/scala/kafka/utils/CommandLineUtils.scala b/core/src/main/scala/kafka/utils/CommandLineUtils.scala index 726c30257dcb5..086a62483fad0 100644 --- a/core/src/main/scala/kafka/utils/CommandLineUtils.scala +++ b/core/src/main/scala/kafka/utils/CommandLineUtils.scala @@ -18,31 +18,59 @@ import joptsimple.{OptionSpec, OptionSet, OptionParser} import scala.collection.Set + import java.util.Properties /** * Helper functions for dealing with command line utilities */ object CommandLineUtils extends Logging { + /** + * Check that all the listed options are present + */ def checkRequiredArgs(parser: OptionParser, options: OptionSet, required: OptionSpec[_]*) { for(arg <- required) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } + if(!options.has(arg)) + printUsageAndDie(parser, "Missing required argument \"" + arg + "\"") } } + /** + * Check that none of the listed options are present + */ def checkInvalidArgs(parser: OptionParser, options: OptionSet, usedOption: OptionSpec[_], invalidOptions: Set[OptionSpec[_]]) { if(options.has(usedOption)) { for(arg <- invalidOptions) { - if(options.has(arg)) { - System.err.println("Option \"" + usedOption + "\" can't be used with option\"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } + if(options.has(arg)) + printUsageAndDie(parser, "Option \"" + usedOption + "\" can't be used with option\"" + arg + "\"") + } + } + } + + /** + * Print usage and exit + */ + def printUsageAndDie(parser: OptionParser, message: String) { + System.err.println(message) + parser.printHelpOn(System.err) + System.exit(1) + } + + /** + * Parse key-value pairs in the form key=value + */ + def parseKeyValueArgs(args: Iterable[String]): Properties = { + val splits = args.map(_ split "=").filterNot(_.length == 0) + + val props = new Properties + for(a <- splits) { + if (a.length == 1) props.put(a(0), "") + else if (a.length == 2) props.put(a(0), a(1)) + else { + System.err.println("Invalid command line properties: " + args.mkString(" ")) + System.exit(1) } } + props } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala new file mode 100755 index 0000000000000..168a18d380c20 --- /dev/null +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -0,0 +1,311 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.utils + +import java.io._ +import java.nio._ +import charset.Charset +import java.nio.channels._ +import java.util.concurrent.locks.{ReadWriteLock, Lock} +import java.lang.management._ +import javax.management._ + +import org.apache.kafka.common.protocol.SecurityProtocol + +import scala.collection._ +import scala.collection.mutable +import java.util.Properties +import kafka.cluster.EndPoint +import kafka.common.KafkaException +import kafka.common.KafkaStorageException +import org.apache.kafka.common.utils.Crc32 +import org.apache.kafka.common.utils.Utils + + +/** + * General helper functions! + * + * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in + * the standard library etc. + * + * If you are making a new helper function and want to add it to this class please ensure the following: + * 1. It has documentation + * 2. It is the most general possible utility, not just the thing you needed in one particular place + * 3. You have tests for it if it is nontrivial in any way + */ +object CoreUtils extends Logging { + + /** + * Wrap the given function in a java.lang.Runnable + * @param fun A function + * @return A Runnable that just executes the function + */ + def runnable(fun: => Unit): Runnable = + new Runnable { + def run() = fun + } + + /** + * Create a daemon thread + * @param name The name of the thread + * @param fun The runction to execute in the thread + * @return The unstarted thread + */ + def daemonThread(name: String, fun: => Unit): Thread = + Utils.daemonThread(name, runnable(fun)) + + /** + * Do the given action and log any exceptions thrown without rethrowing them + * @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) { + try { + action + } catch { + case e: Throwable => log(e.getMessage(), e) + } + } + + /** + * Recursively delete the given file/directory and any subfiles (if any exist) + * @param file The root file at which to begin deleting + */ + def rm(file: String): Unit = rm(new File(file)) + + /** + * Recursively delete the list of files/directories and any subfiles (if any exist) + * @param files sequence of files to be deleted + */ + def rm(files: Seq[String]): Unit = files.foreach(f => rm(new File(f))) + + /** + * Recursively delete the given file/directory and any subfiles (if any exist) + * @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) + } + file.delete() + } else { + file.delete() + } + } + + /** + * Register the given mbean with the platform mbean server, + * unregistering any mbean that was there before. Note, + * this method will not throw an exception if the registration + * fails (since there is nothing you can do and it isn't fatal), + * instead it just returns false indicating the registration failed. + * @param mbean The object to register as an mbean + * @param name The name to register this mbean with + * @return true if the registration succeeded + */ + def registerMBean(mbean: Object, name: String): Boolean = { + try { + val mbs = ManagementFactory.getPlatformMBeanServer() + mbs synchronized { + val objName = new ObjectName(name) + if(mbs.isRegistered(objName)) + mbs.unregisterMBean(objName) + mbs.registerMBean(mbean, objName) + true + } + } catch { + case e: Exception => { + error("Failed to register Mbean " + name, e) + false + } + } + } + + /** + * Unregister the mbean with the given name, if there is one registered + * @param name The mbean name to unregister + */ + def unregisterMBean(name: String) { + val mbs = ManagementFactory.getPlatformMBeanServer() + mbs synchronized { + val objName = new ObjectName(name) + if(mbs.isRegistered(objName)) + mbs.unregisterMBean(objName) + } + } + + /** + * Compute the CRC32 of the byte array + * @param bytes The array to compute the checksum for + * @return The CRC32 + */ + def crc32(bytes: Array[Byte]): Long = crc32(bytes, 0, bytes.length) + + /** + * Compute the CRC32 of the segment of the byte array given by the specificed size and offset + * @param bytes The bytes to checksum + * @param offset the offset at which to begin checksumming + * @param size the number of bytes to checksum + * @return The CRC32 + */ + def crc32(bytes: Array[Byte], offset: Int, size: Int): Long = { + val crc = new Crc32() + crc.update(bytes, offset, size) + crc.getValue() + } + + /** + * Read some bytes into the provided buffer, and return the number of bytes read. If the + * channel has been closed or we get -1 on the read for any reason, throw an EOFException + */ + def read(channel: ReadableByteChannel, buffer: ByteBuffer): Int = { + channel.read(buffer) match { + case -1 => throw new EOFException("Received -1 when reading from channel, socket has likely been closed.") + case n: Int => n + } + } + + /** + * This method gets comma separated values which contains key,value pairs and returns a map of + * key value pairs. the format of allCSVal is key1:val1, key2:val2 .... + * Also supports strings with multiple ":" such as IpV6 addresses, taking the last occurrence + * of the ":" in the pair as the split, eg a:b:c:val1, d:e:f:val2 => a:b:c -> val1, d:e:f -> val2 + */ + def parseCsvMap(str: String): Map[String, String] = { + val map = new mutable.HashMap[String, String] + if ("".equals(str)) + return map + val keyVals = str.split("\\s*,\\s*").map(s => { + val lio = s.lastIndexOf(":") + (s.substring(0,lio).trim, s.substring(lio + 1).trim) + }) + keyVals.toMap + } + + /** + * Parse a comma separated string into a sequence of strings. + * Whitespace surrounding the comma will be removed. + */ + def parseCsvList(csvList: String): Seq[String] = { + if(csvList == null || csvList.isEmpty) + Seq.empty[String] + else { + csvList.split("\\s*,\\s*").filter(v => !v.equals("")) + } + } + + /** + * Create an instance of the class with the given class name + */ + def createObject[T<:AnyRef](className: String, args: AnyRef*): T = { + val klass = Class.forName(className).asInstanceOf[Class[T]] + val constructor = klass.getConstructor(args.map(_.getClass): _*) + constructor.newInstance(args: _*) + } + + /** + * Create a circular (looping) iterator over a collection. + * @param coll An iterable over the underlying collection. + * @return A circular iterator over the collection. + */ + def circularIterator[T](coll: Iterable[T]) = { + val stream: Stream[T] = + for (forever <- Stream.continually(1); t <- coll) yield t + stream.iterator + } + + /** + * 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 + } + + /** + * 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 { + fun + } finally { + lock.unlock() + } + } + + def inReadLock[T](lock: ReadWriteLock)(fun: => T): T = inLock[T](lock.readLock)(fun) + + def inWriteLock[T](lock: ReadWriteLock)(fun: => T): T = inLock[T](lock.writeLock)(fun) + + + //JSON strings need to be escaped based on ECMA-404 standard http://json.org + def JSONEscapeString (s : String) : String = { + s.map { + case '"' => "\\\"" + case '\\' => "\\\\" + case '/' => "\\/" + case '\b' => "\\b" + case '\f' => "\\f" + case '\n' => "\\n" + case '\r' => "\\r" + case '\t' => "\\t" + /* We'll unicode escape any control characters. These include: + * 0x0 -> 0x1f : ASCII Control (C0 Control Codes) + * 0x7f : ASCII DELETE + * 0x80 -> 0x9f : C1 Control Codes + * + * Per RFC4627, section 2.5, we're not technically required to + * encode the C1 codes, but we do to be safe. + */ + case c if ((c >= '\u0000' && c <= '\u001f') || (c >= '\u007f' && c <= '\u009f')) => "\\u%04x".format(c: Int) + case c => c + }.mkString + } + + /** + * Returns a list of duplicated items + */ + def duplicates[T](s: Traversable[T]): Iterable[T] = { + s.groupBy(identity) + .map{ case (k,l) => (k,l.size)} + .filter{ case (k,l) => (l > 1) } + .keys + } + + def listenerListToEndPoints(listeners: String): immutable.Map[SecurityProtocol, EndPoint] = { + val listenerList = parseCsvList(listeners) + listenerList.map(listener => EndPoint.createEndPoint(listener)).map(ep => ep.protocolType -> ep).toMap + } +} diff --git a/core/src/main/scala/kafka/utils/Crc32.java b/core/src/main/scala/kafka/utils/Crc32.java deleted file mode 100644 index af9fe0d7d4ab2..0000000000000 --- a/core/src/main/scala/kafka/utils/Crc32.java +++ /dev/null @@ -1,637 +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.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/DelayedItem.scala b/core/src/main/scala/kafka/utils/DelayedItem.scala index d7276494072f1..cbab2a01bfbfb 100644 --- a/core/src/main/scala/kafka/utils/DelayedItem.scala +++ b/core/src/main/scala/kafka/utils/DelayedItem.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 @@ -20,34 +20,25 @@ package kafka.utils import java.util.concurrent._ import scala.math._ -class DelayedItem[T](val item: T, delay: Long, unit: TimeUnit) extends Delayed with Logging { +class DelayedItem(delayMs: Long) extends Delayed with Logging { - val createdMs = SystemTime.milliseconds - val delayMs = { - val given = unit.toMillis(delay) - if (given < 0 || (createdMs + given) < 0) (Long.MaxValue - createdMs) - else given - } + private val dueMs = SystemTime.milliseconds + delayMs - def this(item: T, delayMs: Long) = - this(item, delayMs, TimeUnit.MILLISECONDS) + def this(delay: Long, unit: TimeUnit) = this(unit.toMillis(delay)) /** * The remaining delay time */ def getDelay(unit: TimeUnit): Long = { - val elapsedMs = (SystemTime.milliseconds - createdMs) - unit.convert(max(delayMs - elapsedMs, 0), TimeUnit.MILLISECONDS) + unit.convert(max(dueMs - SystemTime.milliseconds, 0), TimeUnit.MILLISECONDS) } - + def compareTo(d: Delayed): Int = { - val delayed = d.asInstanceOf[DelayedItem[T]] - val myEnd = createdMs + delayMs - val yourEnd = delayed.createdMs + delayed.delayMs + val other = d.asInstanceOf[DelayedItem] - if(myEnd < yourEnd) -1 - else if(myEnd > yourEnd) 1 + if(dueMs < other.dueMs) -1 + else if(dueMs > other.dueMs) 1 else 0 } - + } diff --git a/core/src/main/scala/kafka/utils/DoublyLinkedList.scala b/core/src/main/scala/kafka/utils/DoublyLinkedList.scala new file mode 100644 index 0000000000000..e637ef31d0945 --- /dev/null +++ b/core/src/main/scala/kafka/utils/DoublyLinkedList.scala @@ -0,0 +1,126 @@ +/** + * 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 + +/** + * Simple doubly LinkedList node + * @param element The element + * @tparam T The type of element + */ +class DoublyLinkedListNode[T] (val element: T) { + var prev: DoublyLinkedListNode[T] = null + var next: DoublyLinkedListNode[T] = null +} + +/** + * A simple doubly linked list util to allow O(1) remove. + * @tparam T type of element in nodes + */ +@threadsafe +class DoublyLinkedList[T] { + private var head: DoublyLinkedListNode[T] = null + private var tail: DoublyLinkedListNode[T] = null + @volatile private var listSize: Int = 0 + + /** + * Add offset to the tail of the list + * @param node the node to be added to the tail of the list + */ + def add (node: DoublyLinkedListNode[T]) { + this synchronized { + if (head == null) { + // empty list + head = node + tail = node + node.prev = null + node.next = null + } else { + // add to tail + tail.next = node + node.next = null + node.prev = tail + tail = node + } + listSize += 1 + } + } + + /** + * Remove a node from the list. The list will not check if the node is really in the list. + * @param node the node to be removed from the list + */ + def remove (node: DoublyLinkedListNode[T]) { + this synchronized { + if (node ne head) + node.prev.next = node.next + else + head = node.next + + if (node ne tail) + node.next.prev = node.prev + else + tail = node.prev + + node.prev = null + node.next = null + + listSize -= 1 + } + } + + /** + * Remove the first node in the list and return it if the list is not empty. + * @return The first node in the list if the list is not empty. Return Null if the list is empty. + */ + def remove(): DoublyLinkedListNode[T] = { + this synchronized { + if (head != null) { + val node = head + remove(head) + node + } else { + null + } + } + } + + /** + * Get the first node in the list without removing it. + * @return the first node in the list. + */ + def peek(): DoublyLinkedListNode[T] = head + + def size: Int = listSize + + def iterator: Iterator[DoublyLinkedListNode[T]] = { + new IteratorTemplate[DoublyLinkedListNode[T]] { + var current = head + override protected def makeNext(): DoublyLinkedListNode[T] = { + this synchronized { + if (current != null) { + val nextNode = current + current = current.next + nextNode + } else { + allDone() + } + } + } + } + } +} diff --git a/core/src/main/scala/kafka/utils/KafkaScheduler.scala b/core/src/main/scala/kafka/utils/KafkaScheduler.scala old mode 100644 new mode 100755 index 8e3750506c180..5bab08d473eb5 --- a/core/src/main/scala/kafka/utils/KafkaScheduler.scala +++ b/core/src/main/scala/kafka/utils/KafkaScheduler.scala @@ -19,7 +19,7 @@ package kafka.utils import java.util.concurrent._ import atomic._ -import collection.mutable.HashMap +import org.apache.kafka.common.utils.Utils /** * A scheduler for running jobs @@ -40,6 +40,11 @@ trait Scheduler { */ def shutdown() + /** + * Check if the scheduler has been started + */ + def isStarted: Boolean + /** * Schedule a task * @param name The name of this task @@ -63,13 +68,13 @@ trait Scheduler { class KafkaScheduler(val threads: Int, val threadNamePrefix: String = "kafka-scheduler-", daemon: Boolean = true) extends Scheduler with Logging { - @volatile private var executor: ScheduledThreadPoolExecutor = null + private var executor: ScheduledThreadPoolExecutor = null private val schedulerThreadId = new AtomicInteger(0) override def startup() { debug("Initializing task scheduler.") this synchronized { - if(executor != null) + if(isStarted) throw new IllegalStateException("This scheduler has already been started!") executor = new ScheduledThreadPoolExecutor(threads) executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false) @@ -83,18 +88,21 @@ class KafkaScheduler(val threads: Int, override def shutdown() { debug("Shutting down task scheduler.") - ensureStarted - executor.shutdown() - executor.awaitTermination(1, TimeUnit.DAYS) - this.executor = null + this synchronized { + if(isStarted) { + executor.shutdown() + executor.awaitTermination(1, TimeUnit.DAYS) + this.executor = null + } + } } 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() = { + this synchronized { + ensureStarted + val runnable = CoreUtils.runnable { try { trace("Begining execution of scheduled task '%s'.".format(name)) fun() @@ -104,15 +112,21 @@ class KafkaScheduler(val threads: Int, trace("Completed execution of scheduled task '%s'.".format(name)) } } + if(period >= 0) + executor.scheduleAtFixedRate(runnable, delay, period, unit) + else + executor.schedule(runnable, delay, unit) + } + } + + def isStarted: Boolean = { + this synchronized { + executor != null } - if(period >= 0) - executor.scheduleAtFixedRate(runnable, delay, period, unit) - else - executor.schedule(runnable, delay, unit) } private def ensureStarted = { - if(executor == null) + if(!isStarted) throw new IllegalStateException("Kafka scheduler has not been started") } } diff --git a/core/src/main/scala/kafka/utils/Log4jController.scala b/core/src/main/scala/kafka/utils/Log4jController.scala old mode 100644 new mode 100755 index a015c8109f050..673d84e33a48a --- a/core/src/main/scala/kafka/utils/Log4jController.scala +++ b/core/src/main/scala/kafka/utils/Log4jController.scala @@ -26,7 +26,7 @@ object Log4jController { private val controller = new Log4jController - Utils.registerMBean(controller, "kafka:type=kafka.Log4jController") + CoreUtils.registerMBean(controller, "kafka:type=kafka.Log4jController") } diff --git a/core/src/main/scala/kafka/utils/Logging.scala b/core/src/main/scala/kafka/utils/Logging.scala old mode 100644 new mode 100755 index 2890e7f51ddcb..d34c46421c281 --- a/core/src/main/scala/kafka/utils/Logging.scala +++ b/core/src/main/scala/kafka/utils/Logging.scala @@ -44,7 +44,7 @@ trait Logging { logger.trace(msgWithLogIdent(msg),e) } def swallowTrace(action: => Unit) { - Utils.swallow(logger.trace, action) + CoreUtils.swallow(logger.trace, action) } def debug(msg: => String): Unit = { @@ -60,7 +60,7 @@ trait Logging { logger.debug(msgWithLogIdent(msg),e) } def swallowDebug(action: => Unit) { - Utils.swallow(logger.debug, action) + CoreUtils.swallow(logger.debug, action) } def info(msg: => String): Unit = { @@ -76,7 +76,7 @@ trait Logging { logger.info(msgWithLogIdent(msg),e) } def swallowInfo(action: => Unit) { - Utils.swallow(logger.info, action) + CoreUtils.swallow(logger.info, action) } def warn(msg: => String): Unit = { @@ -89,7 +89,7 @@ trait Logging { logger.warn(msgWithLogIdent(msg),e) } def swallowWarn(action: => Unit) { - Utils.swallow(logger.warn, action) + CoreUtils.swallow(logger.warn, action) } def swallow(action: => Unit) = swallowWarn(action) @@ -103,7 +103,7 @@ trait Logging { logger.error(msgWithLogIdent(msg),e) } def swallowError(action: => Unit) { - Utils.swallow(logger.error, action) + CoreUtils.swallow(logger.error, action) } def fatal(msg: => String): Unit = { diff --git a/core/src/main/scala/kafka/utils/Mx4jLoader.scala b/core/src/main/scala/kafka/utils/Mx4jLoader.scala index db9f20bd5b03e..aa120abc0bce9 100644 --- a/core/src/main/scala/kafka/utils/Mx4jLoader.scala +++ b/core/src/main/scala/kafka/utils/Mx4jLoader.scala @@ -35,11 +35,11 @@ object Mx4jLoader extends Logging { def maybeLoad(): Boolean = { val props = new VerifiableProperties(System.getProperties()) if (props.getBoolean("kafka_mx4jenable", false)) - false + return false val address = props.getString("mx4jaddress", "0.0.0.0") val port = props.getInt("mx4jport", 8082) try { - debug("Will try to load MX4j now, if it's in the classpath"); + debug("Will try to load MX4j now, if it's in the classpath") val mbs = ManagementFactory.getPlatformMBeanServer() val processorName = new ObjectName("Server:name=XSLTProcessor") @@ -62,10 +62,10 @@ object Mx4jLoader extends Logging { } catch { case e: ClassNotFoundException => { - info("Will not load MX4J, mx4j-tools.jar is not in the classpath"); + info("Will not load MX4J, mx4j-tools.jar is not in the classpath") } case e: Throwable => { - warn("Could not start register mbean in JMX", e); + warn("Could not start register mbean in JMX", e) } } false diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala new file mode 100644 index 0000000000000..783ba1026d0be --- /dev/null +++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala @@ -0,0 +1,92 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.utils + +import kafka.api.LeaderAndIsr +import kafka.common.TopicAndPartition +import kafka.controller.LeaderIsrAndControllerEpoch +import org.I0Itec.zkclient.ZkClient +import org.apache.zookeeper.data.Stat + +import scala.collection._ + +object ReplicationUtils extends Logging { + + val IsrChangeNotificationPrefix = "isr_change_" + + def updateLeaderAndIsr(zkClient: ZkClient, topic: String, partitionId: Int, newLeaderAndIsr: LeaderAndIsr, controllerEpoch: Int, + zkVersion: Int): (Boolean,Int) = { + debug("Updated ISR for partition [%s,%d] to %s".format(topic, partitionId, newLeaderAndIsr.isr.mkString(","))) + val path = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partitionId) + val newLeaderData = ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch) + // use the epoch of the controller that made the leadership decision, instead of the current controller epoch + val updatePersistentPath: (Boolean, Int) = ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) + if (updatePersistentPath._1) { + val topicAndPartition: TopicAndPartition = TopicAndPartition(topic, partitionId) + val isrChangeNotificationPath: String = ZkUtils.createSequentialPersistentPath( + zkClient, ZkUtils.IsrChangeNotificationPath + "/" + IsrChangeNotificationPrefix, + topicAndPartition.toJson) + debug("Added " + isrChangeNotificationPath + " for " + topicAndPartition) + } + updatePersistentPath + } + + def checkLeaderAndIsrZkData(zkClient: ZkClient, path: String, expectedLeaderAndIsrInfo: String): (Boolean,Int) = { + try { + val writtenLeaderAndIsrInfo = ZkUtils.readDataMaybeNull(zkClient, path) + val writtenLeaderOpt = writtenLeaderAndIsrInfo._1 + val writtenStat = writtenLeaderAndIsrInfo._2 + val expectedLeader = parseLeaderAndIsr(expectedLeaderAndIsrInfo, path, writtenStat) + writtenLeaderOpt match { + case Some(writtenData) => + val writtenLeader = parseLeaderAndIsr(writtenData, path, writtenStat) + (expectedLeader,writtenLeader) match { + case (Some(expectedLeader),Some(writtenLeader)) => + if(expectedLeader == writtenLeader) + return (true,writtenStat.getVersion()) + case _ => + } + case None => + } + } catch { + case e1: Exception => + } + (false,-1) + } + + def getLeaderIsrAndEpochForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = { + val leaderAndIsrPath = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition) + val (leaderAndIsrOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, leaderAndIsrPath) + leaderAndIsrOpt.flatMap(leaderAndIsrStr => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat)) + } + + private def parseLeaderAndIsr(leaderAndIsrStr: String, path: String, stat: Stat) + : Option[LeaderIsrAndControllerEpoch] = { + Json.parseFull(leaderAndIsrStr).flatMap {m => + val leaderIsrAndEpochInfo = m.asInstanceOf[Map[String, Any]] + val leader = leaderIsrAndEpochInfo.get("leader").get.asInstanceOf[Int] + val epoch = leaderIsrAndEpochInfo.get("leader_epoch").get.asInstanceOf[Int] + val isr = leaderIsrAndEpochInfo.get("isr").get.asInstanceOf[List[Int]] + val controllerEpoch = leaderIsrAndEpochInfo.get("controller_epoch").get.asInstanceOf[Int] + val zkPathVersion = stat.getVersion + debug("Leader %d, Epoch %d, Isr %s, Zk path version %d for leaderAndIsrPath %s".format(leader, epoch, + isr.toString(), zkPathVersion, path)) + Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch))} + } + +} diff --git a/core/src/main/scala/kafka/utils/ShutdownableThread.scala b/core/src/main/scala/kafka/utils/ShutdownableThread.scala index cf8adc9f468f4..fc226c863095b 100644 --- a/core/src/main/scala/kafka/utils/ShutdownableThread.scala +++ b/core/src/main/scala/kafka/utils/ShutdownableThread.scala @@ -27,20 +27,29 @@ abstract class ShutdownableThread(val name: String, val isInterruptible: Boolean val isRunning: AtomicBoolean = new AtomicBoolean(true) private val shutdownLatch = new CountDownLatch(1) + def shutdown() = { + initiateShutdown() + awaitShutdown() + } - def shutdown(): Unit = { - info("Shutting down") - isRunning.set(false) - if (isInterruptible) - interrupt() - shutdownLatch.await() - info("Shutdown completed") + def initiateShutdown(): Boolean = { + if(isRunning.compareAndSet(true, false)) { + info("Shutting down") + isRunning.set(false) + if (isInterruptible) + interrupt() + true + } else + false } /** - * After calling shutdown(), use this API to wait until the shutdown is complete + * After calling initiateShutdown(), use this API to wait until the shutdown is complete */ - def awaitShutdown(): Unit = shutdownLatch.await() + def awaitShutdown(): Unit = { + shutdownLatch.await() + info("Shutdown completed") + } def doWork(): Unit diff --git a/core/src/main/scala/kafka/utils/Throttler.scala b/core/src/main/scala/kafka/utils/Throttler.scala index c6c3c75ee8408..998ade14ef82d 100644 --- a/core/src/main/scala/kafka/utils/Throttler.scala +++ b/core/src/main/scala/kafka/utils/Throttler.scala @@ -15,8 +15,10 @@ * limitations under the License. */ -package kafka.utils; +package kafka.utils +import kafka.metrics.KafkaMetricsGroup +import java.util.concurrent.TimeUnit import java.util.Random import scala.math._ @@ -33,14 +35,18 @@ import scala.math._ @threadsafe class Throttler(val desiredRatePerSec: Double, val checkIntervalMs: Long = 100L, - val throttleDown: Boolean = true, - val time: Time = SystemTime) extends Logging { + val throttleDown: Boolean = true, + metricName: String = "throttler", + units: String = "entries", + val time: Time = SystemTime) extends Logging with KafkaMetricsGroup { private val lock = new Object + private val meter = newMeter(metricName, units, TimeUnit.SECONDS) private var periodStartNs: Long = time.nanoseconds private var observedSoFar: Double = 0.0 def maybeThrottle(observed: Double) { + meter.mark(observed.toLong) lock synchronized { observedSoFar += observed val now = time.nanoseconds @@ -72,7 +78,7 @@ object Throttler { def main(args: Array[String]) { val rand = new Random() - val throttler = new Throttler(100000, 100, true, SystemTime) + val throttler = new Throttler(100000, 100, true, time = SystemTime) val interval = 30000 var start = System.currentTimeMillis var total = 0 @@ -89,4 +95,4 @@ object Throttler { } } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/utils/ToolsUtils.scala b/core/src/main/scala/kafka/utils/ToolsUtils.scala new file mode 100644 index 0000000000000..fef93929ea03e --- /dev/null +++ b/core/src/main/scala/kafka/utils/ToolsUtils.scala @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ +package kafka.utils + +import joptsimple.OptionParser + +object ToolsUtils { + + def validatePortOrDie(parser: OptionParser, hostPort: String) = { + val hostPorts: Array[String] = if(hostPort.contains(',')) + hostPort.split(",") + else + Array(hostPort) + val validHostPort = hostPorts.filter { + hostPortData => + org.apache.kafka.common.utils.Utils.getPort(hostPortData) != null + } + val isValid = !(validHostPort.isEmpty) && validHostPort.size == hostPorts.length + if(!isValid) + CommandLineUtils.printUsageAndDie(parser, "Please provide valid host:port like host1:9091,host2:9092\n ") + } +} diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala deleted file mode 100644 index a89b0463685e6..0000000000000 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ /dev/null @@ -1,544 +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.utils - -import java.io._ -import java.nio._ -import charset.Charset -import java.nio.channels._ -import java.util.concurrent.locks.Lock -import java.lang.management._ -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 - - -/** - * General helper functions! - * - * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in - * the standard library etc. - * - * If you are making a new helper function and want to add it to this class please ensure the following: - * 1. It has documentation - * 2. It is the most general possible utility, not just the thing you needed in one particular place - * 3. You have tests for it if it is nontrivial in any way - */ -object Utils extends Logging { - - /** - * Wrap the given function in a java.lang.Runnable - * @param fun A function - * @return A Runnable that just executes the function - */ - def runnable(fun: () => Unit): Runnable = - new Runnable() { - def run() = fun() - } - - /** - * Create a daemon thread - * @param runnable The runnable to execute in the background - * @return The unstarted thread - */ - def daemonThread(runnable: Runnable): Thread = - newThread(runnable, true) - - /** - * Create a daemon thread - * @param name The name of the thread - * @param runnable The runnable to execute in the background - * @return The unstarted thread - */ - def daemonThread(name: String, runnable: Runnable): Thread = - newThread(name, runnable, true) - - /** - * Create a daemon thread - * @param name The name of the thread - * @param fun The runction to execute in the thread - * @return The unstarted thread - */ - def daemonThread(name: String, fun: () => Unit): Thread = - daemonThread(name, runnable(fun)) - - /** - * Create a new thread - * @param name The name of the thread - * @param runnable The work for the thread to do - * @param daemon Should the thread block JVM shutdown? - * @return The unstarted thread - */ - def newThread(name: String, runnable: Runnable, daemon: Boolean): Thread = { - val thread = new Thread(runnable, name) - thread.setDaemon(daemon) - thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { - def uncaughtException(t: Thread, e: Throwable) { - error("Uncaught exception in thread '" + t.getName + "':", e) - } - }) - thread - } - - /** - * Create a new thread - * @param runnable The work for the thread to do - * @param daemon Should the thread block JVM shutdown? - * @return The unstarted thread - */ - def newThread(runnable: Runnable, daemon: Boolean): Thread = { - val thread = new Thread(runnable) - thread.setDaemon(daemon) - thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { - def uncaughtException(t: Thread, e: Throwable) { - error("Uncaught exception in thread '" + t.getName + "':", e) - } - }) - thread - } - - /** - * Read the given byte buffer into a byte array - */ - def readBytes(buffer: ByteBuffer): Array[Byte] = readBytes(buffer, 0, buffer.limit) - - /** - * Read a byte array from the given offset and size in the buffer - */ - def readBytes(buffer: ByteBuffer, offset: Int, size: Int): Array[Byte] = { - val dest = new Array[Byte](size) - if(buffer.hasArray) { - System.arraycopy(buffer.array, buffer.arrayOffset() + offset, dest, 0, size) - } else { - buffer.mark() - buffer.get(dest) - buffer.reset() - } - dest - } - - /** - * Read a properties file from the given path - * @param filename The path of the file to read - */ - def loadProps(filename: String): Properties = { - val propStream = new FileInputStream(filename) - val props = new Properties() - props.load(propStream) - props - } - - /** - * Open a channel for the given file - */ - def openChannel(file: File, mutable: Boolean): FileChannel = { - if(mutable) - new RandomAccessFile(file, "rw").getChannel() - else - new FileInputStream(file).getChannel() - } - - /** - * Do the given action and log any exceptions thrown without rethrowing them - * @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) { - try { - action - } catch { - case e: Throwable => log(e.getMessage(), e) - } - } - - /** - * Test if two byte buffers are equal. In this case equality means having - * the same bytes from the current position to the limit - */ - def equal(b1: ByteBuffer, b2: ByteBuffer): Boolean = { - // two byte buffers are equal if their position is the same, - // their remaining bytes are the same, and their contents are the same - if(b1.position != b2.position) - return false - if(b1.remaining != b2.remaining) - return false - for(i <- 0 until b1.remaining) - if(b1.get(i) != b2.get(i)) - return false - return true - } - - /** - * Translate the given buffer into a string - * @param buffer The buffer to translate - * @param encoding The encoding to use in translating bytes to characters - */ - def readString(buffer: ByteBuffer, encoding: String = Charset.defaultCharset.toString): String = { - val bytes = new Array[Byte](buffer.remaining) - buffer.get(bytes) - new String(bytes, encoding) - } - - /** - * Print an error message and shutdown the JVM - * @param message The error message - */ - def croak(message: String) { - System.err.println(message) - System.exit(1) - } - - /** - * Recursively delete the given file/directory and any subfiles (if any exist) - * @param file The root file at which to begin deleting - */ - def rm(file: String): Unit = rm(new File(file)) - - /** - * Recursively delete the list of files/directories and any subfiles (if any exist) - * @param a sequence of files to be deleted - */ - def rm(files: Seq[String]): Unit = files.map(f => rm(new File(f))) - - /** - * Recursively delete the given file/directory and any subfiles (if any exist) - * @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) - } - file.delete() - } else { - file.delete() - } - } - - /** - * Register the given mbean with the platform mbean server, - * unregistering any mbean that was there before. Note, - * this method will not throw an exception if the registration - * fails (since there is nothing you can do and it isn't fatal), - * instead it just returns false indicating the registration failed. - * @param mbean The object to register as an mbean - * @param name The name to register this mbean with - * @return true if the registration succeeded - */ - def registerMBean(mbean: Object, name: String): Boolean = { - try { - val mbs = ManagementFactory.getPlatformMBeanServer() - mbs synchronized { - val objName = new ObjectName(name) - if(mbs.isRegistered(objName)) - mbs.unregisterMBean(objName) - mbs.registerMBean(mbean, objName) - true - } - } catch { - case e: Exception => { - error("Failed to register Mbean " + name, e) - false - } - } - } - - /** - * Unregister the mbean with the given name, if there is one registered - * @param name The mbean name to unregister - */ - def unregisterMBean(name: String) { - val mbs = ManagementFactory.getPlatformMBeanServer() - mbs synchronized { - val objName = new ObjectName(name) - if(mbs.isRegistered(objName)) - mbs.unregisterMBean(objName) - } - } - - /** - * Read an unsigned integer from the current position in the buffer, - * incrementing the position by 4 bytes - * @param buffer The buffer to read from - * @return The integer read, as a long to avoid signedness - */ - def readUnsignedInt(buffer: ByteBuffer): Long = - buffer.getInt() & 0xffffffffL - - /** - * Read an unsigned integer from the given position without modifying the buffers - * position - * @param buffer the buffer to read from - * @param index the index from which to read the integer - * @return The integer read, as a long to avoid signedness - */ - def readUnsignedInt(buffer: ByteBuffer, index: Int): Long = - buffer.getInt(index) & 0xffffffffL - - /** - * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. - * @param buffer The buffer to write to - * @param value The value to write - */ - def writetUnsignedInt(buffer: ByteBuffer, value: Long): Unit = - buffer.putInt((value & 0xffffffffL).asInstanceOf[Int]) - - /** - * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. - * @param buffer The buffer to write to - * @param index The position in the buffer at which to begin writing - * @param value The value to write - */ - def writeUnsignedInt(buffer: ByteBuffer, index: Int, value: Long): Unit = - buffer.putInt(index, (value & 0xffffffffL).asInstanceOf[Int]) - - /** - * Compute the CRC32 of the byte array - * @param bytes The array to compute the checksum for - * @return The CRC32 - */ - def crc32(bytes: Array[Byte]): Long = crc32(bytes, 0, bytes.length) - - /** - * Compute the CRC32 of the segment of the byte array given by the specificed size and offset - * @param bytes The bytes to checksum - * @param offset the offset at which to begin checksumming - * @param size the number of bytes to checksum - * @return The CRC32 - */ - def crc32(bytes: Array[Byte], offset: Int, size: Int): Long = { - val crc = new Crc32() - crc.update(bytes, offset, size) - crc.getValue() - } - - /** - * Compute the hash code for the given items - */ - def hashcode(as: Any*): Int = { - if(as == null) - return 0 - var h = 1 - var i = 0 - while(i < as.length) { - if(as(i) != null) { - h = 31 * h + as(i).hashCode - i += 1 - } - } - return h - } - - /** - * Group the given values by keys extracted with the given function - */ - def groupby[K,V](vals: Iterable[V], f: V => K): Map[K,List[V]] = { - val m = new mutable.HashMap[K, List[V]] - for(v <- vals) { - val k = f(v) - m.get(k) match { - case Some(l: List[V]) => m.put(k, v :: l) - case None => m.put(k, List(v)) - } - } - m - } - - /** - * Read some bytes into the provided buffer, and return the number of bytes read. If the - * channel has been closed or we get -1 on the read for any reason, throw an EOFException - */ - def read(channel: ReadableByteChannel, buffer: ByteBuffer): Int = { - channel.read(buffer) match { - case -1 => throw new EOFException("Received -1 when reading from channel, socket has likely been closed.") - case n: Int => n - } - } - - /** - * Throw an exception if the given value is null, else return it. You can use this like: - * val myValue = Utils.notNull(expressionThatShouldntBeNull) - */ - def notNull[V](v: V) = { - if(v == null) - throw new KafkaException("Value cannot be null.") - else - v - } - - /** - * Parse a host and port out of a string - */ - def parseHostPort(hostport: String) : (String, Int) = { - val splits = hostport.split(":") - (splits(0), splits(1).toInt) - } - - /** - * Get the stack trace from an exception as a string - */ - def stackTrace(e: Throwable): String = { - val sw = new StringWriter; - val pw = new PrintWriter(sw); - e.printStackTrace(pw); - sw.toString(); - } - - /** - * This method gets comma separated values which contains key,value pairs and returns a map of - * key value pairs. the format of allCSVal is key1:val1, key2:val2 .... - */ - def parseCsvMap(str: String): Map[String, String] = { - val map = new mutable.HashMap[String, String] - if("".equals(str)) - return map - val keyVals = str.split("\\s*,\\s*").map(s => s.split("\\s*:\\s*")) - keyVals.map(pair => (pair(0), pair(1))).toMap - } - - /** - * Parse a comma separated string into a sequence of strings. - * Whitespace surrounding the comma will be removed. - */ - def parseCsvList(csvList: String): Seq[String] = { - if(csvList == null || csvList.isEmpty) - Seq.empty[String] - else { - csvList.split("\\s*,\\s*").filter(v => !v.equals("")) - } - } - - /** - * Create an instance of the class with the given class name - */ - def createObject[T<:AnyRef](className: String, args: AnyRef*): T = { - val klass = Class.forName(className).asInstanceOf[Class[T]] - val constructor = klass.getConstructor(args.map(_.getClass): _*) - constructor.newInstance(args: _*).asInstanceOf[T] - } - - /** - * Is the given string null or empty ("")? - */ - def nullOrEmpty(s: String): Boolean = s == null || s.equals("") - - /** - * Create a circular (looping) iterator over a collection. - * @param coll An iterable over the underlying collection. - * @return A circular iterator over the collection. - */ - def circularIterator[T](coll: Iterable[T]) = { - val stream: Stream[T] = - for (forever <- Stream.continually(1); t <- coll) yield t - stream.iterator - } - - /** - * Attempt to read a file as a string - */ - def readFileAsString(path: String, charset: Charset = Charset.defaultCharset()): String = { - val stream = new FileInputStream(new File(path)) - try { - val fc = stream.getChannel() - val bb = fc.map(FileChannel.MapMode.READ_ONLY, 0, fc.size()) - charset.decode(bb).toString() - } - finally { - stream.close() - } - } - - /** - * Get the absolute value of the given number. If the number is Int.MinValue return 0. - * This is different from java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!). - */ - def abs(n: Int) = n & 0x7fffffff - - /** - * 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 old mode 100644 new mode 100755 index b070bb4df117d..34cab87034254 --- a/core/src/main/scala/kafka/utils/VerifiableProperties.scala +++ b/core/src/main/scala/kafka/utils/VerifiableProperties.scala @@ -20,6 +20,8 @@ package kafka.utils import java.util.Properties import java.util.Collections import scala.collection._ +import kafka.message.{CompressionCodec, NoCompressionCodec} + class VerifiableProperties(val props: Properties) extends Logging { private val referenceSet = mutable.HashSet[String]() @@ -33,7 +35,7 @@ class VerifiableProperties(val props: Properties) extends Logging { def getProperty(name: String): String = { val value = props.getProperty(name) referenceSet.add(name) - return value + if(value == null) value else value.trim() } /** @@ -124,14 +126,14 @@ class VerifiableProperties(val props: Properties) extends Logging { * Get a required argument as a double * @param name The property name * @return the value - * @throw IllegalArgumentException If the given property is not present + * @throws 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 + * @param default The default value for the property if not present */ def getDouble(name: String, default: Double): Double = { if(containsKey(name)) @@ -181,7 +183,7 @@ class VerifiableProperties(val props: Properties) extends Logging { */ def getMap(name: String, valid: String => Boolean = s => true): Map[String, String] = { try { - val m = Utils.parseCsvMap(getString(name, "")) + val m = CoreUtils.parseCsvMap(getString(name, "")) m.foreach { case(key, value) => if(!valid(value)) @@ -193,6 +195,24 @@ class VerifiableProperties(val props: Properties) extends Logging { } } + /** + * Parse compression codec from a property list in either. Codecs may be specified as integers, or as strings. + * See [[kafka.message.CompressionCodec]] for more details. + * @param name The property name + * @param default Default compression codec + * @return compression codec + */ + def getCompressionCodec(name: String, default: CompressionCodec) = { + val prop = getString(name, NoCompressionCodec.name) + try { + CompressionCodec.getCompressionCodec(prop.toInt) + } + catch { + case nfe: NumberFormatException => + CompressionCodec.getCompressionCodec(prop) + } + } + def verify() { info("Verifying properties") val propNames = { diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index fa86bb94475de..166814c2959a4 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -17,26 +17,23 @@ package kafka.utils -import kafka.cluster.{Broker, Cluster} -import kafka.consumer.TopicCount -import org.I0Itec.zkclient.{IZkDataListener, ZkClient} -import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, ZkMarshallingError} +import kafka.cluster._ +import kafka.consumer.{ConsumerThreadId, TopicCount} +import org.I0Itec.zkclient.ZkClient +import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, + ZkMarshallingError, ZkBadVersionException} import org.I0Itec.zkclient.serialize.ZkSerializer +import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.protocol.SecurityProtocol import collection._ import kafka.api.LeaderAndIsr -import mutable.ListBuffer import org.apache.zookeeper.data.Stat -import java.util.concurrent.locks.{ReentrantLock, Condition} import kafka.admin._ import kafka.common.{KafkaException, NoEpochForPartitionException} import kafka.controller.ReassignedPartitionsContext -import kafka.controller.PartitionAndReplica import kafka.controller.KafkaController -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" @@ -49,6 +46,8 @@ object ZkUtils extends Logging { val ReassignPartitionsPath = "/admin/reassign_partitions" val DeleteTopicsPath = "/admin/delete_topics" val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" + val BrokerSequenceIdPath = "/brokers/seqid" + val IsrChangeNotificationPath = "/isr_change_notification" def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic @@ -58,7 +57,7 @@ object ZkUtils extends Logging { getTopicPath(topic) + "/partitions" } - def getTopicConfigPath(topic: String): String = + def getTopicConfigPath(topic: String): String = TopicConfigPath + "/" + topic def getDeleteTopicPath(topic: String): String = @@ -85,43 +84,20 @@ object ZkUtils extends Logging { brokerIds.map(_.toInt).map(getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) } - def getLeaderIsrAndEpochForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = { - val leaderAndIsrPath = getTopicPartitionLeaderAndIsrPath(topic, partition) - val leaderAndIsrInfo = readDataMaybeNull(zkClient, leaderAndIsrPath) - val leaderAndIsrOpt = leaderAndIsrInfo._1 - val stat = leaderAndIsrInfo._2 - leaderAndIsrOpt match { - case Some(leaderAndIsrStr) => parseLeaderAndIsr(leaderAndIsrStr, topic, partition, stat) - case None => None - } + def getAllBrokerEndPointsForChannel(zkClient: ZkClient, protocolType: SecurityProtocol): Seq[BrokerEndPoint] = { + getAllBrokersInCluster(zkClient).map(_.getBrokerEndPoint(protocolType)) } def getLeaderAndIsrForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderAndIsr] = { - getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr) + ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr) } - + def setupCommonPaths(zkClient: ZkClient) { - for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath)) + for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath, + DeleteTopicsPath, BrokerSequenceIdPath)) makeSurePersistentPathExists(zkClient, path) } - def parseLeaderAndIsr(leaderAndIsrStr: String, topic: String, partition: Int, stat: Stat) - : Option[LeaderIsrAndControllerEpoch] = { - Json.parseFull(leaderAndIsrStr) match { - case Some(m) => - val leaderIsrAndEpochInfo = m.asInstanceOf[Map[String, Any]] - val leader = leaderIsrAndEpochInfo.get("leader").get.asInstanceOf[Int] - val epoch = leaderIsrAndEpochInfo.get("leader_epoch").get.asInstanceOf[Int] - val isr = leaderIsrAndEpochInfo.get("isr").get.asInstanceOf[List[Int]] - val controllerEpoch = leaderIsrAndEpochInfo.get("controller_epoch").get.asInstanceOf[Int] - val zkPathVersion = stat.getVersion - debug("Leader %d, Epoch %d, Isr %s, Zk path version %d for partition [%s,%d]".format(leader, epoch, - isr.toString(), zkPathVersion, topic, partition)) - Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch)) - case None => None - } - } - def getLeaderForPartition(zkClient: ZkClient, topic: String, partition: Int): Option[Int] = { val leaderAndIsrOpt = readDataMaybeNull(zkClient, getTopicPartitionLeaderAndIsrPath(topic, partition))._1 leaderAndIsrOpt match { @@ -153,6 +129,14 @@ object ZkUtils extends Logging { } } + /** returns a sequence id generated by updating BrokerSequenceIdPath in Zk. + * users can provide brokerId in the config , inorder to avoid conflicts between zk generated + * seqId and config.brokerId we increment zk seqId by KafkaConfig.MaxReservedBrokerId. + */ + def getBrokerSequenceId(zkClient: ZkClient, MaxReservedBrokerId: Int): Int = { + getSequenceId(zkClient, BrokerSequenceIdPath) + MaxReservedBrokerId + } + /** * Gets the in-sync replicas (ISR) for a specific topic and partition */ @@ -189,12 +173,28 @@ object ZkUtils extends Logging { } } - def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, timeout: Int, jmxPort: Int) { + /** + * Register brokers with v2 json format (which includes multiple endpoints). + * This format also includes default endpoints for compatibility with older clients. + * @param zkClient + * @param id + * @param advertisedEndpoints + * @param timeout + * @param jmxPort + */ + def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, advertisedEndpoints: immutable.Map[SecurityProtocol, EndPoint], timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) - val expectedBroker = new Broker(id, host, port) + val brokerInfo = Json.encode(Map("version" -> 2, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.values.map(_.connectionString).toArray, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val expectedBroker = new Broker(id, advertisedEndpoints) + + registerBrokerInZk(zkClient, brokerIdPath, brokerInfo, expectedBroker, timeout) + + info("Registered broker %d at path %s with addresses: %s".format(id, brokerIdPath, advertisedEndpoints.mkString(","))) + } + + private def registerBrokerInZk(zkClient: ZkClient, brokerIdPath: String, brokerInfo: String, expectedBroker: Broker, timeout: Int) { try { createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, (brokerString: String, broker: Any) => Broker.createBroker(broker.asInstanceOf[Broker].id, brokerString).equals(broker.asInstanceOf[Broker]), @@ -203,11 +203,10 @@ object ZkUtils extends Logging { } catch { case e: ZkNodeExistsException => throw new RuntimeException("A broker is already registered on the path " + brokerIdPath - + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " - + "else you have shutdown this broker and restarted it faster than the zookeeper " - + "timeout so it appears to be re-registering.") + + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " + + "else you have shutdown this broker and restarted it faster than the zookeeper " + + "timeout so it appears to be re-registering.") } - info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, host, port)) } def getConsumerPartitionOwnerPath(group: String, topic: String, partition: Int): String = { @@ -233,7 +232,7 @@ object ZkUtils extends Logging { */ def makeSurePersistentPathExists(client: ZkClient, path: String) { if (!client.exists(path)) - client.createPersistent(path, true) // won't throw NoNodeException or NodeExistsException + ZkPath.createPersistent(client, path, true) //won't throw NoNodeException or NodeExistsException } /** @@ -241,8 +240,9 @@ object ZkUtils extends Logging { */ private def createParentPath(client: ZkClient, path: String): Unit = { val parentDir = path.substring(0, path.lastIndexOf('/')) - if (parentDir.length != 0) - client.createPersistent(parentDir, true) + if (parentDir.length != 0) { + ZkPath.createPersistent(client, parentDir, true) + } } /** @@ -250,11 +250,11 @@ object ZkUtils extends Logging { */ private def createEphemeralPath(client: ZkClient, path: String, data: String): Unit = { try { - client.createEphemeral(path, data) + ZkPath.createEphemeral(client, path, data) } catch { case e: ZkNoNodeException => { createParentPath(client, path) - client.createEphemeral(path, data) + ZkPath.createEphemeral(client, path, data) } } } @@ -334,17 +334,17 @@ object ZkUtils extends Logging { */ def createPersistentPath(client: ZkClient, path: String, data: String = ""): Unit = { try { - client.createPersistent(path, data) + ZkPath.createPersistent(client, path, data) } catch { case e: ZkNoNodeException => { createParentPath(client, path) - client.createPersistent(path, data) + ZkPath.createPersistent(client, path, data) } } } def createSequentialPersistentPath(client: ZkClient, path: String, data: String = ""): String = { - client.createPersistentSequential(path, data) + ZkPath.createPersistentSequential(client, path, data) } /** @@ -359,7 +359,7 @@ object ZkUtils extends Logging { case e: ZkNoNodeException => { createParentPath(client, path) try { - client.createPersistent(path, data) + ZkPath.createPersistent(client, path, data) } catch { case e: ZkNodeExistsException => client.writeData(path, data) @@ -373,17 +373,30 @@ object ZkUtils extends Logging { /** * Conditional update the persistent path data, return (true, newVersion) if it succeeds, otherwise (the path doesn't * exist, the current version is not the expected version, etc.) return (false, -1) + * + * When there is a ConnectionLossException during the conditional update, zkClient will retry the update and may fail + * since the previous update may have succeeded (but the stored zkVersion no longer matches the expected one). + * In this case, we will run the optionalChecker to further check if the previous write did indeed succeeded. */ - def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = { + def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int, + optionalChecker:Option[(ZkClient, String, String) => (Boolean,Int)] = None): (Boolean, Int) = { try { val stat = client.writeDataReturnStat(path, data, expectVersion) debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d" .format(path, data, expectVersion, stat.getVersion)) (true, stat.getVersion) } catch { - case e: Exception => - error("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, - expectVersion, e.getMessage)) + case e1: ZkBadVersionException => + optionalChecker match { + case Some(checker) => return checker(client, path, data) + case _ => debug("Checker method is not passed skipping zkData match") + } + warn("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, + expectVersion, e1.getMessage)) + (false, -1) + case e2: Exception => + warn("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, + expectVersion, e2.getMessage)) (false, -1) } } @@ -417,12 +430,12 @@ object ZkUtils extends Logging { } catch { case e: ZkNoNodeException => { createParentPath(client, path) - client.createEphemeral(path, data) + ZkPath.createEphemeral(client, path, data) } case e2: Throwable => throw e2 } } - + def deletePath(client: ZkClient, path: String): Boolean = { try { client.delete(path) @@ -445,10 +458,10 @@ object ZkUtils extends Logging { case e2: Throwable => throw e2 } } - + def maybeDeletePath(zkUrl: String, dir: String) { try { - val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) + val zk = createZkClient(zkUrl, 30*1000, 30*1000) zk.deleteRecursive(dir) zk.close() } catch { @@ -486,7 +499,7 @@ object ZkUtils extends Logging { try { client.getChildren(path) } catch { - case e: ZkNoNodeException => return Nil + case e: ZkNoNodeException => Nil case e2: Throwable => throw e2 } } @@ -512,7 +525,7 @@ object ZkUtils extends Logging { : mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = { val ret = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch] for(topicAndPartition <- topicAndPartitions) { - ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition) match { + ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition) match { case Some(leaderIsrAndControllerEpoch) => ret.put(topicAndPartition, leaderIsrAndControllerEpoch) case None => } @@ -587,23 +600,28 @@ object ZkUtils extends Logging { } } - def parsePartitionReassignmentData(jsonData: String): Map[TopicAndPartition, Seq[Int]] = { - val reassignedPartitions: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map() + // Parses without deduplicating keys so the the data can be checked before allowing reassignment to proceed + def parsePartitionReassignmentDataWithoutDedup(jsonData: String): Seq[(TopicAndPartition, Seq[Int])] = { Json.parseFull(jsonData) match { case Some(m) => m.asInstanceOf[Map[String, Any]].get("partitions") match { case Some(partitionsSeq) => - partitionsSeq.asInstanceOf[Seq[Map[String, Any]]].foreach(p => { + partitionsSeq.asInstanceOf[Seq[Map[String, Any]]].map(p => { val topic = p.get("topic").get.asInstanceOf[String] val partition = p.get("partition").get.asInstanceOf[Int] val newReplicas = p.get("replicas").get.asInstanceOf[Seq[Int]] - reassignedPartitions += TopicAndPartition(topic, partition) -> newReplicas + TopicAndPartition(topic, partition) -> newReplicas }) case None => + Seq.empty } case None => + Seq.empty } - reassignedPartitions + } + + def parsePartitionReassignmentData(jsonData: String): Map[TopicAndPartition, Seq[Int]] = { + parsePartitionReassignmentDataWithoutDedup(jsonData).toMap } def parseTopicsData(jsonData: String): Seq[String] = { @@ -639,7 +657,7 @@ object ZkUtils extends Logging { val jsonData = getPartitionReassignmentZkData(partitionsToBeReassigned) try { updatePersistentPath(zkClient, zkPath, jsonData) - info("Updated partition reassignment path with %s".format(jsonData)) + debug("Updated partition reassignment path with %s".format(jsonData)) } catch { case nne: ZkNoNodeException => ZkUtils.createPersistentPath(zkClient, zkPath, jsonData) @@ -670,12 +688,12 @@ object ZkUtils extends Logging { getChildren(zkClient, dirs.consumerRegistryDir) } - def getConsumersPerTopic(zkClient: ZkClient, group: String) : mutable.Map[String, List[String]] = { + def getConsumersPerTopic(zkClient: ZkClient, group: String, excludeInternalTopics: Boolean) : mutable.Map[String, List[ConsumerThreadId]] = { val dirs = new ZKGroupDirs(group) val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir) - val consumersPerTopicMap = new mutable.HashMap[String, List[String]] + val consumersPerTopicMap = new mutable.HashMap[String, List[ConsumerThreadId]] for (consumer <- consumers) { - val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient) + val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient, excludeInternalTopics) for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic) { for (consumerThreadId <- consumerThreadIdSet) consumersPerTopicMap.get(topic) match { @@ -703,6 +721,30 @@ object ZkUtils extends Logging { } } + /** + * This API produces a sequence number by creating / updating given path in zookeeper + * It uses the stat returned by the zookeeper and return the version. Every time + * client updates the path stat.version gets incremented + */ + def getSequenceId(client: ZkClient, path: String): Int = { + try { + val stat = client.writeDataReturnStat(path, "", -1) + stat.getVersion + } catch { + case e: ZkNoNodeException => { + createParentPath(client, BrokerSequenceIdPath) + try { + client.createPersistent(BrokerSequenceIdPath, "") + 0 + } catch { + case e: ZkNodeExistsException => + val stat = client.writeDataReturnStat(BrokerSequenceIdPath, "", -1) + stat.getVersion + } + } + } + } + def getAllTopics(zkClient: ZkClient): Seq[String] = { val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) if(topics == null) @@ -720,44 +762,34 @@ object ZkUtils extends Logging { }.flatten.toSet } } -} -class LeaderExistsOrChangedListener(topic: String, - partition: Int, - leaderLock: ReentrantLock, - leaderExistsOrChanged: Condition, - oldLeaderOpt: Option[Int] = None, - zkClient: ZkClient = null) extends IZkDataListener with Logging { - @throws(classOf[Exception]) - def handleDataChange(dataPath: String, data: Object) { - val t = dataPath.split("/").takeRight(3).head - val p = dataPath.split("/").takeRight(2).head.toInt - 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)) - leaderExistsOrChanged.signal() - } - else { - val newLeaderOpt = ZkUtils.getLeaderForPartition(zkClient, t, p) - if(newLeaderOpt.isDefined && newLeaderOpt.get != oldLeaderOpt.get){ - trace("In leader change listener on partition [%s, %d], leader has been moved from %d to %d".format(topic, partition, oldLeaderOpt.get, newLeaderOpt.get)) - leaderExistsOrChanged.signal() - } - } + def getConsumerGroups(zkClient: ZkClient) = { + ZkUtils.getChildren(zkClient, ConsumersPath) + } + + def getTopicsByConsumerGroup(zkClient: ZkClient,consumerGroup:String) = { + ZkUtils.getChildrenParentMayNotExist(zkClient, new ZKGroupDirs(consumerGroup).consumerGroupOwnersDir) + } + + def getAllConsumerGroupsForTopic(zkClient: ZkClient, topic: String): Set[String] = { + val groups = ZkUtils.getChildrenParentMayNotExist(zkClient, ConsumersPath) + if (groups == null) Set.empty + else { + groups.foldLeft(Set.empty[String]) {(consumerGroupsForTopic, group) => + val topics = getChildren(zkClient, new ZKGroupDirs(group).consumerGroupOffsetsDir) + if (topics.contains(topic)) consumerGroupsForTopic + group + else consumerGroupsForTopic } } } - @throws(classOf[Exception]) - def handleDataDeleted(dataPath: String) { - inLock(leaderLock) { - leaderExistsOrChanged.signal() - } + def createZkClient(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int): ZkClient = { + val zkClient = new ZkClient(zkUrl, sessionTimeout, connectionTimeout, ZKStringSerializer) + zkClient } } -object ZKStringSerializer extends ZkSerializer { +private object ZKStringSerializer extends ZkSerializer { @throws(classOf[ZkMarshallingError]) def serialize(data : Object) : Array[Byte] = data.asInstanceOf[String].getBytes("UTF-8") @@ -775,11 +807,13 @@ class ZKGroupDirs(val group: String) { def consumerDir = ZkUtils.ConsumersPath def consumerGroupDir = consumerDir + "/" + group def consumerRegistryDir = consumerGroupDir + "/ids" + def consumerGroupOffsetsDir = consumerGroupDir + "/offsets" + def consumerGroupOwnersDir = consumerGroupDir + "/owners" } class ZKGroupTopicDirs(group: String, topic: String) extends ZKGroupDirs(group) { - def consumerOffsetDir = consumerGroupDir + "/offsets/" + topic - def consumerOwnerDir = consumerGroupDir + "/owners/" + topic + def consumerOffsetDir = consumerGroupOffsetsDir + "/" + topic + def consumerOwnerDir = consumerGroupOwnersDir + "/" + topic } @@ -796,3 +830,41 @@ class ZKConfig(props: VerifiableProperties) { /** how far a ZK follower can be behind a ZK leader */ val zkSyncTimeMs = props.getInt("zookeeper.sync.time.ms", 2000) } + +object ZkPath { + @volatile private var isNamespacePresent: Boolean = false + + def checkNamespace(client: ZkClient) { + if(isNamespacePresent) + return + + if (!client.exists("/")) { + throw new ConfigException("Zookeeper namespace does not exist") + } + isNamespacePresent = true + } + + def resetNamespaceCheckedState { + isNamespacePresent = false + } + + def createPersistent(client: ZkClient, path: String, data: Object) { + checkNamespace(client) + client.createPersistent(path, data) + } + + def createPersistent(client: ZkClient, path: String, createParents: Boolean) { + checkNamespace(client) + client.createPersistent(path, createParents) + } + + def createEphemeral(client: ZkClient, path: String, data: Object) { + checkNamespace(client) + client.createEphemeral(path, data) + } + + def createPersistentSequential(client: ZkClient, path: String, data: Object): String = { + checkNamespace(client) + client.createPersistentSequential(path, data) + } +} diff --git a/core/src/main/scala/kafka/utils/timer/Timer.scala b/core/src/main/scala/kafka/utils/timer/Timer.scala new file mode 100644 index 0000000000000..bdd0e75b93e4b --- /dev/null +++ b/core/src/main/scala/kafka/utils/timer/Timer.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.utils.timer + +import java.util.concurrent.{DelayQueue, ExecutorService, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.locks.ReentrantReadWriteLock + +import kafka.utils.threadsafe + +@threadsafe +class Timer(taskExecutor: ExecutorService, tickMs: Long = 1, wheelSize: Int = 20, startMs: Long = System.currentTimeMillis) { + + private[this] val delayQueue = new DelayQueue[TimerTaskList]() + private[this] val taskCounter = new AtomicInteger(0) + private[this] val timingWheel = new TimingWheel( + tickMs = tickMs, + wheelSize = wheelSize, + startMs = startMs, + taskCounter = taskCounter, + delayQueue + ) + + // Locks used to protect data structures while ticking + private[this] val readWriteLock = new ReentrantReadWriteLock() + private[this] val readLock = readWriteLock.readLock() + private[this] val writeLock = readWriteLock.writeLock() + + def add(timerTask: TimerTask): Unit = { + readLock.lock() + try { + addTimerTaskEntry(new TimerTaskEntry(timerTask)) + } finally { + readLock.unlock() + } + } + + private def addTimerTaskEntry(timerTaskEntry: TimerTaskEntry): Unit = { + if (!timingWheel.add(timerTaskEntry)) { + // Already expired or cancelled + if (!timerTaskEntry.cancelled) + taskExecutor.submit(timerTaskEntry.timerTask) + } + } + + private[this] val reinsert = (timerTaskEntry: TimerTaskEntry) => addTimerTaskEntry(timerTaskEntry) + + /* + * Advances the clock if there is an expired bucket. If there isn't any expired bucket when called, + * waits up to timeoutMs before giving up. + */ + def advanceClock(timeoutMs: Long): Boolean = { + var bucket = delayQueue.poll(timeoutMs, TimeUnit.MILLISECONDS) + if (bucket != null) { + writeLock.lock() + try { + while (bucket != null) { + timingWheel.advanceClock(bucket.getExpiration()) + bucket.flush(reinsert) + bucket = delayQueue.poll() + } + } finally { + writeLock.unlock() + } + true + } else { + false + } + } + + def size(): Int = taskCounter.get +} + diff --git a/core/src/main/scala/kafka/utils/timer/TimerTask.scala b/core/src/main/scala/kafka/utils/timer/TimerTask.scala new file mode 100644 index 0000000000000..d6b3a2eb3f70a --- /dev/null +++ b/core/src/main/scala/kafka/utils/timer/TimerTask.scala @@ -0,0 +1,47 @@ +/** + * 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.timer + +trait TimerTask extends Runnable { + + val expirationMs: Long // timestamp in millisecond + + private[this] var timerTaskEntry: TimerTaskEntry = null + + def cancel(): Unit = { + synchronized { + if (timerTaskEntry != null) timerTaskEntry.remove() + timerTaskEntry = null + } + } + + private[timer] def setTimerTaskEntry(entry: TimerTaskEntry): Unit = { + synchronized { + // if this timerTask is already held by an existing timer task entry, + // we will remove such an entry first. + if (timerTaskEntry != null && timerTaskEntry != entry) + timerTaskEntry.remove() + + timerTaskEntry = entry + } + } + + private[timer] def getTimerTaskEntry(): TimerTaskEntry = { + timerTaskEntry + } + +} diff --git a/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala new file mode 100644 index 0000000000000..c4aeb5d20f1e3 --- /dev/null +++ b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala @@ -0,0 +1,161 @@ +/** + * 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.timer + +import java.util.concurrent.{TimeUnit, Delayed} +import java.util.concurrent.atomic.{AtomicLong, AtomicInteger} + +import kafka.utils.{SystemTime, threadsafe} + +import scala.math._ + +@threadsafe +private[timer] class TimerTaskList(taskCounter: AtomicInteger) extends Delayed { + + // TimerTaskList forms a doubly linked cyclic list using a dummy root entry + // root.next points to the head + // root.prev points to the tail + private[this] val root = new TimerTaskEntry(null) + root.next = root + root.prev = root + + private[this] val expiration = new AtomicLong(-1L) + + // Set the bucket's expiration time + // Returns true if the expiration time is changed + def setExpiration(expirationMs: Long): Boolean = { + expiration.getAndSet(expirationMs) != expirationMs + } + + // Get the bucket's expiration time + def getExpiration(): Long = { + expiration.get() + } + + // Apply the supplied function to each of tasks in this list + def foreach(f: (TimerTask)=>Unit): Unit = { + synchronized { + var entry = root.next + while (entry ne root) { + val nextEntry = entry.next + + if (!entry.cancelled) f(entry.timerTask) + + entry = nextEntry + } + } + } + + // Add a timer task entry to this list + def add(timerTaskEntry: TimerTaskEntry): Unit = { + var done = false + while (!done) { + // Remove the timer task entry if it is already in any other list + // We do this outside of the sync block below to avoid deadlocking. + // We may retry until timerTaskEntry.list becomes null. + timerTaskEntry.remove() + + synchronized { + timerTaskEntry.synchronized { + if (timerTaskEntry.list == null) { + // put the timer task entry to the end of the list. (root.prev points to the tail entry) + val tail = root.prev + timerTaskEntry.next = root + timerTaskEntry.prev = tail + timerTaskEntry.list = this + tail.next = timerTaskEntry + root.prev = timerTaskEntry + taskCounter.incrementAndGet() + done = true + } + } + } + } + } + + // Remove the specified timer task entry from this list + def remove(timerTaskEntry: TimerTaskEntry): Unit = { + synchronized { + timerTaskEntry.synchronized { + if (timerTaskEntry.list eq this) { + timerTaskEntry.next.prev = timerTaskEntry.prev + timerTaskEntry.prev.next = timerTaskEntry.next + timerTaskEntry.next = null + timerTaskEntry.prev = null + timerTaskEntry.list = null + taskCounter.decrementAndGet() + } + } + } + } + + // Remove all task entries and apply the supplied function to each of them + def flush(f: (TimerTaskEntry)=>Unit): Unit = { + synchronized { + var head = root.next + while (head ne root) { + remove(head) + f(head) + head = root.next + } + expiration.set(-1L) + } + } + + def getDelay(unit: TimeUnit): Long = { + unit.convert(max(getExpiration - SystemTime.milliseconds, 0), TimeUnit.MILLISECONDS) + } + + def compareTo(d: Delayed): Int = { + + val other = d.asInstanceOf[TimerTaskList] + + if(getExpiration < other.getExpiration) -1 + else if(getExpiration > other.getExpiration) 1 + else 0 + } + +} + +private[timer] class TimerTaskEntry(val timerTask: TimerTask) { + + @volatile + var list: TimerTaskList = null + var next: TimerTaskEntry = null + var prev: TimerTaskEntry = null + + // if this timerTask is already held by an existing timer task entry, + // setTimerTaskEntry will remove it. + if (timerTask != null) timerTask.setTimerTaskEntry(this) + + def cancelled: Boolean = { + timerTask.getTimerTaskEntry != this + } + + def remove(): Unit = { + var currentList = list + // If remove is called when another thread is moving the entry from a task entry list to another, + // this may fail to remove the entry due to the change of value of list. Thus, we retry until the list becomes null. + // In a rare case, this thread sees null and exits the loop, but the other thread insert the entry to another list later. + while (currentList != null) { + currentList.remove(this) + currentList = list + } + } + +} + diff --git a/core/src/main/scala/kafka/utils/timer/TimingWheel.scala b/core/src/main/scala/kafka/utils/timer/TimingWheel.scala new file mode 100644 index 0000000000000..f5b6efe29349a --- /dev/null +++ b/core/src/main/scala/kafka/utils/timer/TimingWheel.scala @@ -0,0 +1,166 @@ +/** + * 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.timer + +import kafka.utils.nonthreadsafe + +import java.util.concurrent.DelayQueue +import java.util.concurrent.atomic.AtomicInteger + +/* + * Hierarchical Timing Wheels + * + * A simple timing wheel is a circular list of buckets of timer tasks. Let u be the time unit. + * A timing wheel with size n has n buckets and can hold timer tasks in n * u time interval. + * Each bucket holds timer tasks that fall into the corresponding time range. At the beginning, + * the first bucket holds tasks for [0, u), the second bucket holds tasks for [u, 2u), …, + * the n-th bucket for [u * (n -1), u * n). Every interval of time unit u, the timer ticks and + * moved to the next bucket then expire all timer tasks in it. So, the timer never insert a task + * into the bucket for the current time since it is already expired. The timer immediately runs + * the expired task. The emptied bucket is then available for the next round, so if the current + * bucket is for the time t, it becomes the bucket for [t + u * n, t + (n + 1) * u) after a tick. + * A timing wheel has O(1) cost for insert/delete (start-timer/stop-timer) whereas priority queue + * based timers, such as java.util.concurrent.DelayQueue and java.util.Timer, have O(log n) + * insert/delete cost. + * + * A major drawback of a simple timing wheel is that it assumes that a timer request is within + * the time interval of n * u from the current time. If a timer request is out of this interval, + * it is an overflow. A hierarchical timing wheel deals with such overflows. It is a hierarchically + * organized timing wheels. The lowest level has the finest time resolution. As moving up the + * hierarchy, time resolutions become coarser. If the resolution of a wheel at one level is u and + * the size is n, the resolution of the next level should be n * u. At each level overflows are + * delegated to the wheel in one level higher. When the wheel in the higher level ticks, it reinsert + * timer tasks to the lower level. An overflow wheel can be created on-demand. When a bucket in an + * overflow bucket expires, all tasks in it are reinserted into the timer recursively. The tasks + * are then moved to the finer grain wheels or be executed. The insert (start-timer) cost is O(m) + * where m is the number of wheels, which is usually very small compared to the number of requests + * in the system, and the delete (stop-timer) cost is still O(1). + * + * Example + * Let's say that u is 1 and n is 3. If the start time is c, + * then the buckets at different levels are: + * + * level buckets + * 1 [c,c] [c+1,c+1] [c+2,c+2] + * 2 [c,c+2] [c+3,c+5] [c+6,c+8] + * 3 [c,c+8] [c+9,c+17] [c+18,c+26] + * + * The bucket expiration is at the time of bucket beginning. + * So at time = c+1, buckets [c,c], [c,c+2] and [c,c+8] are expired. + * Level 1's clock moves to c+1, and [c+3,c+3] is created. + * Level 2 and level3's clock stay at c since their clocks move in unit of 3 and 9, respectively. + * So, no new buckets are created in level 2 and 3. + * + * Note that bucket [c,c+2] in level 2 won't receive any task since that range is already covered in level 1. + * The same is true for the bucket [c,c+8] in level 3 since its range is covered in level 2. + * This is a bit wasteful, but simplifies the implementation. + * + * 1 [c+1,c+1] [c+2,c+2] [c+3,c+3] + * 2 [c,c+2] [c+3,c+5] [c+6,c+8] + * 3 [c,c+8] [c+9,c+17] [c+18,c+26] + * + * At time = c+2, [c+1,c+1] is newly expired. + * Level 1 moves to c+2, and [c+4,c+4] is created, + * + * 1 [c+2,c+2] [c+3,c+3] [c+4,c+4] + * 2 [c,c+2] [c+3,c+5] [c+6,c+8] + * 3 [c,c+8] [c+9,c+17] [c+18,c+18] + * + * At time = c+3, [c+2,c+2] is newly expired. + * Level 2 moves to c+3, and [c+5,c+5] and [c+9,c+11] are created. + * Level 3 stay at c. + * + * 1 [c+3,c+3] [c+4,c+4] [c+5,c+5] + * 2 [c+3,c+5] [c+6,c+8] [c+9,c+11] + * 3 [c,c+8] [c+9,c+17] [c+8,c+11] + * + * The hierarchical timing wheels works especially well when operations are completed before they time out. + * Even when everything times out, it still has advantageous when there are many items in the timer. + * Its insert cost (including reinsert) and delete cost are O(m) and O(1), respectively while priority + * queue based timers takes O(log N) for both insert and delete where N is the number of items in the queue. + * + * This class is not thread-safe. There should not be any add calls while advanceClock is executing. + * It is caller's responsibility to enforce it. Simultaneous add calls are thread-safe. + */ +@nonthreadsafe +private[timer] class TimingWheel(tickMs: Long, wheelSize: Int, startMs: Long, taskCounter: AtomicInteger, queue: DelayQueue[TimerTaskList]) { + + private[this] val interval = tickMs * wheelSize + private[this] val buckets = Array.tabulate[TimerTaskList](wheelSize) { _ => new TimerTaskList(taskCounter) } + + private[this] var currentTime = startMs - (startMs % tickMs) // rounding down to multiple of tickMs + + // overflowWheel can potentially be updated and read by two concurrent threads through add(). + // Therefore, it needs to be volatile due to the issue of Double-Checked Locking pattern with JVM + @volatile private[this] var overflowWheel: TimingWheel = null + + private[this] def addOverflowWheel(): Unit = { + synchronized { + if (overflowWheel == null) { + overflowWheel = new TimingWheel( + tickMs = interval, + wheelSize = wheelSize, + startMs = currentTime, + taskCounter = taskCounter, + queue + ) + } + } + } + + def add(timerTaskEntry: TimerTaskEntry): Boolean = { + val expiration = timerTaskEntry.timerTask.expirationMs + + if (timerTaskEntry.cancelled) { + // Cancelled + false + } else if (expiration < currentTime + tickMs) { + // Already expired + false + } else if (expiration < currentTime + interval) { + // Put in its own bucket + val virtualId = expiration / tickMs + val bucket = buckets((virtualId % wheelSize.toLong).toInt) + bucket.add(timerTaskEntry) + + // Set the bucket expiration time + if (bucket.setExpiration(virtualId * tickMs)) { + // The bucket needs to be enqueued because it was an expired bucket + // We only need to enqueue the bucket when its expiration time has changed, i.e. the wheel has advanced + // and the previous buckets gets reused; further calls to set the expiration within the same wheel cycle + // will pass in the same value and hence return false, thus the bucket with the same expiration will not + // be enqueued multiple times. + queue.offer(bucket) + } + true + } else { + // Out of the interval. Put it into the parent timer + if (overflowWheel == null) addOverflowWheel() + overflowWheel.add(timerTaskEntry) + } + } + + // Try to advance the clock + def advanceClock(timeMs: Long): Unit = { + if (timeMs >= currentTime + tickMs) { + currentTime = timeMs - (timeMs % tickMs) + + // Try to advance the clock of the overflow wheel if present + if (overflowWheel != null) overflowWheel.advanceClock(currentTime) + } + } +} diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index d7d03ea1b1329..1b7d5d8f7d5fa 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -19,6 +19,7 @@ log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n log4j.logger.kafka=ERROR +log4j.logger.org.apache.kafka=ERROR # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala new file mode 100644 index 0000000000000..b0750faa43dfe --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -0,0 +1,152 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package kafka.api + +import kafka.server.KafkaConfig +import kafka.utils.{Logging, ShutdownableThread, TestUtils} +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} +import org.apache.kafka.common.TopicPartition +import org.junit.Assert._ + +import scala.collection.JavaConversions._ + +/** + * Integration tests for the new consumer that cover basic usage as well as server failures + */ +class ConsumerBounceTest extends IntegrationTestHarness with Logging { + + val producerCount = 1 + val consumerCount = 2 + val serverCount = 3 + + val topic = "topic" + val part = 0 + val tp = new TopicPartition(topic, part) + + // configure the servers and clients + this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset + this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") + this.serverConfig.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, "10") // set small enough session timeout + this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "20") + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + + override def generateConfigs() = { + FixedPortTestUtils.createBrokerConfigs(serverCount, zkConnect,enableControlledShutdown = false) + .map(KafkaConfig.fromProps(_, serverConfig)) + } + + override def setUp() { + super.setUp() + + // create the test topic with all the brokers as replicas + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + } + + def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(20) + + /* + * 1. Produce a bunch of messages + * 2. Then consume the messages while killing and restarting brokers at random + */ + def consumeWithBrokerFailures(numIters: Int) { + val numRecords = 1000 + sendRecords(numRecords) + this.producers.foreach(_.close) + + var consumed = 0 + val consumer = this.consumers(0) + consumer.subscribe(topic) + + val scheduler = new BounceBrokerScheduler(numIters) + scheduler.start() + + while (scheduler.isRunning.get()) { + for (record <- consumer.poll(100)) { + assertEquals(consumed.toLong, record.offset()) + consumed += 1 + } + + consumer.commit(CommitType.SYNC) + assertEquals(consumer.position(tp), consumer.committed(tp)) + + if (consumer.position(tp) == numRecords) { + consumer.seekToBeginning() + consumed = 0 + } + } + scheduler.shutdown() + } + + def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) + + def seekAndCommitWithBrokerFailures(numIters: Int) { + val numRecords = 1000 + sendRecords(numRecords) + this.producers.foreach(_.close) + + val consumer = this.consumers(0) + consumer.subscribe(tp) + consumer.seek(tp, 0) + + val scheduler = new BounceBrokerScheduler(numIters) + scheduler.start() + + while(scheduler.isRunning.get()) { + val coin = TestUtils.random.nextInt(3) + if (coin == 0) { + info("Seeking to end of log") + consumer.seekToEnd() + assertEquals(numRecords.toLong, consumer.position(tp)) + } else if (coin == 1) { + val pos = TestUtils.random.nextInt(numRecords).toLong + info("Seeking to " + pos) + consumer.seek(tp, pos) + assertEquals(pos, consumer.position(tp)) + } else if (coin == 2) { + info("Committing offset.") + consumer.commit(CommitType.SYNC) + assertEquals(consumer.position(tp), consumer.committed(tp)) + } + } + } + + private class BounceBrokerScheduler(val numIters: Int) extends ShutdownableThread("daemon-bounce-broker", false) + { + var iter: Int = 0 + + override def doWork(): Unit = { + killRandomBroker() + Thread.sleep(500) + restartDeadBrokers() + + iter += 1 + if (iter == numIters) + initiateShutdown() + else + Thread.sleep(500) + } + } + + private def sendRecords(numRecords: Int) { + val futures = (0 until numRecords).map { i => + this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) + } + futures.map(_.get) + } +} diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala new file mode 100644 index 0000000000000..3eb5f95731a3f --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -0,0 +1,278 @@ +/** + * 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.{lang, util} + +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import org.apache.kafka.common.TopicPartition + +import kafka.utils.{TestUtils, Logging} +import kafka.server.KafkaConfig + +import java.util.ArrayList +import org.junit.Assert._ + +import scala.collection.JavaConversions._ +import kafka.coordinator.ConsumerCoordinator + + +/** + * Integration tests for the new consumer that cover basic usage as well as server failures + */ +class ConsumerTest extends IntegrationTestHarness with Logging { + + val producerCount = 1 + val consumerCount = 2 + val serverCount = 3 + + val topic = "topic" + val part = 0 + val tp = new TopicPartition(topic, part) + val part2 = 1 + val tp2 = new TopicPartition(topic, part2) + + // configure the servers and clients + this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset + this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") + this.serverConfig.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, "100") // set small enough session timeout + this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + override def setUp() { + super.setUp() + + // create the test topic with all the brokers as replicas + TestUtils.createTopic(this.zkClient, topic, 2, serverCount, this.servers) + } + + def testSimpleConsumption() { + val numRecords = 10000 + sendRecords(numRecords) + + assertEquals(0, this.consumers(0).subscriptions.size) + this.consumers(0).subscribe(tp) + assertEquals(1, this.consumers(0).subscriptions.size) + + this.consumers(0).seek(tp, 0) + consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) + + // check async commit callbacks + val commitCallback = new CountConsumerCommitCallback() + this.consumers(0).commit(CommitType.ASYNC, commitCallback) + + // shouldn't make progress until poll is invoked + Thread.sleep(10) + assertEquals(0, commitCallback.count) + awaitCommitCallback(this.consumers(0), commitCallback) + } + + def testCommitSpecifiedOffsets() { + sendRecords(5, tp) + sendRecords(7, tp2) + + this.consumers(0).subscribe(tp) + this.consumers(0).subscribe(tp2) + + // Need to poll to join the group + this.consumers(0).poll(50) + val pos1 = this.consumers(0).position(tp) + val pos2 = this.consumers(0).position(tp2) + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp, 3L)), CommitType.SYNC) + assertEquals(3, this.consumers(0).committed(tp)) + intercept[NoOffsetForPartitionException] { + this.consumers(0).committed(tp2) + } + // positions should not change + assertEquals(pos1, this.consumers(0).position(tp)) + assertEquals(pos2, this.consumers(0).position(tp2)) + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 5L)), CommitType.SYNC) + assertEquals(3, this.consumers(0).committed(tp)) + assertEquals(5, this.consumers(0).committed(tp2)) + + // Using async should pick up the committed changes after commit completes + val commitCallback = new CountConsumerCommitCallback() + this.consumers(0).commit(Map[TopicPartition,java.lang.Long]((tp2, 7L)), CommitType.ASYNC, commitCallback) + awaitCommitCallback(this.consumers(0), commitCallback) + assertEquals(7, this.consumers(0).committed(tp2)) + } + + def testAutoOffsetReset() { + sendRecords(1) + this.consumers(0).subscribe(tp) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + def testSeek() { + val consumer = this.consumers(0) + val totalRecords = 50L + sendRecords(totalRecords.toInt) + consumer.subscribe(tp) + + consumer.seekToEnd(tp) + assertEquals(totalRecords, consumer.position(tp)) + assertFalse(consumer.poll(totalRecords).iterator().hasNext) + + consumer.seekToBeginning(tp) + assertEquals(0, consumer.position(tp), 0) + consumeRecords(consumer, numRecords = 1, startingOffset = 0) + + val mid = totalRecords / 2 + consumer.seek(tp, mid) + assertEquals(mid, consumer.position(tp)) + consumeRecords(consumer, numRecords = 1, startingOffset = mid.toInt) + } + + def testGroupConsumption() { + sendRecords(10) + this.consumers(0).subscribe(topic) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + def testPositionAndCommit() { + sendRecords(5) + + // committed() on a partition with no committed offset throws an exception + intercept[NoOffsetForPartitionException] { + this.consumers(0).committed(new TopicPartition(topic, 15)) + } + + // position() on a partition that we aren't subscribed to throws an exception + intercept[IllegalArgumentException] { + this.consumers(0).position(new TopicPartition(topic, 15)) + } + + this.consumers(0).subscribe(tp) + + assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, this.consumers(0).position(tp)) + this.consumers(0).commit(CommitType.SYNC) + assertEquals(0L, this.consumers(0).committed(tp)) + + consumeRecords(this.consumers(0), 5, 0) + assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp)) + this.consumers(0).commit(CommitType.SYNC) + assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp)) + + sendRecords(1) + + // another consumer in the same group should get the same position + this.consumers(1).subscribe(tp) + consumeRecords(this.consumers(1), 1, 5) + } + + def testPartitionsFor() { + val numParts = 2 + TestUtils.createTopic(this.zkClient, "part-test", numParts, 1, this.servers) + val parts = this.consumers(0).partitionsFor("part-test") + assertNotNull(parts) + assertEquals(2, parts.length) + assertNull(this.consumers(0).partitionsFor("non-exist-topic")) + } + + def testPartitionReassignmentCallback() { + val callback = new TestConsumerReassignmentCallback() + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100"); // timeout quickly to avoid slow test + val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumer0.subscribe(topic) + + // the initial subscription should cause a callback execution + while(callback.callsToAssigned == 0) + consumer0.poll(50) + + // get metadata for the topic + var parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName) + while(parts == null) + parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName) + assertEquals(1, parts.size) + assertNotNull(parts(0).leader()) + + // shutdown the coordinator + val coordinator = parts(0).leader().id() + this.servers(coordinator).shutdown() + + // this should cause another callback execution + while(callback.callsToAssigned < 2) + consumer0.poll(50) + + assertEquals(2, callback.callsToAssigned) + assertEquals(2, callback.callsToRevoked) + + consumer0.close() + } + + private class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback { + var callsToAssigned = 0 + var callsToRevoked = 0 + def onPartitionsAssigned(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { + info("onPartitionsAssigned called.") + callsToAssigned += 1 + } + def onPartitionsRevoked(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { + info("onPartitionsRevoked called.") + callsToRevoked += 1 + } + } + + private def sendRecords(numRecords: Int): Unit = { + sendRecords(numRecords, tp) + } + + private def sendRecords(numRecords: Int, tp: TopicPartition) { + val futures = (0 until numRecords).map { i => + this.producers(0).send(new ProducerRecord(tp.topic(), tp.partition(), i.toString.getBytes, i.toString.getBytes)) + } + futures.map(_.get) + } + + private def consumeRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int) { + val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]() + val maxIters = numRecords * 300 + var iters = 0 + while (records.size < numRecords) { + for (record <- consumer.poll(50)) + records.add(record) + if(iters > maxIters) + throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations.") + iters += 1 + } + for (i <- 0 until numRecords) { + val record = records.get(i) + val offset = startingOffset + i + assertEquals(topic, record.topic()) + assertEquals(part, record.partition()) + assertEquals(offset.toLong, record.offset()) + } + } + + private def awaitCommitCallback(consumer: Consumer[Array[Byte], Array[Byte]], commitCallback: CountConsumerCommitCallback): Unit = { + val startCount = commitCallback.count + val started = System.currentTimeMillis() + while (commitCallback.count == startCount && System.currentTimeMillis() - started < 10000) + this.consumers(0).poll(10000) + assertEquals(startCount + 1, commitCallback.count) + } + + private class CountConsumerCommitCallback extends ConsumerCommitCallback { + var count = 0 + + override def onComplete(offsets: util.Map[TopicPartition, lang.Long], exception: Exception): Unit = count += 1 + } + +} \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala b/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala new file mode 100644 index 0000000000000..b26b24243b91f --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.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.api + +import java.io.IOException +import java.net.ServerSocket +import java.util.Properties + +import kafka.utils.TestUtils + +/** + * DO NOT USE THESE UTILITIES UNLESS YOU ABSOLUTELY MUST + * + * These are utilities for selecting fixed (preselected), ephemeral ports to use with tests. This is not a reliable way + * of testing on most machines because you can easily run into port conflicts. If you're using this class, you're almost + * certainly doing something wrong unless you can prove that your test **cannot function** properly without it. + */ +object FixedPortTestUtils { + def choosePorts(count: Int): Seq[Int] = { + try { + val sockets = (0 until count).map(i => new ServerSocket(0)) + val ports = sockets.map(_.getLocalPort()) + sockets.foreach(_.close()) + ports + } catch { + case e: IOException => { + throw new RuntimeException(e) + } + } + } + + def createBrokerConfigs(numConfigs: Int, + zkConnect: String, + enableControlledShutdown: Boolean = true, + enableDeleteTopic: Boolean = false): Seq[Properties] = { + val ports = FixedPortTestUtils.choosePorts(numConfigs) + (0 until numConfigs) + .map(node => TestUtils.createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, ports(node))) + } + +} diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala new file mode 100644 index 0000000000000..afcc349342d44 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -0,0 +1,80 @@ +/** + * 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 org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.consumer.ConsumerConfig +import kafka.utils.TestUtils +import java.util.Properties +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer.KafkaProducer +import kafka.server.{OffsetManager, KafkaConfig} +import kafka.integration.KafkaServerTestHarness +import scala.collection.mutable.Buffer +import kafka.coordinator.ConsumerCoordinator + +/** + * A helper class for writing integration tests that involve producers, consumers, and servers + */ +trait IntegrationTestHarness extends KafkaServerTestHarness { + + val producerCount: Int + val consumerCount: Int + val serverCount: Int + lazy val producerConfig = new Properties + lazy val consumerConfig = new Properties + lazy val serverConfig = new Properties + + var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + var producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() + + override def generateConfigs() = { + val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect) + cfgs.foreach(_.putAll(serverConfig)) + cfgs.map(KafkaConfig.fromProps) + } + + override def setUp() { + super.setUp() + producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) + producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) + consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + consumerConfig.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range") + for(i <- 0 until producerCount) + producers += new KafkaProducer(producerConfig) + for(i <- 0 until consumerCount) + consumers += new KafkaConsumer(consumerConfig) + + // create the consumer offset topic + TestUtils.createTopic(zkClient, ConsumerCoordinator.OffsetsTopicName, + serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, + serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt, + servers, + servers(0).consumerCoordinator.offsetsTopicConfigs) + } + + override def tearDown() { + producers.foreach(_.close()) + consumers.foreach(_.close()) + super.tearDown() + } + +} diff --git a/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala new file mode 100644 index 0000000000000..ce70a0a449883 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala @@ -0,0 +1,161 @@ +/** + * 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.util.Properties + +import kafka.consumer.SimpleConsumer +import kafka.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import kafka.utils.{ShutdownableThread, TestUtils} +import org.apache.kafka.clients.producer._ +import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback +import org.junit.Assert._ +import org.junit.Test + +class ProducerBounceTest extends KafkaServerTestHarness { + private val producerBufferSize = 30000 + private val serverMessageMaxBytes = producerBufferSize/2 + + val numServers = 2 + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) + overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) + // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) + // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long + overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, 1.toString) + + // This is the one of the few tests we currently allow to preallocate ports, despite the fact that this can result in transient + // failures due to ports getting reused. We can't use random ports because of bad behavior that can result from bouncing + // brokers too quickly when they get new, random ports. If we're not careful, the client can end up in a situation + // where metadata is not refreshed quickly enough, and by the time it's actually trying to, all the servers have + // been bounced and have new addresses. None of the bootstrap nodes or current metadata can get them connected to a + // running server. + // + // Since such quick rotation of servers is incredibly unrealistic, we allow this one test to preallocate ports, leaving + // a small risk of hitting errors due to port conflicts. Hopefully this is infrequent enough to not cause problems. + override def generateConfigs() = { + FixedPortTestUtils.createBrokerConfigs(numServers, zkConnect,enableControlledShutdown = false) + .map(KafkaConfig.fromProps(_, overridingProps)) + } + + private var consumer1: SimpleConsumer = null + private var consumer2: SimpleConsumer = null + + private var producer1: KafkaProducer[Array[Byte],Array[Byte]] = null + private var producer2: KafkaProducer[Array[Byte],Array[Byte]] = null + private var producer3: KafkaProducer[Array[Byte],Array[Byte]] = null + private var producer4: KafkaProducer[Array[Byte],Array[Byte]] = null + + private val topic1 = "topic-1" + private val topic2 = "topic-2" + + override def setUp() { + super.setUp() + + producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize) + producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) + producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = producerBufferSize) + } + + override def tearDown() { + if (producer1 != null) producer1.close + if (producer2 != null) producer2.close + if (producer3 != null) producer3.close + if (producer4 != null) producer4.close + + super.tearDown() + } + + /** + * With replication, producer should able able to find new leader after it detects broker failure + */ + @Test + def testBrokerFailure() { + val numPartitions = 3 + val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) + assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined)) + + val scheduler = new ProducerScheduler() + scheduler.start + + // rolling bounce brokers + for (i <- 0 until numServers) { + for (server <- servers) { + server.shutdown() + server.awaitShutdown() + server.startup() + Thread.sleep(2000) + } + + // Make sure the producer do not see any exception + // in returned metadata due to broker failures + assertTrue(scheduler.failed == false) + + // Make sure the leader still exists after bouncing brokers + (0 until numPartitions).foreach(partition => TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partition)) + } + + scheduler.shutdown + + // Make sure the producer do not see any exception + // when draining the left messages on shutdown + assertTrue(scheduler.failed == false) + + // double check that the leader info has been propagated after consecutive bounces + val newLeaders = (0 until numPartitions).map(i => TestUtils.waitUntilMetadataIsPropagated(servers, topic1, i)) + val fetchResponses = newLeaders.zipWithIndex.map { case (leader, partition) => + // Consumers must be instantiated after all the restarts since they use random ports each time they start up + val consumer = new SimpleConsumer("localhost", servers(leader).boundPort(), 100, 1024 * 1024, "") + val response = consumer.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition) + consumer.close + response + } + val messages = fetchResponses.flatMap(r => r.iterator.toList.map(_.message)) + val uniqueMessages = messages.toSet + val uniqueMessageSize = uniqueMessages.size + + assertEquals("Should have fetched " + scheduler.sent + " unique messages", scheduler.sent, uniqueMessageSize) + } + + private class ProducerScheduler extends ShutdownableThread("daemon-producer", false) + { + val numRecords = 1000 + var sent = 0 + var failed = false + + val producer = TestUtils.createNewProducer(brokerList, bufferSize = producerBufferSize, retries = 10) + + override def doWork(): Unit = { + val responses = + for (i <- sent+1 to sent+numRecords) + yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, null, i.toString.getBytes), + new ErrorLoggingCallback(topic1, null, null, true)) + val futures = responses.toList + + try { + futures.map(_.get) + sent += numRecords + } catch { + case e : Exception => failed = true + } + } + + override def shutdown(){ + super.shutdown() + producer.close + } + } +} diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala new file mode 100755 index 0000000000000..83de81cb3f79a --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -0,0 +1,132 @@ +/** + * 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.test + +import java.util.{Properties, Collection, ArrayList} + +import org.scalatest.junit.JUnit3Suite +import org.junit.runners.Parameterized +import org.junit.runner.RunWith +import org.junit.runners.Parameterized.Parameters +import org.junit.{After, Before, Test} +import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer, ProducerConfig} +import org.junit.Assert._ + +import kafka.api.FetchRequestBuilder +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.consumer.SimpleConsumer +import kafka.message.Message +import kafka.zk.ZooKeeperTestHarness +import kafka.utils.{CoreUtils, TestUtils} + + +@RunWith(value = classOf[Parameterized]) +class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooKeeperTestHarness { + private val brokerId = 0 + private var server: KafkaServer = null + + private val topic = "topic" + private val numRecords = 2000 + + @Before + override def setUp() { + super.setUp() + + val props = TestUtils.createBrokerConfig(brokerId, zkConnect) + val config = KafkaConfig.fromProps(props) + + server = TestUtils.createServer(config) + } + + @After + override def tearDown() { + server.shutdown + CoreUtils.rm(server.config.logDirs) + super.tearDown() + } + + /** + * testCompression + * + * Compressed messages should be able to sent and consumed correctly + */ + @Test + def testCompression() { + + val props = new Properties() + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(Seq(server))) + props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compression) + props.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000") + props.put(ProducerConfig.LINGER_MS_CONFIG, "200") + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + var producer = new KafkaProducer[Array[Byte],Array[Byte]](props) + val consumer = new SimpleConsumer("localhost", server.boundPort(), 100, 1024*1024, "") + + try { + // create topic + TestUtils.createTopic(zkClient, topic, 1, 1, List(server)) + val partition = 0 + + // prepare the messages + val messages = for (i <-0 until numRecords) + yield ("value" + i).getBytes + + // make sure the returned messages are correct + val responses = for (message <- messages) + yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, null, null, message)) + val futures = responses.toList + for ((future, offset) <- futures zip (0 until numRecords)) { + assertEquals(offset.toLong, future.get.offset) + } + + // make sure the fetched message count match + val fetchResponse = consumer.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build()) + val messageSet = fetchResponse.messageSet(topic, partition).iterator.toBuffer + assertEquals("Should have fetched " + numRecords + " messages", numRecords, messageSet.size) + + var index = 0 + for (message <- messages) { + assertEquals(new Message(bytes = message), messageSet(index).message) + assertEquals(index.toLong, messageSet(index).offset) + index += 1 + } + } finally { + if (producer != null) { + producer.close() + producer = null + } + if (consumer != null) + consumer.close() + } + } +} + +object ProducerCompressionTest { + + // NOTE: Must return collection of Array[AnyRef] (NOT Array[Any]). + @Parameters + def parameters: Collection[Array[String]] = { + val list = new ArrayList[Array[String]]() + list.add(Array("none")) + list.add(Array("gzip")) + list.add(Array("snappy")) + list.add(Array("lz4")) + list + } +} diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala new file mode 100644 index 0000000000000..ee94011894b46 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -0,0 +1,328 @@ +/** + * 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 org.junit.Test +import org.junit.Assert._ + +import java.util.{Properties, Random} +import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} + +import kafka.common.Topic +import kafka.consumer.SimpleConsumer +import kafka.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import kafka.utils.{ShutdownableThread, TestUtils} + +import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasException, NotEnoughReplicasAfterAppendException} +import org.apache.kafka.clients.producer._ +import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback + +class ProducerFailureHandlingTest extends KafkaServerTestHarness { + private val producerBufferSize = 30000 + private val serverMessageMaxBytes = producerBufferSize/2 + + val numServers = 2 + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) + overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) + // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) + // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long + overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, 1.toString) + + def generateConfigs() = + TestUtils.createBrokerConfigs(numServers, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps)) + + private var consumer1: SimpleConsumer = null + private var consumer2: SimpleConsumer = null + + private var producer1: KafkaProducer[Array[Byte],Array[Byte]] = null + private var producer2: KafkaProducer[Array[Byte],Array[Byte]] = null + private var producer3: KafkaProducer[Array[Byte],Array[Byte]] = null + private var producer4: KafkaProducer[Array[Byte],Array[Byte]] = null + + private val topic1 = "topic-1" + private val topic2 = "topic-2" + + override def setUp() { + super.setUp() + + producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize) + producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) + producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = producerBufferSize) + } + + override def tearDown() { + if (producer1 != null) producer1.close + if (producer2 != null) producer2.close + if (producer3 != null) producer3.close + if (producer4 != null) producer4.close + + super.tearDown() + } + + /** + * With ack == 0 the future metadata will have no exceptions with offset -1 + */ + @Test + def testTooLargeRecordWithAckZero() { + // create topic + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) + + // send a too-large record + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) + assertEquals("Returned metadata should have offset -1", producer1.send(record).get.offset, -1L) + } + + /** + * With ack == 1 the future metadata will throw ExecutionException caused by RecordTooLargeException + */ + @Test + def testTooLargeRecordWithAckOne() { + // create topic + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) + + // send a too-large record + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) + intercept[ExecutionException] { + producer2.send(record).get + } + } + + /** + * With non-exist-topic the future metadata should return ExecutionException caused by TimeoutException + */ + @Test + def testNonExistentTopic() { + // send a record with non-exist topic + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic2, null, "key".getBytes, "value".getBytes) + intercept[ExecutionException] { + producer1.send(record).get + } + } + + /** + * With incorrect broker-list the future metadata should return ExecutionException caused by TimeoutException + * + * TODO: other exceptions that can be thrown in ExecutionException: + * UnknownTopicOrPartitionException + * NotLeaderForPartitionException + * LeaderNotAvailableException + * CorruptRecordException + * TimeoutException + */ + @Test + def testWrongBrokerList() { + // create topic + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) + + // producer with incorrect broker list + producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) + + // send a record with incorrect broker list + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) + intercept[ExecutionException] { + producer4.send(record).get + } + } + + /** + * 1. With ack=0, the future metadata should not be blocked. + * 2. With ack=1, the future metadata should block, + * and subsequent calls will eventually cause buffer full + */ + @Test + def testNoResponse() { + // create topic + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) + + // first send a message to make sure the metadata is refreshed + val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) + producer1.send(record1).get + producer2.send(record1).get + + // stop IO threads and request handling, but leave networking operational + // any requests should be accepted and queue up, but not handled + servers.foreach(server => server.requestHandlerPool.shutdown()) + + producer1.send(record1).get(5000, TimeUnit.MILLISECONDS) + + intercept[TimeoutException] { + producer2.send(record1).get(5000, TimeUnit.MILLISECONDS) + } + + // TODO: expose producer configs after creating them + // send enough messages to get buffer full + val tooManyRecords = 10 + val msgSize = producerBufferSize / tooManyRecords + val value = new Array[Byte](msgSize) + new Random().nextBytes(value) + val record2 = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, value) + + intercept[KafkaException] { + for (i <- 1 to tooManyRecords) + producer2.send(record2) + } + + // do not close produce2 since it will block + // TODO: can we do better? + producer2 = null + } + + /** + * The send call with invalid partition id should throw KafkaException caused by IllegalArgumentException + */ + @Test + def testInvalidPartition() { + // create topic + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) + + // create a record with incorrect partition id, send should fail + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, new Integer(1), "key".getBytes, "value".getBytes) + intercept[IllegalArgumentException] { + producer1.send(record) + } + intercept[IllegalArgumentException] { + producer2.send(record) + } + intercept[IllegalArgumentException] { + producer3.send(record) + } + } + + /** + * The send call after producer closed should throw KafkaException cased by IllegalStateException + */ + @Test + def testSendAfterClosed() { + // create topic + TestUtils.createTopic(zkClient, topic1, 1, numServers, servers) + + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) + + // first send a message to make sure the metadata is refreshed + producer1.send(record).get + producer2.send(record).get + producer3.send(record).get + + intercept[IllegalStateException] { + producer1.close + producer1.send(record) + } + intercept[IllegalStateException] { + producer2.close + producer2.send(record) + } + intercept[IllegalStateException] { + producer3.close + producer3.send(record) + } + + // re-close producer is fine + } + + @Test + def testCannotSendToInternalTopic() { + val thrown = intercept[ExecutionException] { + producer2.send(new ProducerRecord[Array[Byte],Array[Byte]](Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get + } + assertTrue("Unexpected exception while sending to an invalid topic " + thrown.getCause, thrown.getCause.isInstanceOf[InvalidTopicException]) + } + + @Test + def testNotEnoughReplicas() { + val topicName = "minisrtest" + val topicProps = new Properties() + topicProps.put("min.insync.replicas",(numServers+1).toString) + + TestUtils.createTopic(zkClient, topicName, 1, numServers, servers, topicProps) + + val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) + try { + producer3.send(record).get + fail("Expected exception when producing to topic with fewer brokers than min.insync.replicas") + } catch { + case e: ExecutionException => + if (!e.getCause.isInstanceOf[NotEnoughReplicasException]) { + fail("Expected NotEnoughReplicasException when producing to topic with fewer brokers than min.insync.replicas") + } + } + } + + @Test + def testNotEnoughReplicasAfterBrokerShutdown() { + val topicName = "minisrtest2" + val topicProps = new Properties() + topicProps.put("min.insync.replicas",numServers.toString) + + TestUtils.createTopic(zkClient, topicName, 1, numServers, servers,topicProps) + + val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) + // this should work with all brokers up and running + producer3.send(record).get + + // shut down one broker + servers.head.shutdown() + servers.head.awaitShutdown() + try { + producer3.send(record).get + fail("Expected exception when producing to topic with fewer brokers than min.insync.replicas") + } catch { + case e: ExecutionException => + if (!e.getCause.isInstanceOf[NotEnoughReplicasException] && + !e.getCause.isInstanceOf[NotEnoughReplicasAfterAppendException]) { + fail("Expected NotEnoughReplicasException or NotEnoughReplicasAfterAppendException when producing to topic " + + "with fewer brokers than min.insync.replicas, but saw " + e.getCause) + } + } + + // restart the server + servers.head.startup() + } + + private class ProducerScheduler extends ShutdownableThread("daemon-producer", false) + { + val numRecords = 1000 + var sent = 0 + var failed = false + + val producer = TestUtils.createNewProducer(brokerList, bufferSize = producerBufferSize, retries = 10) + + override def doWork(): Unit = { + val responses = + for (i <- sent+1 to sent+numRecords) + yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, null, i.toString.getBytes), + new ErrorLoggingCallback(topic1, null, null, true)) + val futures = responses.toList + + try { + futures.map(_.get) + sent += numRecords + } catch { + case e : Exception => failed = true + } + } + + override def shutdown(){ + super.shutdown() + producer.close + } + } +} diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala new file mode 100644 index 0000000000000..9ce4bd5ee130c --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -0,0 +1,427 @@ +/** + * 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.util.Properties +import java.util.concurrent.TimeUnit + +import kafka.consumer.SimpleConsumer +import kafka.integration.KafkaServerTestHarness +import kafka.message.Message +import kafka.server.KafkaConfig +import kafka.utils.TestUtils +import org.apache.kafka.clients.producer._ +import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.errors.SerializationException +import org.apache.kafka.common.serialization.ByteArraySerializer +import org.junit.Assert._ +import org.junit.Test +import org.scalatest.junit.JUnit3Suite + + +class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { + val numServers = 2 + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) + + def generateConfigs() = + TestUtils.createBrokerConfigs(numServers, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps)) + + private var consumer1: SimpleConsumer = null + private var consumer2: SimpleConsumer = null + + private val topic = "topic" + private val numRecords = 100 + + override def setUp() { + super.setUp() + + // TODO: we need to migrate to new consumers when 0.9 is final + consumer1 = new SimpleConsumer("localhost", servers(0).boundPort(), 100, 1024*1024, "") + consumer2 = new SimpleConsumer("localhost", servers(1).boundPort(), 100, 1024*1024, "") + } + + override def tearDown() { + consumer1.close() + consumer2.close() + + super.tearDown() + } + + /** + * testSendOffset checks the basic send API behavior + * + * 1. Send with null key/value/partition-id should be accepted; send with null topic should be rejected. + * 2. Last message of the non-blocking send should return the correct offset metadata + */ + @Test + def testSendOffset() { + var producer = TestUtils.createNewProducer(brokerList) + val partition = new Integer(0) + + object callback extends Callback { + var offset = 0L + def onCompletion(metadata: RecordMetadata, exception: Exception) { + if (exception == null) { + assertEquals(offset, metadata.offset()) + assertEquals(topic, metadata.topic()) + assertEquals(partition, metadata.partition()) + offset += 1 + } else { + fail("Send callback returns the following exception", exception) + } + } + } + + try { + // create topic + TestUtils.createTopic(zkClient, topic, 1, 2, servers) + + // send a normal record + val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, "value".getBytes) + assertEquals("Should have offset 0", 0L, producer.send(record0, callback).get.offset) + + // send a record with null value should be ok + val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, null) + assertEquals("Should have offset 1", 1L, producer.send(record1, callback).get.offset) + + // send a record with null key should be ok + val record2 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, "value".getBytes) + assertEquals("Should have offset 2", 2L, producer.send(record2, callback).get.offset) + + // send a record with null part id should be ok + val record3 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes) + assertEquals("Should have offset 3", 3L, producer.send(record3, callback).get.offset) + + // send a record with null topic should fail + try { + val record4 = new ProducerRecord[Array[Byte],Array[Byte]](null, partition, "key".getBytes, "value".getBytes) + producer.send(record4, callback) + fail("Should not allow sending a record without topic") + } catch { + case iae: IllegalArgumentException => // this is ok + case e: Throwable => fail("Only expecting IllegalArgumentException", e) + } + + // non-blocking send a list of records + for (i <- 1 to numRecords) + producer.send(record0, callback) + + // check that all messages have been acked via offset + assertEquals("Should have offset " + (numRecords + 4), numRecords + 4L, producer.send(record0, callback).get.offset) + + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + } + + @Test + def testSerializer() { + // send a record with a wrong type should receive a serialization exception + try { + val producer = createNewProducerWithWrongSerializer(brokerList) + val record5 = new ProducerRecord[Array[Byte],Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes) + producer.send(record5) + fail("Should have gotten a SerializationException") + } catch { + case se: SerializationException => // this is ok + } + + try { + createNewProducerWithNoSerializer(brokerList) + fail("Instantiating a producer without specifying a serializer should cause a ConfigException") + } catch { + case ce : ConfigException => // this is ok + } + + // create a producer with explicit serializers should succeed + createNewProducerWithExplicitSerializer(brokerList) + } + + private def createNewProducerWithWrongSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = { + import org.apache.kafka.clients.producer.ProducerConfig + + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") + return new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + } + + private def createNewProducerWithNoSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = { + import org.apache.kafka.clients.producer.ProducerConfig + + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + return new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + } + + private def createNewProducerWithExplicitSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = { + import org.apache.kafka.clients.producer.ProducerConfig + + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + return new KafkaProducer[Array[Byte],Array[Byte]](producerProps, new ByteArraySerializer, new ByteArraySerializer) + } + + /** + * testClose checks the closing behavior + * + * After close() returns, all messages should be sent with correct returned offset metadata + */ + @Test + def testClose() { + var producer = TestUtils.createNewProducer(brokerList) + + try { + // create topic + TestUtils.createTopic(zkClient, topic, 1, 2, servers) + + // non-blocking send a list of records + val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes) + for (i <- 1 to numRecords) + producer.send(record0) + val response0 = producer.send(record0) + + // close the producer + producer.close() + producer = null + + // check that all messages have been acked via offset, + // this also checks that messages with same key go to the same partition + assertTrue("The last message should be acked before producer is shutdown", response0.isDone) + assertEquals("Should have offset " + numRecords, numRecords.toLong, response0.get.offset) + + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + } + + /** + * testSendToPartition checks the partitioning behavior + * + * The specified partition-id should be respected + */ + @Test + def testSendToPartition() { + var producer = TestUtils.createNewProducer(brokerList) + + try { + // create topic + val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers) + val partition = 1 + + // make sure leaders exist + val leader1 = leaders(partition) + assertTrue("Leader for topic \"topic\" partition 1 should exist", leader1.isDefined) + + val responses = + for (i <- 1 to numRecords) + yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, ("value" + i).getBytes)) + val futures = responses.toList + futures.map(_.get) + for (future <- futures) + assertTrue("Request should have completed", future.isDone) + + // make sure all of them end up in the same partition with increasing offset values + for ((future, offset) <- futures zip (0 until numRecords)) { + assertEquals(offset.toLong, future.get.offset) + assertEquals(topic, future.get.topic) + assertEquals(partition, future.get.partition) + } + + // make sure the fetched messages also respect the partitioning and ordering + val fetchResponse1 = if(leader1.get == configs(0).brokerId) { + consumer1.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build()) + } else { + consumer2.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build()) + } + val messageSet1 = fetchResponse1.messageSet(topic, partition).iterator.toBuffer + assertEquals("Should have fetched " + numRecords + " messages", numRecords, messageSet1.size) + + // TODO: also check topic and partition after they are added in the return messageSet + for (i <- 0 to numRecords - 1) { + assertEquals(new Message(bytes = ("value" + (i + 1)).getBytes), messageSet1(i).message) + assertEquals(i.toLong, messageSet1(i).offset) + } + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + } + + /** + * testAutoCreateTopic + * + * The topic should be created upon sending the first message + */ + @Test + def testAutoCreateTopic() { + var producer = TestUtils.createNewProducer(brokerList, retries = 5) + + try { + // Send a message to auto-create the topic + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes) + assertEquals("Should have offset 0", 0L, producer.send(record).get.offset) + + // double check that the topic is created with leader elected + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + } + + /** + * Test that flush immediately sends all accumulated requests. + */ + @Test + def testFlush() { + var producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue) + try { + TestUtils.createTopic(zkClient, topic, 2, 2, servers) + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, "value".getBytes) + for(i <- 0 until 50) { + val responses = (0 until numRecords) map (i => producer.send(record)) + assertTrue("No request is complete.", responses.forall(!_.isDone())) + producer.flush() + assertTrue("All requests are complete.", responses.forall(_.isDone())) + } + } finally { + if (producer != null) + producer.close() + } + } + + /** + * Test close with zero timeout from caller thread + */ + @Test + def testCloseWithZeroTimeoutFromCallerThread() { + var producer: KafkaProducer[Array[Byte],Array[Byte]] = null + try { + // create topic + val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers) + val leader0 = leaders(0) + val leader1 = leaders(1) + + // create record + val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, null, "value".getBytes) + val record1 = new ProducerRecord[Array[Byte], Array[Byte]](topic, 1, null, "value".getBytes) + + // Test closing from caller thread. + for(i <- 0 until 50) { + producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue) + val responses = (0 until numRecords) map (i => producer.send(record0)) + assertTrue("No request is complete.", responses.forall(!_.isDone())) + producer.close(0, TimeUnit.MILLISECONDS) + responses.foreach { future => + try { + future.get() + fail("No message should be sent successfully.") + } catch { + case e: Exception => + assertEquals("java.lang.IllegalStateException: Producer is closed forcefully.", e.getMessage) + } + } + val fetchResponse = if (leader0.get == configs(0).brokerId) { + consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) + } else { + consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) + } + assertEquals("Fetch response should have no message returned.", 0, fetchResponse.messageSet(topic, 0).size) + } + } finally { + if (producer != null) + producer.close() + } + } + + /** + * Test close with zero and non-zero timeout from sender thread + */ + @Test + def testCloseWithZeroTimeoutFromSenderThread() { + var producer: KafkaProducer[Array[Byte],Array[Byte]] = null + try { + // create topic + val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers) + val leader0 = leaders(0) + val leader1 = leaders(1) + + // create record + val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, null, "value".getBytes) + val record1 = new ProducerRecord[Array[Byte], Array[Byte]](topic, 1, null, "value".getBytes) + + // Test closing from sender thread. + class CloseCallback(producer: KafkaProducer[Array[Byte], Array[Byte]]) extends Callback { + override def onCompletion(metadata: RecordMetadata, exception: Exception) { + // Trigger another batch in accumulator before close the producer. These messages should + // not be sent. + (0 until numRecords) map (i => producer.send(record1)) + // The close call will be called by all the message callbacks. This tests idempotence of the close call. + producer.close(0, TimeUnit.MILLISECONDS) + // Test close with non zero timeout. Should not block at all. + producer.close(Long.MaxValue, TimeUnit.MICROSECONDS) + } + } + for(i <- 0 until 50) { + producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue) + // send message to partition 0 + var responses = (0 until numRecords) map (i => producer.send(record0)) + // send message to partition 1 + responses ++= ((0 until numRecords) map (i => producer.send(record1, new CloseCallback(producer)))) + assertTrue("No request is complete.", responses.forall(!_.isDone())) + // flush the messages. + producer.flush() + assertTrue("All request are complete.", responses.forall(_.isDone())) + // Check the messages received by broker. + val fetchResponse0 = if (leader0.get == configs(0).brokerId) { + consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) + } else { + consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) + } + val fetchResponse1 = if (leader1.get == configs(0).brokerId) { + consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 1, 0, Int.MaxValue).build()) + } else { + consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 1, 0, Int.MaxValue).build()) + } + val expectedNumRecords = (i + 1) * numRecords + assertEquals("Fetch response to partition 0 should have %d messages.".format(expectedNumRecords), + expectedNumRecords, fetchResponse0.messageSet(topic, 0).size) + assertEquals("Fetch response to partition 1 should have %d messages.".format(expectedNumRecords), + expectedNumRecords, fetchResponse1.messageSet(topic, 1).size) + } + } finally { + if (producer != null) + producer.close() + } + } +} diff --git a/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala b/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala new file mode 100644 index 0000000000000..6d1f51c866e23 --- /dev/null +++ b/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala @@ -0,0 +1,79 @@ +/* + * 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 kafka.producer +import kafka.tools.ConsoleProducer.{LineMessageReader, MessageReader,ProducerConfig} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig} +import joptsimple.UnrecognizedOptionException +import org.junit.Assert +import org.junit.Test + + +class ConsoleProducerTest { + + val validArgs: Array[String] = Array( + "--broker-list", + "localhost:1001,localhost:1002", + "--topic", + "t3", + "--property", + "parse.key=true", + "--property", + "key.separator=#" + ) + + val invalidArgs: Array[String] = Array( + "--t", // not a valid argument + "t3" + ) + + @Test + def testValidConfigsNewProducer() { + val config = new ConsoleProducer.ProducerConfig(validArgs) + // New ProducerConfig constructor is package private, so we can't call it directly + // Creating new Producer to validate instead + new KafkaProducer[Array[Byte],Array[Byte]](ConsoleProducer.getNewProducerProps(config)) + } + + @Test + def testValidConfigsOldProducer() { + val config = new ConsoleProducer.ProducerConfig(validArgs) + new producer.ProducerConfig(ConsoleProducer.getOldProducerProps(config)); + } + + @Test + def testInvalidConfigs() { + try { + val config = new ConsoleProducer.ProducerConfig(invalidArgs) + Assert.fail("Should have thrown an UnrecognizedOptionException") + } catch { + case e: joptsimple.OptionException => // expected exception + } + } + + @Test + def testParseKeyProp(): Unit = { + val config = new ConsoleProducer.ProducerConfig(validArgs) + val reader = Class.forName(config.readerClass).newInstance().asInstanceOf[LineMessageReader]; + reader.init(System.in,ConsoleProducer.getReaderProps(config)) + assert(reader.keySeparator == "#") + assert(reader.parseKey == true) + } + +} diff --git a/core/src/test/scala/kafka/tools/TestEndToEndLatency.scala b/core/src/test/scala/kafka/tools/TestEndToEndLatency.scala new file mode 100755 index 0000000000000..99b77a1ffbd8b --- /dev/null +++ b/core/src/test/scala/kafka/tools/TestEndToEndLatency.scala @@ -0,0 +1,91 @@ +/** + * 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 org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, KafkaProducer} +import kafka.consumer._ +import java.util.Properties +import java.util.Arrays +import scala.Option.option2Iterable + +object TestEndToEndLatency { + def main(args: Array[String]) { + if (args.length != 6) { + System.err.println("USAGE: java " + getClass().getName + " broker_list zookeeper_connect topic num_messages consumer_fetch_max_wait producer_acks") + System.exit(1) + } + + val brokerList = args(0) + val zkConnect = args(1) + val topic = args(2) + val numMessages = args(3).toInt + val consumerFetchMaxWait = args(4).toInt + val producerAcks = args(5).toInt + + val consumerProps = new Properties() + consumerProps.put("group.id", topic) + consumerProps.put("auto.commit.enable", "false") + consumerProps.put("auto.offset.reset", "largest") + consumerProps.put("zookeeper.connect", zkConnect) + consumerProps.put("fetch.wait.max.ms", consumerFetchMaxWait.toString) + consumerProps.put("socket.timeout.ms", 1201000.toString) + + val config = new ConsumerConfig(consumerProps) + val connector = Consumer.create(config) + val stream = connector.createMessageStreams(Map(topic -> 1)).get(topic).head.head + val iter = stream.iterator + + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0") + producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") + producerProps.put(ProducerConfig.ACKS_CONFIG, producerAcks.toString) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + + // make sure the consumer fetcher has started before sending data since otherwise + // the consumption from the tail will skip the first message and hence be blocked + Thread.sleep(5000) + + val message = "hello there beautiful".getBytes + var totalTime = 0.0 + val latencies = new Array[Long](numMessages) + for (i <- 0 until numMessages) { + val begin = System.nanoTime + producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, message)) + val received = iter.next + val elapsed = System.nanoTime - begin + // poor man's progress bar + if (i % 1000 == 0) + println(i + "\t" + elapsed / 1000.0 / 1000.0) + totalTime += elapsed + latencies(i) = (elapsed / 1000 / 1000) + } + println("Avg latency: %.4f ms\n".format(totalTime / numMessages / 1000.0 / 1000.0)) + Arrays.sort(latencies) + val p50 = latencies((latencies.length * 0.5).toInt) + val p99 = latencies((latencies.length * 0.99).toInt) + val p999 = latencies((latencies.length * 0.999).toInt) + println("Percentiles: 50th = %d, 99th = %d, 99.9th = %d".format(p50, p99, p999)) + producer.close() + connector.commitOffsets(true) + connector.shutdown() + System.exit(0) + } +} \ No newline at end of file diff --git a/core/src/test/scala/other/kafka/TestLogCleaning.scala b/core/src/test/scala/kafka/tools/TestLogCleaning.scala old mode 100644 new mode 100755 similarity index 84% rename from core/src/test/scala/other/kafka/TestLogCleaning.scala rename to core/src/test/scala/kafka/tools/TestLogCleaning.scala index 22b16e54980e3..dcbfbe1c1fc14 --- a/core/src/test/scala/other/kafka/TestLogCleaning.scala +++ b/core/src/test/scala/kafka/tools/TestLogCleaning.scala @@ -15,20 +15,18 @@ * limitations under the License. */ -package kafka +package kafka.tools 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 +import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer, ProducerConfig} /** * This is a torture test that runs against an existing broker. Here is how it works: @@ -54,6 +52,11 @@ object TestLogCleaning { .describedAs("count") .ofType(classOf[java.lang.Long]) .defaultsTo(Long.MaxValue) + val messageCompressionOpt = parser.accepts("compression-type", "message compression type") + .withOptionalArg() + .describedAs("compressionType") + .ofType(classOf[java.lang.String]) + .defaultsTo("none") val numDupsOpt = parser.accepts("duplicates", "The number of duplicates for each key.") .withRequiredArg .describedAs("count") @@ -86,60 +89,61 @@ object TestLogCleaning { .withRequiredArg .describedAs("directory") .ofType(classOf[String]) - + val options = parser.parse(args:_*) - + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(parser, "An integration test for log cleaning.") + 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) - } - + + CommandLineUtils.checkRequiredArgs(parser, options, brokerOpt, zkConnectOpt, numMessagesOpt) + // parse options val messages = options.valueOf(numMessagesOpt).longValue + val compressionType = options.valueOf(messageCompressionOpt) 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) + val producedDataFile = produceMessages(brokerUrl, topics, messages, compressionType, 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...") + println("De-duplicating and validating output files...") validateOutput(producedDataFile, consumedDataFile) producedDataFile.delete() consumedDataFile.delete() } def dumpLog(dir: File) { - require(dir.exists, "Non-existant directory: " + dir.getAbsolutePath) + require(dir.exists, "Non-existent 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 key = TestUtils.readString(entry.message.key) val content = if(entry.message.isNull) null else - Utils.readString(entry.message.payload) + TestUtils.readString(entry.message.payload) println("offset = %s, key = %s, content = %s".format(entry.offset, key, content)) } } @@ -171,9 +175,9 @@ object TestLogCleaning { } producedDeduped.close() consumedDeduped.close() + println("Validated " + total + " values, " + mismatched + " mismatches.") 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() @@ -235,20 +239,20 @@ object TestLogCleaning { }.start() new BufferedReader(new InputStreamReader(process.getInputStream()), 10*1024*1024) } - - def produceMessages(brokerUrl: String, - topics: Array[String], - messages: Long, + + def produceMessages(brokerUrl: String, + topics: Array[String], + messages: Long, + compressionType: String, 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)) + producerProps.setProperty(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") + producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, compressionType) + val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) val rand = new Random(1) val keyCount = (messages / dups).toInt val producedFile = File.createTempFile("kafka-log-cleaner-produced-", ".txt") @@ -260,9 +264,9 @@ object TestLogCleaning { val delete = i % 100 < percentDeletes val msg = if(delete) - new KeyedMessage[String, String](topic = topic, key = key.toString, message = null) + new ProducerRecord[Array[Byte],Array[Byte]](topic, key.toString.getBytes(), null) else - new KeyedMessage[String, String](topic = topic, key = key.toString, message = i.toString) + new ProducerRecord[Array[Byte],Array[Byte]](topic, key.toString.getBytes(), i.toString.getBytes()) producer.send(msg) producedWriter.write(TestRecord(topic, key, i, delete).toString) producedWriter.newLine() @@ -275,8 +279,9 @@ object TestLogCleaning { 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) + consumerProps.setProperty("zookeeper.connect", zkUrl) + consumerProps.setProperty("consumer.timeout.ms", (20*1000).toString) + consumerProps.setProperty("auto.offset.reset", "smallest") new ZookeeperConsumerConnector(new ConsumerConfig(consumerProps)) } diff --git a/core/src/test/scala/other/kafka/DeleteZKPath.scala b/core/src/test/scala/other/kafka/DeleteZKPath.scala old mode 100644 new mode 100755 index 25545037517b1..fb8ab9f01c1ba --- a/core/src/test/scala/other/kafka/DeleteZKPath.scala +++ b/core/src/test/scala/other/kafka/DeleteZKPath.scala @@ -18,8 +18,9 @@ package kafka import consumer.ConsumerConfig -import utils.{ZKStringSerializer, ZkUtils, Utils} +import utils.ZkUtils import org.I0Itec.zkclient.ZkClient +import org.apache.kafka.common.utils.Utils object DeleteZKPath { def main(args: Array[String]) { @@ -31,8 +32,7 @@ object DeleteZKPath { val config = new ConsumerConfig(Utils.loadProps(args(0))) val zkPath = args(1) - val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) try { ZkUtils.deletePathRecursive(zkClient, zkPath); diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala old mode 100644 new mode 100755 index 8fcd068b24868..225d77bca9c09 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -17,6 +17,7 @@ package kafka +import java.util.Properties import java.util.concurrent.atomic._ import kafka.common._ import kafka.message._ @@ -33,10 +34,13 @@ object StressTestLog { def main(args: Array[String]) { val dir = TestUtils.tempDir() val time = new MockTime + val logProprties = new Properties() + logProprties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer) + logProprties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer) + logProprties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer) + val log = new Log(dir = dir, - config = LogConfig(segmentSize = 64*1024*1024, - maxMessageSize = Int.MaxValue, - maxIndexSize = 1024*1024), + config = LogConfig(logProprties), recoveryPoint = 0L, scheduler = time.scheduler, time = time) @@ -50,7 +54,7 @@ object StressTestLog { running.set(false) writer.join() reader.join() - Utils.rm(dir) + CoreUtils.rm(dir) } }) @@ -91,7 +95,7 @@ object StressTestLog { @volatile var offset = 0 override def work() { try { - log.read(offset, 1024, Some(offset+1)) match { + log.read(offset, 1024, Some(offset+1)).messageSet match { case read: FileMessageSet if read.sizeInBytes > 0 => { val first = read.head require(first.offset == offset, "We should either read nothing or the message we asked for.") diff --git a/core/src/test/scala/other/kafka/TestCrcPerformance.scala b/core/src/test/scala/other/kafka/TestCrcPerformance.scala old mode 100644 new mode 100755 index 42e3c62055c15..0c1e1ad924bb9 --- a/core/src/test/scala/other/kafka/TestCrcPerformance.scala +++ b/core/src/test/scala/other/kafka/TestCrcPerformance.scala @@ -18,7 +18,8 @@ package kafka.log import java.util.Random import kafka.message._ -import kafka.utils.{TestUtils, Utils} +import kafka.utils.TestUtils +import org.apache.kafka.common.utils.Utils object TestCrcPerformance { diff --git a/core/src/test/scala/other/kafka/TestEndToEndLatency.scala b/core/src/test/scala/other/kafka/TestEndToEndLatency.scala deleted file mode 100644 index c4aed10f50ca5..0000000000000 --- a/core/src/test/scala/other/kafka/TestEndToEndLatency.scala +++ /dev/null @@ -1,71 +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 - -import java.util.Properties -import kafka.consumer._ -import kafka.producer._ -import kafka.message._ - -object TestEndToEndLatency { - def main(args: Array[String]) { - if(args.length != 3) { - System.err.println("USAGE: java " + getClass().getName + " broker_list zookeeper_connect num_messages") - System.exit(1) - } - - val brokerList = args(0) - val zkConnect = args(1) - val numMessages = args(2).toInt - val topic = "test" - - val consumerProps = new Properties() - consumerProps.put("group.id", topic) - consumerProps.put("auto.commit", "true") - consumerProps.put("auto.offset.reset", "largest") - consumerProps.put("zookeeper.connect", zkConnect) - consumerProps.put("socket.timeout.ms", 1201000.toString) - - val config = new ConsumerConfig(consumerProps) - val connector = Consumer.create(config) - var stream = connector.createMessageStreams(Map(topic -> 1)).get(topic).head.head - val iter = stream.iterator - - val producerProps = new Properties() - producerProps.put("metadata.broker.list", brokerList) - producerProps.put("producer.type", "sync") - val producer = new Producer[Any, Any](new ProducerConfig(producerProps)) - - val message = new Message("hello there beautiful".getBytes) - var totalTime = 0.0 - for(i <- 0 until numMessages) { - var begin = System.nanoTime - producer.send(new KeyedMessage(topic, message)) - val received = iter.next - val elapsed = System.nanoTime - begin - // poor man's progress bar - if(i % 10000 == 0) - println(i + "\t" + elapsed / 1000.0 / 1000.0) - totalTime += elapsed - } - println("Avg latency: " + (totalTime / numMessages / 1000.0 / 1000.0) + "ms") - producer.close() - connector.shutdown() - System.exit(0) - } -} \ No newline at end of file diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala old mode 100644 new mode 100755 index eeb8c8856200c..236d8579d9bce --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -20,7 +20,7 @@ package kafka import java.io._ import java.nio._ import java.nio.channels._ -import java.util.Random +import java.util.{Properties, Random} import kafka.log._ import kafka.utils._ import kafka.message._ @@ -83,13 +83,7 @@ object TestLinearWriteSpeed { val options = parser.parse(args : _*) - for(arg <- List(bytesOpt, sizeOpt, filesOpt)) { - if(!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } + CommandLineUtils.checkRequiredArgs(parser, options, bytesOpt, sizeOpt, filesOpt) var bytesToWrite = options.valueOf(bytesOpt).longValue val bufferSize = options.valueOf(sizeOpt).intValue @@ -116,7 +110,10 @@ object TestLinearWriteSpeed { 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) + val logProperties = new Properties() + logProperties.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProperties.put(LogConfig.FlushMessagesProp, flushInterval: java.lang.Long) + writables(i) = new LogWritable(new File(dir, "kafka-test-" + i), new LogConfig(logProperties), scheduler, messageSet) } else { System.err.println("Must specify what to write to with one of --log, --channel, or --mmap") System.exit(1) @@ -199,7 +196,7 @@ object TestLinearWriteSpeed { } class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: ByteBufferMessageSet) extends Writable { - Utils.rm(dir) + CoreUtils.rm(dir) val log = new Log(dir, config, 0L, scheduler, SystemTime) def write(): Int = { log.append(messages, true) @@ -207,7 +204,7 @@ object TestLinearWriteSpeed { } def close() { log.close() - Utils.rm(log.dir) + CoreUtils.rm(log.dir) } } diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala new file mode 100644 index 0000000000000..e0e46c8838f71 --- /dev/null +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -0,0 +1,308 @@ +/** + * 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 other.kafka + +import org.I0Itec.zkclient.ZkClient +import kafka.api._ +import kafka.utils.{ZkUtils, ShutdownableThread} +import org.apache.kafka.common.protocol.SecurityProtocol +import scala.collection._ +import kafka.client.ClientUtils +import joptsimple.OptionParser +import kafka.common.{ErrorMapping, OffsetAndMetadata, TopicAndPartition} +import kafka.network.BlockingChannel +import scala.util.Random +import java.io.IOException +import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicInteger +import java.nio.channels.ClosedByInterruptException + + +object TestOffsetManager { + + val random = new Random + val SocketTimeoutMs = 10000 + + class StatsThread(reportingIntervalMs: Long, commitThreads: Seq[CommitThread], fetchThread: FetchThread) + extends ShutdownableThread("stats-thread") { + + def printStats() { + println("--------------------------------------------------------------------------------") + println("Aggregate stats for commits:") + println("Error count: %d; Max:%f; Min: %f; Mean: %f; Commit count: %d".format( + commitThreads.map(_.numErrors.get).sum, + commitThreads.map(_.timer.max()).max, + commitThreads.map(_.timer.min()).min, + commitThreads.map(_.timer.mean()).sum / commitThreads.size, + commitThreads.map(_.numCommits.get).sum)) + println("--------------------------------------------------------------------------------") + commitThreads.foreach(t => println(t.stats)) + println(fetchThread.stats) + } + + override def doWork() { + printStats() + Thread.sleep(reportingIntervalMs) + } + + } + + class CommitThread(id: Int, partitionCount: Int, commitIntervalMs: Long, zkClient: ZkClient) + extends ShutdownableThread("commit-thread") + with KafkaMetricsGroup { + + private val groupId = "group-" + id + private val metadata = "Metadata from commit thread " + id + private var offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkClient, SocketTimeoutMs) + private var offset = 0L + val numErrors = new AtomicInteger(0) + val numCommits = new AtomicInteger(0) + val timer = newTimer("commit-thread", TimeUnit.MILLISECONDS, TimeUnit.SECONDS) + private val commitTimer = new KafkaTimer(timer) + val shutdownLock = new Object + + private def ensureConnected() { + if (!offsetsChannel.isConnected) + offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkClient, SocketTimeoutMs) + } + + override def doWork() { + val commitRequest = OffsetCommitRequest(groupId, immutable.Map((1 to partitionCount).map(TopicAndPartition("topic-" + id, _) -> OffsetAndMetadata(offset, metadata)):_*)) + try { + ensureConnected() + offsetsChannel.send(commitRequest) + numCommits.getAndIncrement + commitTimer.time { + val response = OffsetCommitResponse.readFrom(offsetsChannel.receive().payload()) + if (response.commitStatus.exists(_._2 != ErrorMapping.NoError)) numErrors.getAndIncrement + } + offset += 1 + } + catch { + case e1: ClosedByInterruptException => + offsetsChannel.disconnect() + case e2: IOException => + println("Commit thread %d: Error while committing offsets to %s:%d for group %s due to %s.".format(id, offsetsChannel.host, offsetsChannel.port, groupId, e2)) + offsetsChannel.disconnect() + } + finally { + Thread.sleep(commitIntervalMs) + } + } + + override def shutdown() { + super.shutdown() + awaitShutdown() + offsetsChannel.disconnect() + println("Commit thread %d ended. Last committed offset: %d.".format(id, offset)) + } + + def stats = { + "Commit thread %d :: Error count: %d; Max:%f; Min: %f; Mean: %f; Commit count: %d" + .format(id, numErrors.get(), timer.max(), timer.min(), timer.mean(), numCommits.get()) + } + } + + class FetchThread(numGroups: Int, fetchIntervalMs: Long, zkClient: ZkClient) + extends ShutdownableThread("fetch-thread") + with KafkaMetricsGroup { + + private val timer = newTimer("fetch-thread", TimeUnit.MILLISECONDS, TimeUnit.SECONDS) + private val fetchTimer = new KafkaTimer(timer) + + private val channels = mutable.Map[Int, BlockingChannel]() + private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs) + + private val numErrors = new AtomicInteger(0) + + override def doWork() { + val id = random.nextInt().abs % numGroups + val group = "group-" + id + try { + metadataChannel.send(ConsumerMetadataRequest(group)) + val coordinatorId = ConsumerMetadataResponse.readFrom(metadataChannel.receive().payload()).coordinatorOpt.map(_.id).getOrElse(-1) + + val channel = if (channels.contains(coordinatorId)) + channels(coordinatorId) + else { + val newChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs) + channels.put(coordinatorId, newChannel) + newChannel + } + + try { + // send the offset fetch request + val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition("topic-"+id, 1))) + channel.send(fetchRequest) + + fetchTimer.time { + val response = OffsetFetchResponse.readFrom(channel.receive().payload()) + if (response.requestInfo.exists(_._2.error != ErrorMapping.NoError)) { + numErrors.getAndIncrement + } + } + } + catch { + case e1: ClosedByInterruptException => + channel.disconnect() + channels.remove(coordinatorId) + case e2: IOException => + println("Error while fetching offset from %s:%d due to %s.".format(channel.host, channel.port, e2)) + channel.disconnect() + channels.remove(coordinatorId) + } + } + catch { + case e: IOException => + println("Error while querying %s:%d - shutting down query channel.".format(metadataChannel.host, metadataChannel.port)) + metadataChannel.disconnect() + println("Creating new query channel.") + metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs) + } + finally { + Thread.sleep(fetchIntervalMs) + } + + } + + override def shutdown() { + super.shutdown() + awaitShutdown() + channels.foreach(_._2.disconnect()) + metadataChannel.disconnect() + } + + def stats = { + "Fetch thread :: Error count: %d; Max:%f; Min: %f; Mean: %f; Fetch count: %d" + .format(numErrors.get(), timer.max(), timer.min(), timer.mean(), timer.count()) + } + } + + def main(args: Array[String]) { + val parser = new OptionParser + val zookeeperOpt = parser.accepts("zookeeper", "The ZooKeeper connection URL.") + .withRequiredArg + .describedAs("ZooKeeper URL") + .ofType(classOf[java.lang.String]) + .defaultsTo("localhost:2181") + + val commitIntervalOpt = parser.accepts("commit-interval-ms", "Offset commit interval.") + .withRequiredArg + .describedAs("interval") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(100) + + val fetchIntervalOpt = parser.accepts("fetch-interval-ms", "Offset fetch interval.") + .withRequiredArg + .describedAs("interval") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1000) + + val numPartitionsOpt = parser.accepts("partition-count", "Number of partitions per commit.") + .withRequiredArg + .describedAs("interval") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) + + val numThreadsOpt = parser.accepts("thread-count", "Number of commit threads.") + .withRequiredArg + .describedAs("threads") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) + + val reportingIntervalOpt = parser.accepts("reporting-interval-ms", "Interval at which stats are reported.") + .withRequiredArg + .describedAs("interval (ms)") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(3000) + + val helpOpt = parser.accepts("help", "Print this message.") + + val options = parser.parse(args : _*) + + if (options.has(helpOpt)) { + parser.printHelpOn(System.out) + System.exit(0) + } + + val commitIntervalMs = options.valueOf(commitIntervalOpt).intValue() + val fetchIntervalMs = options.valueOf(fetchIntervalOpt).intValue() + val threadCount = options.valueOf(numThreadsOpt).intValue() + val partitionCount = options.valueOf(numPartitionsOpt).intValue() + val zookeeper = options.valueOf(zookeeperOpt) + val reportingIntervalMs = options.valueOf(reportingIntervalOpt).intValue() + println("Commit thread count: %d; Partition count: %d, Commit interval: %d ms; Fetch interval: %d ms; Reporting interval: %d ms" + .format(threadCount, partitionCount, commitIntervalMs, fetchIntervalMs, reportingIntervalMs)) + + var zkClient: ZkClient = null + var commitThreads: Seq[CommitThread] = Seq() + var fetchThread: FetchThread = null + var statsThread: StatsThread = null + try { + zkClient = ZkUtils.createZkClient(zookeeper, 6000, 2000) + commitThreads = (0 to (threadCount-1)).map { threadId => + new CommitThread(threadId, partitionCount, commitIntervalMs, zkClient) + } + + fetchThread = new FetchThread(threadCount, fetchIntervalMs, zkClient) + + val statsThread = new StatsThread(reportingIntervalMs, commitThreads, fetchThread) + + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run() { + cleanShutdown() + statsThread.printStats() + } + }) + + commitThreads.foreach(_.start()) + + fetchThread.start() + + statsThread.start() + + commitThreads.foreach(_.join()) + fetchThread.join() + statsThread.join() + } + catch { + case e: Throwable => + println("Error: ", e) + } + finally { + cleanShutdown() + } + + def cleanShutdown() { + commitThreads.foreach(_.shutdown()) + commitThreads.foreach(_.join()) + if (fetchThread != null) { + fetchThread.shutdown() + fetchThread.join() + } + if (statsThread != null) { + statsThread.shutdown() + statsThread.join() + } + zkClient.close() + } + + } +} + diff --git a/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala new file mode 100644 index 0000000000000..744be3bed0633 --- /dev/null +++ b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala @@ -0,0 +1,292 @@ +/** + * 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 java.lang.management.ManagementFactory +import java.lang.management.OperatingSystemMXBean +import java.util.Random +import java.util.concurrent._ + +import joptsimple._ +import kafka.server.{DelayedOperationPurgatory, DelayedOperation} +import kafka.utils._ + +import scala.math._ +import scala.collection.JavaConversions._ + +/** + * This is a benchmark test of the purgatory. + */ +object TestPurgatoryPerformance { + + def main(args: Array[String]): Unit = { + val parser = new OptionParser + val keySpaceSizeOpt = parser.accepts("key-space-size", "The total number of possible keys") + .withRequiredArg + .describedAs("total_num_possible_keys") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(100) + val numRequestsOpt = parser.accepts("num", "The number of requests") + .withRequiredArg + .describedAs("num_requests") + .ofType(classOf[java.lang.Double]) + val requestRateOpt = parser.accepts("rate", "The request rate per second") + .withRequiredArg + .describedAs("request_per_second") + .ofType(classOf[java.lang.Double]) + val requestDataSizeOpt = parser.accepts("size", "The request data size in bytes") + .withRequiredArg + .describedAs("num_bytes") + .ofType(classOf[java.lang.Long]) + val numKeysOpt = parser.accepts("keys", "The number of keys for each request") + .withRequiredArg + .describedAs("num_keys") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(3) + val timeoutOpt = parser.accepts("timeout", "The request timeout in ms") + .withRequiredArg + .describedAs("timeout_milliseconds") + .ofType(classOf[java.lang.Long]) + val pct75Opt = parser.accepts("pct75", "75th percentile of request latency in ms (log-normal distribution)") + .withRequiredArg + .describedAs("75th_percentile") + .ofType(classOf[java.lang.Double]) + val pct50Opt = parser.accepts("pct50", "50th percentile of request latency in ms (log-normal distribution)") + .withRequiredArg + .describedAs("50th_percentile") + .ofType(classOf[java.lang.Double]) + val verboseOpt = parser.accepts("verbose", "show additional information") + .withRequiredArg + .describedAs("true|false") + .ofType(classOf[java.lang.Boolean]) + .defaultsTo(true) + + val options = parser.parse(args: _*) + + CommandLineUtils.checkRequiredArgs(parser, options, numRequestsOpt, requestRateOpt, requestDataSizeOpt, pct75Opt, pct50Opt) + + val numRequests = options.valueOf(numRequestsOpt).intValue + val requestRate = options.valueOf(requestRateOpt).doubleValue + val requestDataSize = options.valueOf(requestDataSizeOpt).intValue + val numPossibleKeys = options.valueOf(keySpaceSizeOpt).intValue + val numKeys = options.valueOf(numKeysOpt).intValue + val timeout = options.valueOf(timeoutOpt).longValue + val pct75 = options.valueOf(pct75Opt).doubleValue + val pct50 = options.valueOf(pct50Opt).doubleValue + val verbose = options.valueOf(verboseOpt).booleanValue + + val gcMXBeans = ManagementFactory.getGarbageCollectorMXBeans().sortBy(_.getName) + val osMXBean = ManagementFactory.getOperatingSystemMXBean + val latencySamples = new LatencySamples(1000000, pct75, pct50) + val intervalSamples = new IntervalSamples(1000000, requestRate) + + val purgatory = new DelayedOperationPurgatory[FakeOperation]("fake purgatory") + val queue = new CompletionQueue() + + val gcNames = gcMXBeans.map(_.getName) + + val initialCpuTimeNano = getProcessCpuTimeNanos(osMXBean) + val latch = new CountDownLatch(numRequests) + val start = System.currentTimeMillis + val rand = new Random() + val keys = (0 until numKeys).map(i => "fakeKey%d".format(rand.nextInt(numPossibleKeys))) + @volatile var requestArrivalTime = start + @volatile var end = 0L + val generator = new Runnable { + def run(): Unit = { + var i = numRequests + while (i > 0) { + i -= 1 + val requestArrivalInterval = intervalSamples.next() + val latencyToComplete = latencySamples.next() + val now = System.currentTimeMillis + requestArrivalTime = requestArrivalTime + requestArrivalInterval + + if (requestArrivalTime > now) Thread.sleep(requestArrivalTime - now) + + val request = new FakeOperation(timeout, requestDataSize, latencyToComplete, latch) + if (latencyToComplete < timeout) queue.add(request) + purgatory.tryCompleteElseWatch(request, keys) + } + end = System.currentTimeMillis + } + } + val generatorThread = new Thread(generator) + + generatorThread.start() + generatorThread.join() + latch.await() + val done = System.currentTimeMillis + queue.shutdown() + + if (verbose) { + latencySamples.printStats() + intervalSamples.printStats() + println("# enqueue rate (%d requests):".format(numRequests)) + val gcCountHeader = gcNames.map("<" + _ + " count>").mkString(" ") + val gcTimeHeader = gcNames.map("<" + _ + " time ms>").mkString(" ") + println("# \t\t\t\t%s\t%s".format(gcCountHeader, gcTimeHeader)) + } + + val targetRate = numRequests.toDouble * 1000d / (requestArrivalTime - start).toDouble + val actualRate = numRequests.toDouble * 1000d / (end - start).toDouble + + val cpuTime = getProcessCpuTimeNanos(osMXBean).map(x => (x - initialCpuTimeNano.get) / 1000000L) + val gcCounts = gcMXBeans.map(_.getCollectionCount) + val gcTimes = gcMXBeans.map(_.getCollectionTime) + + println("%d\t%f\t%f\t%d\t%s\t%s".format(done - start, targetRate, actualRate, cpuTime.getOrElse(-1L), gcCounts.mkString(" "), gcTimes.mkString(" "))) + + purgatory.shutdown() + } + + // Use JRE-specific class to get process CPU time + private def getProcessCpuTimeNanos(osMXBean : OperatingSystemMXBean) = { + try { + Some(Class.forName("com.sun.management.OperatingSystemMXBean").getMethod("getProcessCpuTime").invoke(osMXBean).asInstanceOf[Long]) + } catch { + case _: Throwable => try { + Some(Class.forName("com.ibm.lang.management.OperatingSystemMXBean").getMethod("getProcessCpuTimeByNS").invoke(osMXBean).asInstanceOf[Long]) + } catch { + case _: Throwable => None + } + } + } + + // log-normal distribution (http://en.wikipedia.org/wiki/Log-normal_distribution) + // mu: the mean of the underlying normal distribution (not the mean of this log-normal distribution) + // sigma: the standard deviation of the underlying normal distribution (not the stdev of this log-normal distribution) + private class LogNormalDistribution(mu: Double, sigma: Double) { + val rand = new Random + def next(): Double = { + val n = rand.nextGaussian() * sigma + mu + math.exp(n) + } + } + + // exponential distribution (http://en.wikipedia.org/wiki/Exponential_distribution) + // lambda : the rate parameter of the exponential distribution + private class ExponentialDistribution(lambda: Double) { + val rand = new Random + def next(): Double = { + math.log(1d - rand.nextDouble()) / (- lambda) + } + } + + // Samples of Latencies to completion + // They are drawn from a log normal distribution. + // A latency value can never be negative. A log-normal distribution is a convenient way to + // model such a random variable. + private class LatencySamples(sampleSize: Int, pct75: Double, pct50: Double) { + private[this] val rand = new Random + private[this] val samples = { + val normalMean = math.log(pct50) + val normalStDev = (math.log(pct75) - normalMean) / 0.674490d // 0.674490 is 75th percentile point in N(0,1) + val dist = new LogNormalDistribution(normalMean, normalStDev) + (0 until sampleSize).map { _ => dist.next().toLong }.toArray + } + def next() = samples(rand.nextInt(sampleSize)) + + def printStats(): Unit = { + val p75 = samples.sorted.apply((sampleSize.toDouble * 0.75d).toInt) + val p50 = samples.sorted.apply((sampleSize.toDouble * 0.5d).toInt) + + println("# latency samples: pct75 = %d, pct50 = %d, min = %d, max = %d".format(p75, p50, samples.min, samples.max)) + } + } + + // Samples of Request arrival intervals + // The request arrival is modeled as a Poisson process. + // So, the internals are drawn from an exponential distribution. + private class IntervalSamples(sampleSize: Int, requestPerSecond: Double) { + private[this] val rand = new Random + private[this] val samples = { + val dist = new ExponentialDistribution(requestPerSecond / 1000d) + var residue = 0.0 + (0 until sampleSize).map { _ => + val interval = dist.next() + residue + val roundedInterval = interval.toLong + residue = interval - roundedInterval.toDouble + roundedInterval + }.toArray + } + + def next() = samples(rand.nextInt(sampleSize)) + + def printStats(): Unit = { + println( + "# interval samples: rate = %f, min = %d, max = %d" + .format(1000d / (samples.map(_.toDouble).sum / sampleSize.toDouble), samples.min, samples.max) + ) + } + } + + private class FakeOperation(delayMs: Long, size: Int, val latencyMs: Long, latch: CountDownLatch) extends DelayedOperation(delayMs) { + private[this] val data = new Array[Byte](size) + val completesAt = System.currentTimeMillis + latencyMs + + def onExpiration(): Unit = {} + + def onComplete(): Unit = { + latch.countDown() + } + + def tryComplete(): Boolean = { + if (System.currentTimeMillis >= completesAt) + forceComplete() + else + false + } + } + + private class CompletionQueue { + private[this] val delayQueue = new DelayQueue[Scheduled]() + private[this] val thread = new ShutdownableThread(name = "completion thread", isInterruptible = false) { + override def doWork(): Unit = { + val scheduled = delayQueue.poll(100, TimeUnit.MILLISECONDS) + if (scheduled != null) { + scheduled.operation.forceComplete() + } + } + } + thread.start() + + def add(operation: FakeOperation): Unit = { + delayQueue.offer(new Scheduled(operation)) + } + + def shutdown() = { + thread.shutdown() + } + + private class Scheduled(val operation: FakeOperation) extends Delayed { + def getDelay(unit: TimeUnit): Long = { + unit.convert(max(operation.completesAt - SystemTime.milliseconds, 0), TimeUnit.MILLISECONDS) + } + + def compareTo(d: Delayed): Int = { + + val other = d.asInstanceOf[Scheduled] + + if (operation.completesAt < other.operation.completesAt) -1 + else if (operation.completesAt > other.operation.completesAt) 1 + else 0 + } + } + } +} diff --git a/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala b/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala deleted file mode 100644 index 31534ca303e91..0000000000000 --- a/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala +++ /dev/null @@ -1,73 +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 - -import consumer._ -import utils.Utils -import java.util.concurrent.CountDownLatch - -object TestZKConsumerOffsets { - def main(args: Array[String]): Unit = { - if(args.length < 1) { - println("USAGE: " + TestZKConsumerOffsets.getClass.getName + " consumer.properties topic latest") - System.exit(1) - } - println("Starting consumer...") - val topic = args(1) - val autoOffsetReset = args(2) - val props = Utils.loadProps(args(0)) - props.put("auto.offset.reset", "largest") - - val config = new ConsumerConfig(props) - val consumerConnector: ConsumerConnector = Consumer.create(config) - val topicMessageStreams = consumerConnector.createMessageStreams(Predef.Map(topic -> 1)) - var threadList = List[ConsumerThread]() - for ((topic, streamList) <- topicMessageStreams) - for (stream <- streamList) - threadList ::= new ConsumerThread(stream) - - for (thread <- threadList) - thread.start - - // attach shutdown handler to catch control-c - Runtime.getRuntime().addShutdownHook(new Thread() { - override def run() = { - consumerConnector.shutdown - threadList.foreach(_.shutdown) - println("consumer threads shutted down") - } - }) - } -} - -private class ConsumerThread(stream: KafkaStream[Array[Byte], Array[Byte]]) extends Thread { - val shutdownLatch = new CountDownLatch(1) - - override def run() { - println("Starting consumer thread..") - for (messageAndMetadata <- stream) { - println("consumed: " + new String(messageAndMetadata.message, "UTF-8")) - } - shutdownLatch.countDown - println("thread shutdown !" ) - } - - def shutdown() { - shutdownLatch.await - } -} diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala new file mode 100644 index 0000000000000..4764c8976022d --- /dev/null +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -0,0 +1,120 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka + +import java.io.{FileOutputStream, File} +import java.security.Permission + +import kafka.server.KafkaConfig +import org.junit.{After, Before, Test} +import junit.framework.Assert._ + +class KafkaTest { + + val originalSecurityManager: SecurityManager = System.getSecurityManager + + class ExitCalled extends SecurityException { + } + + private class NoExitSecurityManager extends SecurityManager { + override def checkExit(status: Int): Unit = { + throw new ExitCalled + } + + override def checkPermission(perm : Permission): Unit = { + } + + override def checkPermission(perm : Permission, context: Object): Unit = { + } + } + + @Before + def setSecurityManager() : Unit = { + System.setSecurityManager(new NoExitSecurityManager) + } + + @After + def setOriginalSecurityManager() : Unit = { + System.setSecurityManager(originalSecurityManager) + } + + @Test + def testGetKafkaConfigFromArgs(): Unit = { + val propertiesFile = prepareDefaultConfig() + + // We should load configuration file without any arguments + val config1 = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile))) + assertEquals(1, config1.brokerId) + + // We should be able to override given property on command line + val config2 = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "--override", "broker.id=2"))) + assertEquals(2, config2.brokerId) + + // We should be also able to set completely new property + val config3 = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "--override", "log.cleanup.policy=compact"))) + assertEquals(1, config3.brokerId) + assertEquals("compact", config3.logCleanupPolicy) + + // We should be also able to set several properties + val config4 = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "--override", "log.cleanup.policy=compact", "--override", "broker.id=2"))) + assertEquals(2, config4.brokerId) + assertEquals("compact", config4.logCleanupPolicy) + } + + @Test(expected = classOf[ExitCalled]) + def testGetKafkaConfigFromArgsWrongSetValue(): Unit = { + val propertiesFile = prepareDefaultConfig() + KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "--override", "a=b=c"))) + } + + @Test(expected = classOf[ExitCalled]) + def testGetKafkaConfigFromArgsNonArgsAtTheEnd(): Unit = { + val propertiesFile = prepareDefaultConfig() + KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "--override", "broker.id=1", "broker.id=2"))) + } + + @Test(expected = classOf[ExitCalled]) + def testGetKafkaConfigFromArgsNonArgsOnly(): Unit = { + val propertiesFile = prepareDefaultConfig() + KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "broker.id=1", "broker.id=2"))) + } + + @Test(expected = classOf[ExitCalled]) + def testGetKafkaConfigFromArgsNonArgsAtTheBegging(): Unit = { + val propertiesFile = prepareDefaultConfig() + KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "broker.id=1", "--override", "broker.id=2"))) + } + + def prepareDefaultConfig(): String = { + prepareConfig(Array("broker.id=1", "zookeeper.connect=somewhere")) + } + + def prepareConfig(lines : Array[String]): String = { + val file = File.createTempFile("kafkatest", ".properties") + file.deleteOnExit() + + val writer = new FileOutputStream(file) + lines.foreach { l => + writer.write(l.getBytes) + writer.write("\n".getBytes) + } + + writer.close + + file.getAbsolutePath + } +} diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala old mode 100644 new mode 100755 index 115e20305a154..8b14bcfe7af60 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -17,31 +17,18 @@ package kafka.admin +import org.apache.kafka.common.protocol.SecurityProtocol import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import junit.framework.Assert._ -import kafka.utils.{ZkUtils, Utils, TestUtils} +import kafka.utils.{ZkUtils, CoreUtils, TestUtils} import kafka.cluster.Broker import kafka.client.ClientUtils import kafka.server.{KafkaConfig, KafkaServer} class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { - val brokerId1 = 0 - val brokerId2 = 1 - val brokerId3 = 2 - val brokerId4 = 3 - - val port1 = TestUtils.choosePort() - val port2 = TestUtils.choosePort() - val port3 = TestUtils.choosePort() - val port4 = TestUtils.choosePort() - - val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) - val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) - val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3) - val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4) - + var configs: Seq[KafkaConfig] = null var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] var brokers: Seq[Broker] = Seq.empty[Broker] @@ -54,47 +41,22 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() - // start all the servers - val server1 = TestUtils.createServer(new KafkaConfig(configProps1)) - val server2 = TestUtils.createServer(new KafkaConfig(configProps2)) - val server3 = TestUtils.createServer(new KafkaConfig(configProps3)) - val server4 = TestUtils.createServer(new KafkaConfig(configProps4)) - - servers ++= List(server1, server2, server3, server4) - brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.config.port)) - - // create topics with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic1, Map(0->Seq(0,1))) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic2, Map(0->Seq(1,2))) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic3, Map(0->Seq(2,3,0,1))) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic4, Map(0->Seq(0,3))) - - // wait until leader is elected - var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partitionId, 500) - var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, partitionId, 500) - var leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, partitionId, 500) - var leader4 = waitUntilLeaderIsElectedOrChanged(zkClient, topic4, partitionId, 500) - - debug("Leader for " + topic1 + " is elected to be: %s".format(leader1.getOrElse(-1))) - debug("Leader for " + topic2 + " is elected to be: %s".format(leader1.getOrElse(-1))) - debug("Leader for " + topic3 + "is elected to be: %s".format(leader1.getOrElse(-1))) - debug("Leader for " + topic4 + "is elected to be: %s".format(leader1.getOrElse(-1))) - - assertTrue("Leader should get elected", leader1.isDefined) - assertTrue("Leader should get elected", leader2.isDefined) - assertTrue("Leader should get elected", leader3.isDefined) - assertTrue("Leader should get elected", leader4.isDefined) - - assertTrue("Leader could be broker 0 or broker 1 for " + topic1, (leader1.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 1)) - assertTrue("Leader could be broker 1 or broker 2 for " + topic2, (leader2.getOrElse(-1) == 1) || (leader1.getOrElse(-1) == 2)) - assertTrue("Leader could be broker 2 or broker 3 for " + topic3, (leader3.getOrElse(-1) == 2) || (leader1.getOrElse(-1) == 3)) - assertTrue("Leader could be broker 3 or broker 4 for " + topic4, (leader4.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 3)) + configs = (0 until 4).map(i => KafkaConfig.fromProps(TestUtils.createBrokerConfig(i, zkConnect, enableControlledShutdown = false))) + // start all the servers + servers = configs.map(c => TestUtils.createServer(c)) + brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.boundPort)) + + // create topics first + createTopic(zkClient, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) + createTopic(zkClient, topic2, partitionReplicaAssignment = Map(0->Seq(1,2)), servers = servers) + createTopic(zkClient, topic3, partitionReplicaAssignment = Map(0->Seq(2,3,0,1)), servers = servers) + createTopic(zkClient, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers) } override def tearDown() { - servers.map(server => server.shutdown()) - servers.map(server => Utils.rm(server.config.logDirs)) + servers.foreach(_.shutdown()) + servers.foreach(server => CoreUtils.rm(server.config.logDirs)) super.tearDown() } @@ -121,20 +83,20 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testIncrementPartitions { AdminUtils.addPartitions(zkClient, topic1, 3) // wait until leader is elected - var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1, 500) - var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2, 500) + var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1) + var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2) val leader1FromZk = ZkUtils.getLeaderForPartition(zkClient, topic1, 1).get val leader2FromZk = ZkUtils.getLeaderForPartition(zkClient, topic1, 2).get assertEquals(leader1.get, leader1FromZk) assertEquals(leader2.get, leader2FromZk) // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 1, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 2, 1000) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers, "AddPartitionsTest-testIncrementPartitions", + TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 1) + TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 2) + val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testIncrementPartitions", 2000,0).topicsMetadata val metaDataForTopic1 = metadata.filter(p => p.topic.equals(topic1)) - val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata + val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata.sortBy(_.partitionId) assertEquals(partitionDataForTopic1.size, 3) assertEquals(partitionDataForTopic1(1).partitionId, 1) assertEquals(partitionDataForTopic1(2).partitionId, 2) @@ -146,20 +108,20 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testManualAssignmentOfReplicas { 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) + var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 1) + var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 2) val leader1FromZk = ZkUtils.getLeaderForPartition(zkClient, topic2, 1).get val leader2FromZk = ZkUtils.getLeaderForPartition(zkClient, topic2, 2).get assertEquals(leader1.get, leader1FromZk) assertEquals(leader2.get, leader2FromZk) // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 1, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 2, 1000) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers, "AddPartitionsTest-testManualAssignmentOfReplicas", + TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 1) + TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 2) + val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testManualAssignmentOfReplicas", 2000,0).topicsMetadata val metaDataForTopic2 = metadata.filter(p => p.topic.equals(topic2)) - val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata + val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata.sortBy(_.partitionId) assertEquals(partitionDataForTopic2.size, 3) assertEquals(partitionDataForTopic2(1).partitionId, 1) assertEquals(partitionDataForTopic2(2).partitionId, 2) @@ -171,46 +133,26 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testReplicaPlacement { AdminUtils.addPartitions(zkClient, topic3, 7) - // wait until leader is elected - var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 1, 500) - var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 2, 500) - var leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 3, 500) - var leader4 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 4, 500) - var leader5 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 5, 500) - var leader6 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 6, 500) - - val leader1FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 1).get - val leader2FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 2).get - val leader3FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 3).get - val leader4FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 4).get - val leader5FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 5).get - val leader6FromZk = ZkUtils.getLeaderForPartition(zkClient, topic3, 6).get - - assertEquals(leader1.get, leader1FromZk) - assertEquals(leader2.get, leader2FromZk) - assertEquals(leader3.get, leader3FromZk) - assertEquals(leader4.get, leader4FromZk) - assertEquals(leader5.get, leader5FromZk) - assertEquals(leader6.get, leader6FromZk) // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 1, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 2, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 3, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 4, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 5, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 6, 1000) - - val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers, "AddPartitionsTest-testReplicaPlacement", + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 1) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 2) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 3) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 4) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 5) + TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 6) + + val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testReplicaPlacement", 2000,0).topicsMetadata val metaDataForTopic3 = metadata.filter(p => p.topic.equals(topic3)).head - val partition1DataForTopic3 = metaDataForTopic3.partitionsMetadata(1) - val partition2DataForTopic3 = metaDataForTopic3.partitionsMetadata(2) - val partition3DataForTopic3 = metaDataForTopic3.partitionsMetadata(3) - val partition4DataForTopic3 = metaDataForTopic3.partitionsMetadata(4) - val partition5DataForTopic3 = metaDataForTopic3.partitionsMetadata(5) - val partition6DataForTopic3 = metaDataForTopic3.partitionsMetadata(6) + val partitionsMetadataForTopic3 = metaDataForTopic3.partitionsMetadata.sortBy(_.partitionId) + val partition1DataForTopic3 = partitionsMetadataForTopic3(1) + val partition2DataForTopic3 = partitionsMetadataForTopic3(2) + val partition3DataForTopic3 = partitionsMetadataForTopic3(3) + val partition4DataForTopic3 = partitionsMetadataForTopic3(4) + val partition5DataForTopic3 = partitionsMetadataForTopic3(5) + val partition6DataForTopic3 = partitionsMetadataForTopic3(6) assertEquals(partition1DataForTopic3.replicas.size, 4) assertEquals(partition1DataForTopic3.replicas(0).id, 3) @@ -248,4 +190,4 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(partition6DataForTopic3.replicas(2).id, 2) assertEquals(partition6DataForTopic3.replicas(3).id, 3) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala old mode 100644 new mode 100755 index d5644ea40ec76..252ac813c8df1 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -145,7 +145,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 @@ -156,16 +156,18 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { assertTrue("Partition reassignment attempt failed for [test, 0]", reassignPartitionsCommand.reassignPartitions()) // wait until reassignment is completed TestUtils.waitUntilTrue(() => { - val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); - ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, - Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; - }, 1000) + val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); + ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, + Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; + }, + "Partition reassignment should complete") val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) // in sync replicas should not have any replica that is not in the new assigned replicas checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas) assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas) ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers) - assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) + TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, + "New replicas should exist on brokers") servers.foreach(_.shutdown()) } @@ -174,7 +176,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 @@ -185,15 +187,18 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) // wait until reassignment is completed TestUtils.waitUntilTrue(() => { - val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); - ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, - Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; - }, 1000) + val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); + ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, + Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; + }, + "Partition reassignment should complete") val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas) checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas) ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers) - assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) + TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, + "New replicas should exist on brokers") + servers.foreach(_.shutdown()) } @@ -202,7 +207,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1)) val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 @@ -213,15 +218,17 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) // wait until reassignment is completed TestUtils.waitUntilTrue(() => { - val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); - ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, - Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; - }, 2000) + val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); + ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, + Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; + }, + "Partition reassignment should complete") val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 2, 3", newReplicas, assignedReplicas) checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas) ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers) - assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) + TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, + "New replicas should exist on brokers") servers.foreach(_.shutdown()) } @@ -229,13 +236,13 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { def testReassigningNonExistingPartition() { val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // reassign partition 0 val newReplicas = Seq(2, 3) val partitionToBeReassigned = 0 val topicAndPartition = TopicAndPartition(topic, partitionToBeReassigned) val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) - assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) + assertFalse("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) val reassignedPartitions = ZkUtils.getPartitionsBeingReassigned(zkClient) assertFalse("Partition should not be reassigned", reassignedPartitions.contains(topicAndPartition)) servers.foreach(_.shutdown()) @@ -255,14 +262,18 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) reassignPartitionsCommand.reassignPartitions // create brokers - val servers = TestUtils.createBrokerConfigs(2).map(b => TestUtils.createServer(new KafkaConfig(b))) - TestUtils.waitUntilTrue(() => checkIfReassignPartitionPathExists(zkClient), 1000) + val servers = TestUtils.createBrokerConfigs(2, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) + + // wait until reassignment completes + TestUtils.waitUntilTrue(() => !checkIfReassignPartitionPathExists(zkClient), + "Partition reassignment should complete") val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 0, 1", newReplicas, assignedReplicas) checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas) // ensure that there are no under replicated partitions ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers) - assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) + TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, + "New replicas should exist on brokers") servers.foreach(_.shutdown()) } @@ -287,16 +298,16 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val partition = 1 val preferredReplica = 0 // create brokers - val serverConfigs = TestUtils.createBrokerConfigs(3).map(new KafkaConfig(_)) + val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false).map(KafkaConfig.fromProps) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) - val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) + val servers = serverConfigs.reverseMap(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 + val currentLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, oldLeaderOpt = None).get // trigger preferred replica election val preferredReplicaElection = new PreferredReplicaLeaderElectionCommand(zkClient, Set(TopicAndPartition(topic, partition))) preferredReplicaElection.moveLeaderToPreferredReplica() - val newLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, 1000, Some(currentLeader)).get + val newLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, oldLeaderOpt = Some(currentLeader)).get assertEquals("Preferred replica election failed", preferredReplica, newLeader) servers.foreach(_.shutdown()) } @@ -307,11 +318,10 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val topic = "test" val partition = 1 // create brokers - val serverConfigs = TestUtils.createBrokerConfigs(3).map(new KafkaConfig(_)) - val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) + val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false).map(KafkaConfig.fromProps) + val servers = serverConfigs.reverseMap(s => TestUtils.createServer(s)) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, partition, 1000) + TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = servers) val controllerId = ZkUtils.getController(zkClient) val controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController @@ -319,10 +329,11 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { var activeServers = servers.filter(s => s.config.brokerId != 2) try { // wait for the update metadata request to trickle to the brokers - assertTrue("Topic test not created after timeout", TestUtils.waitUntilTrue(() => - activeServers.foldLeft(true)(_ && _.apis.metadataCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.isr.size != 3), 1000)) + TestUtils.waitUntilTrue(() => + activeServers.forall(_.apis.metadataCache.getPartitionInfo(topic,partition).get.leaderIsrAndControllerEpoch.leaderAndIsr.isr.size != 3), + "Topic test not created after timeout") assertEquals(0, partitionsRemaining.size) - var partitionStateInfo = activeServers.head.apis.metadataCache(TopicAndPartition(topic, partition)) + var partitionStateInfo = activeServers.head.apis.metadataCache.getPartitionInfo(topic,partition).get var leaderAfterShutdown = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader assertEquals(0, leaderAfterShutdown) assertEquals(2, partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.isr.size) @@ -331,15 +342,15 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { partitionsRemaining = controller.shutdownBroker(1) assertEquals(0, partitionsRemaining.size) activeServers = servers.filter(s => s.config.brokerId == 0) - partitionStateInfo = activeServers.head.apis.metadataCache(TopicAndPartition(topic, partition)) + partitionStateInfo = activeServers.head.apis.metadataCache.getPartitionInfo(topic,partition).get leaderAfterShutdown = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader assertEquals(0, leaderAfterShutdown) - assertTrue(servers.foldLeft(true)(_ && _.apis.metadataCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) + assertTrue(servers.forall(_.apis.metadataCache.getPartitionInfo(topic,partition).get.leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) partitionsRemaining = controller.shutdownBroker(0) assertEquals(1, partitionsRemaining.size) // leader doesn't change since all the replicas are shut down - assertTrue(servers.foldLeft(true)(_ && _.apis.metadataCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) + assertTrue(servers.forall(_.apis.metadataCache.getPartitionInfo(topic,partition).get.leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) } finally { servers.foreach(_.shutdown()) @@ -354,12 +365,12 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { def testTopicConfigChange() { val partitions = 3 val topic = "my-topic" - val server = TestUtils.createServer(new KafkaConfig(TestUtils.createBrokerConfig(0))) + val server = TestUtils.createServer(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) def makeConfig(messageSize: Int, retentionMs: Long) = { var props = new Properties() props.setProperty(LogConfig.MaxMessageBytesProp, messageSize.toString) - props.setProperty(LogConfig.RententionMsProp, retentionMs.toString) + props.setProperty(LogConfig.RetentionMsProp, retentionMs.toString) props } @@ -386,7 +397,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { checkConfig(2*maxMessageSize, 2 * retentionMs) } finally { server.shutdown() - server.config.logDirs.map(Utils.rm(_)) + server.config.logDirs.foreach(CoreUtils.rm(_)) } } diff --git a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala new file mode 100644 index 0000000000000..1913ad6d3e6eb --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala @@ -0,0 +1,212 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.admin + +import org.scalatest.junit.JUnit3Suite +import kafka.utils._ +import kafka.server.KafkaConfig +import org.junit.Test +import kafka.consumer._ +import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer} +import kafka.integration.KafkaServerTestHarness + + +class DeleteConsumerGroupTest extends JUnit3Suite with KafkaServerTestHarness { + def generateConfigs() = TestUtils.createBrokerConfigs(3, zkConnect, false, true).map(KafkaConfig.fromProps) + + @Test + def testGroupWideDeleteInZK() { + val topic = "test" + val groupToDelete = "groupToDelete" + val otherGroup = "otherGroup" + + TestUtils.createTopic(zkClient, topic, 1, 3, servers) + fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, false) + fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false) + + AdminUtils.deleteConsumerGroupInZK(zkClient, groupToDelete) + + TestUtils.waitUntilTrue(() => !groupDirExists(new ZKGroupDirs(groupToDelete)), + "DeleteConsumerGroupInZK should delete the provided consumer group's directory") + TestUtils.waitUntilTrue(() => groupDirExists(new ZKGroupDirs(otherGroup)), + "DeleteConsumerGroupInZK should not delete unrelated consumer group directories") + } + + @Test + def testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup() { + val topic = "test" + val groupToDelete = "groupToDelete" + val otherGroup = "otherGroup" + + TestUtils.createTopic(zkClient, topic, 1, 3, servers) + fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, true) + fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false) + + AdminUtils.deleteConsumerGroupInZK(zkClient, groupToDelete) + + TestUtils.waitUntilTrue(() => groupDirExists(new ZKGroupDirs(groupToDelete)), + "DeleteConsumerGroupInZK should not delete the provided consumer group's directory if the consumer group is still active") + TestUtils.waitUntilTrue(() => groupDirExists(new ZKGroupDirs(otherGroup)), + "DeleteConsumerGroupInZK should not delete unrelated consumer group directories") + } + + @Test + def testGroupTopicWideDeleteInZKForGroupConsumingOneTopic() { + val topic = "test" + val groupToDelete = "groupToDelete" + val otherGroup = "otherGroup" + TestUtils.createTopic(zkClient, topic, 1, 3, servers) + fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, false) + fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false) + + AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, groupToDelete, topic) + + TestUtils.waitUntilTrue(() => !groupDirExists(new ZKGroupDirs(groupToDelete)), + "DeleteConsumerGroupInfoForTopicInZK should delete the provided consumer group's directory if it just consumes from one topic") + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(otherGroup, topic)), + "DeleteConsumerGroupInfoForTopicInZK should not delete unrelated consumer group owner and offset directories") + } + + @Test + def testGroupTopicWideDeleteInZKForGroupConsumingMultipleTopics() { + val topicToDelete = "topicToDelete" + val otherTopic = "otherTopic" + val groupToDelete = "groupToDelete" + val otherGroup = "otherGroup" + TestUtils.createTopic(zkClient, topicToDelete, 1, 3, servers) + TestUtils.createTopic(zkClient, otherTopic, 1, 3, servers) + + fillInConsumerGroupInfo(topicToDelete, groupToDelete, "consumer", 0, 10, false) + fillInConsumerGroupInfo(otherTopic, groupToDelete, "consumer", 0, 10, false) + fillInConsumerGroupInfo(topicToDelete, otherGroup, "consumer", 0, 10, false) + + AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, groupToDelete, topicToDelete) + + TestUtils.waitUntilTrue(() => !groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(groupToDelete, topicToDelete)), + "DeleteConsumerGroupInfoForTopicInZK should delete the provided consumer group's owner and offset directories for the given topic") + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(groupToDelete, otherTopic)), + "DeleteConsumerGroupInfoForTopicInZK should not delete the provided consumer group's owner and offset directories for unrelated topics") + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(otherGroup, topicToDelete)), + "DeleteConsumerGroupInfoForTopicInZK should not delete unrelated consumer group owner and offset directories") + } + + @Test + def testGroupTopicWideDeleteInZKDoesNothingForActiveGroupConsumingMultipleTopics() { + val topicToDelete = "topicToDelete" + val otherTopic = "otherTopic" + val group = "group" + TestUtils.createTopic(zkClient, topicToDelete, 1, 3, servers) + TestUtils.createTopic(zkClient, otherTopic, 1, 3, servers) + + fillInConsumerGroupInfo(topicToDelete, group, "consumer", 0, 10, true) + fillInConsumerGroupInfo(otherTopic, group, "consumer", 0, 10, true) + + AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, group, topicToDelete) + + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(group, topicToDelete)), + "DeleteConsumerGroupInfoForTopicInZK should not delete the provided consumer group's owner and offset directories for the given topic if the consumer group is still active") + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(group, otherTopic)), + "DeleteConsumerGroupInfoForTopicInZK should not delete the provided consumer group's owner and offset directories for unrelated topics") + } + + @Test + def testTopicWideDeleteInZK() { + val topicToDelete = "topicToDelete" + val otherTopic = "otherTopic" + val groups = Seq("group1", "group2") + + TestUtils.createTopic(zkClient, topicToDelete, 1, 3, servers) + TestUtils.createTopic(zkClient, otherTopic, 1, 3, servers) + val groupTopicDirsForTopicToDelete = groups.map(group => new ZKGroupTopicDirs(group, topicToDelete)) + val groupTopicDirsForOtherTopic = groups.map(group => new ZKGroupTopicDirs(group, otherTopic)) + groupTopicDirsForTopicToDelete.foreach(dir => fillInConsumerGroupInfo(topicToDelete, dir.group, "consumer", 0, 10, false)) + groupTopicDirsForOtherTopic.foreach(dir => fillInConsumerGroupInfo(otherTopic, dir.group, "consumer", 0, 10, false)) + + AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkClient, topicToDelete) + + TestUtils.waitUntilTrue(() => !groupTopicDirsForTopicToDelete.exists(groupTopicOffsetAndOwnerDirsExist), + "Consumer group info on deleted topic topic should be deleted by DeleteAllConsumerGroupInfoForTopicInZK") + TestUtils.waitUntilTrue(() => groupTopicDirsForOtherTopic.forall(groupTopicOffsetAndOwnerDirsExist), + "Consumer group info on unrelated topics should not be deleted by DeleteAllConsumerGroupInfoForTopicInZK") + } + + @Test + def testConsumptionOnRecreatedTopicAfterTopicWideDeleteInZK() { + val topic = "topic" + val group = "group" + + TestUtils.createTopic(zkClient, topic, 1, 3, servers) + val dir = new ZKGroupTopicDirs(group, topic) + fillInConsumerGroupInfo(topic, dir.group, "consumer", 0, 10, false) + + AdminUtils.deleteTopic(zkClient, topic) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) + AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkClient, topic) + + TestUtils.waitUntilTrue(() => !groupDirExists(dir), + "Consumer group info on related topics should be deleted by DeleteAllConsumerGroupInfoForTopicInZK") + //produce events + val producer = TestUtils.createNewProducer(brokerList) + produceEvents(producer, topic, List.fill(10)("test")) + + //consume events + val consumerProps = TestUtils.createConsumerProperties(zkConnect, group, "consumer") + consumerProps.put("auto.commit.enable", "false") + consumerProps.put("auto.offset.reset", "smallest") + consumerProps.put("consumer.timeout.ms", "2000") + consumerProps.put("fetch.wait.max.ms", "0") + val consumerConfig = new ConsumerConfig(consumerProps) + val consumerConnector: ConsumerConnector = Consumer.create(consumerConfig) + val messageStream = consumerConnector.createMessageStreams(Map(topic -> 1))(topic).head + consumeEvents(messageStream, 5) + consumerConnector.commitOffsets(false) + producer.close() + consumerConnector.shutdown() + + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(dir), + "Consumer group info should exist after consuming from a recreated topic") + } + + private def fillInConsumerGroupInfo(topic: String, group: String, consumerId: String, partition: Int, offset: Int, registerConsumer: Boolean) { + val consumerProps = TestUtils.createConsumerProperties(zkConnect, group, consumerId) + val consumerConfig = new ConsumerConfig(consumerProps) + val dir = new ZKGroupTopicDirs(group, topic) + TestUtils.updateConsumerOffset(consumerConfig, dir.consumerOffsetDir + "/" + partition, offset) + ZkUtils.createEphemeralPathExpectConflict(zkClient, ZkUtils.getConsumerPartitionOwnerPath(group, topic, partition), "") + ZkUtils.makeSurePersistentPathExists(zkClient, dir.consumerRegistryDir) + if (registerConsumer) { + ZkUtils.createEphemeralPathExpectConflict(zkClient, dir.consumerRegistryDir + "/" + consumerId, "") + } + } + + private def groupDirExists(dir: ZKGroupDirs) = { + ZkUtils.pathExists(zkClient, dir.consumerGroupDir) + } + + private def groupTopicOffsetAndOwnerDirsExist(dir: ZKGroupTopicDirs) = { + ZkUtils.pathExists(zkClient, dir.consumerOffsetDir) && ZkUtils.pathExists(zkClient, dir.consumerOwnerDir) + } + + private def produceEvents(producer: KafkaProducer[Array[Byte], Array[Byte]], topic: String, messages: List[String]) { + messages.foreach(message => producer.send(new ProducerRecord(topic, message.getBytes))) + } + + private def consumeEvents(messageStream: KafkaStream[Array[Byte], Array[Byte]], n: Int) { + val iter = messageStream.iterator + (0 until n).foreach(_ => iter.next) + } +} diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index 974b057a88056..fa8ce259a2832 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -6,7 +6,7 @@ * (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 + * 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, @@ -16,20 +16,15 @@ */ package kafka.admin +import kafka.log.Log import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import junit.framework.Assert._ import kafka.utils.{ZkUtils, TestUtils} import kafka.server.{KafkaServer, KafkaConfig} import org.junit.Test -import kafka.common._ -import kafka.producer.{ProducerConfig, Producer} import java.util.Properties -import kafka.api._ -import kafka.consumer.SimpleConsumer -import kafka.producer.KeyedMessage import kafka.common.TopicAndPartition -import kafka.api.PartitionOffsetRequestInfo class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -40,7 +35,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val servers = createTestTopicAndCluster(topic) // start topic deletion AdminUtils.deleteTopic(zkClient, topic) - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) servers.foreach(_.shutdown()) } @@ -57,15 +52,15 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // start topic deletion AdminUtils.deleteTopic(zkClient, topic) // check if all replicas but the one that is shut down has deleted the log - assertTrue("Replicas 0,1 have not deleted log in 1000ms", TestUtils.waitUntilTrue(() => + TestUtils.waitUntilTrue(() => servers.filter(s => s.config.brokerId != follower.config.brokerId) - .foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isEmpty), 1000)) + .forall(_.getLogManager().getLog(topicAndPartition).isEmpty), "Replicas 0,1 have not deleted log.") // ensure topic deletion is halted - assertTrue("Admin path /admin/delete_topic/test path deleted in 1000ms even when a follower replica is down", - TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), 500)) + TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), + "Admin path /admin/delete_topic/test path deleted even when a follower replica is down") // restart follower replica follower.startup() - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) servers.foreach(_.shutdown()) } @@ -74,161 +69,47 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val topicAndPartition = TopicAndPartition("test", 0) val topic = topicAndPartition.topic val servers = createTestTopicAndCluster(topic) + val controllerId = ZkUtils.getController(zkClient) + val controller = servers.filter(s => s.config.brokerId == controllerId).head + val leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) + val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get && s.config.brokerId != controllerId).last + follower.shutdown() + // start topic deletion AdminUtils.deleteTopic(zkClient, topic) // shut down the controller to trigger controller failover during delete topic - val controllerId = ZkUtils.getController(zkClient) - val controller = servers.filter(s => s.config.brokerId == controllerId).head controller.shutdown() + // ensure topic deletion is halted - assertTrue("Admin path /admin/delete_topic/test path deleted in 500ms even when a replica is down", - TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), 500)) - // restart follower replica - controller.startup() - // wait until admin path for delete topic is deleted, signaling completion of topic deletion - assertTrue("Admin path /admin/delete_topic/test path not deleted in 4000ms even after a follower replica is restarted", - TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), 4000)) - assertTrue("Topic path /brokers/topics/test not deleted after /admin/delete_topic/test path is deleted", - TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)), 100)) - // ensure that logs from all replicas are deleted if delete topic is marked successful in zookeeper - assertTrue("Replica logs not deleted after delete topic is complete", - servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isEmpty)) - servers.foreach(_.shutdown()) - } + TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), + "Admin path /admin/delete_topic/test path deleted even when a replica is down") - @Test - def testRequestHandlingDuringDeleteTopic() { - val topicAndPartition = TopicAndPartition("test", 0) - val topic = topicAndPartition.topic - val servers = createTestTopicAndCluster(topic) - // start topic deletion - AdminUtils.deleteTopic(zkClient, topic) - // shut down one follower replica - var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) - assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) - val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get).last - follower.shutdown() - // test if produce requests are failed with UnknownTopicOrPartitionException during delete topic - val props1 = new Properties() - props1.put("metadata.broker.list", servers.map(s => s.config.hostName + ":" + s.config.port).mkString(",")) - props1.put("serializer.class", "kafka.serializer.StringEncoder") - props1.put("request.required.acks", "1") - val producerConfig1 = new ProducerConfig(props1) - val producer1 = new Producer[String, String](producerConfig1) - try{ - producer1.send(new KeyedMessage[String, String](topic, "test", "test1")) - fail("Test should fail because the topic is being deleted") - } catch { - case e: FailedToSendMessageException => - case oe: Throwable => fail("fails with exception", oe) - } finally { - producer1.close() - } - // test if fetch requests fail during delete topic - servers.filter(s => s.config.brokerId != follower.config.brokerId).foreach { server => - val consumer = new SimpleConsumer(server.config.hostName, server.config.port, 1000000, 64*1024, "") - val request = new FetchRequestBuilder() - .clientId("test-client") - .addFetch(topic, 0, 0, 10000) - .build() - val fetched = consumer.fetch(request) - val fetchResponse = fetched.data(topicAndPartition) - assertTrue("Fetch should fail with UnknownTopicOrPartitionCode", fetchResponse.error == ErrorMapping.UnknownTopicOrPartitionCode) - } - // test if offset requests fail during delete topic - servers.filter(s => s.config.brokerId != follower.config.brokerId).foreach { server => - val consumer = new SimpleConsumer(server.config.hostName, server.config.port, 1000000, 64*1024, "") - val offsetRequest = new OffsetRequest(Map(topicAndPartition -> new PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1))) - val offsetResponse = consumer.getOffsetsBefore(offsetRequest) - val errorCode = offsetResponse.partitionErrorAndOffsets(topicAndPartition).error - assertTrue("Offset request should fail with UnknownTopicOrPartitionCode", errorCode == ErrorMapping.UnknownTopicOrPartitionCode) - // test if offset fetch requests fail during delete topic - val offsetFetchRequest = new OffsetFetchRequest("test-group", Seq(topicAndPartition)) - val offsetFetchResponse = consumer.fetchOffsets(offsetFetchRequest) - val offsetFetchErrorCode = offsetFetchResponse.requestInfo(topicAndPartition).error - assertTrue("Offset fetch request should fail with UnknownTopicOrPartitionCode", - offsetFetchErrorCode == ErrorMapping.UnknownTopicOrPartitionCode) - // TODO: test if offset commit requests fail during delete topic - } - // restart follower replica + controller.startup() follower.startup() - verifyTopicDeletion(topic, servers) - servers.foreach(_.shutdown()) - } - @Test - def testPreferredReplicaElectionDuringDeleteTopic() { - val topicAndPartition = TopicAndPartition("test", 0) - val topic = topicAndPartition.topic - val servers = createTestTopicAndCluster(topic) - var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) - assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) - // shut down the controller to move the leader to a non preferred replica before delete topic - val preferredReplicaId = 0 - val preferredReplica = servers.filter(s => s.config.brokerId == preferredReplicaId).head - preferredReplica.shutdown() - preferredReplica.startup() - val newLeaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 3000, leaderIdOpt) - assertTrue("New leader should be elected prior to delete topic", newLeaderIdOpt.isDefined) - // start topic deletion - AdminUtils.deleteTopic(zkClient, topic) - // test preferred replica election - val preferredReplicaElection = new PreferredReplicaLeaderElectionCommand(zkClient, Set(topicAndPartition)) - preferredReplicaElection.moveLeaderToPreferredReplica() - val leaderAfterPreferredReplicaElectionOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000, newLeaderIdOpt) - assertTrue("Preferred replica election should not move leader during delete topic", - leaderAfterPreferredReplicaElectionOpt.isEmpty || leaderAfterPreferredReplicaElectionOpt.get == newLeaderIdOpt.get) - val newControllerId = ZkUtils.getController(zkClient) - val newController = servers.filter(s => s.config.brokerId == newControllerId).head - assertFalse("Preferred replica election should fail", - newController.kafkaController.controllerContext.partitionsUndergoingPreferredReplicaElection.contains(topicAndPartition)) - verifyTopicDeletion(topic, servers) - servers.foreach(_.shutdown()) - } - - @Test - def testDeleteTopicDuringPreferredReplicaElection() { - val topic = "test" - val topicAndPartition = TopicAndPartition(topic, 0) - val servers = createTestTopicAndCluster(topic) - var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) - assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) - // shut down the controller to move the leader to a non preferred replica before delete topic - val preferredReplicaId = 0 - val preferredReplica = servers.filter(s => s.config.brokerId == preferredReplicaId).head - preferredReplica.shutdown() - preferredReplica.startup() - val newLeaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 3000, leaderIdOpt) - assertTrue("New leader should be elected prior to delete topic", newLeaderIdOpt.isDefined) - // test preferred replica election - val preferredReplicaElection = new PreferredReplicaLeaderElectionCommand(zkClient, Set(topicAndPartition)) - preferredReplicaElection.moveLeaderToPreferredReplica() - // start topic deletion during preferred replica election. This should halt topic deletion but eventually - // complete it successfully - AdminUtils.deleteTopic(zkClient, topic) - val newControllerId = ZkUtils.getController(zkClient) - val newController = servers.filter(s => s.config.brokerId == newControllerId).head - assertTrue("Preferred replica election should succeed after 1000ms", TestUtils.waitUntilTrue(() => - !newController.kafkaController.controllerContext.partitionsUndergoingPreferredReplicaElection.contains(topicAndPartition), 1000)) - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) servers.foreach(_.shutdown()) } @Test def testPartitionReassignmentDuringDeleteTopic() { - val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) + val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" val topicAndPartition = TopicAndPartition(topic, 0) + val brokerConfigs = TestUtils.createBrokerConfigs(4, zkConnect, false) + brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true")) // create brokers - val allServers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) + val allServers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) val servers = allServers.filter(s => expectedReplicaAssignment(0).contains(s.config.brokerId)) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // wait until replica log is created on every broker - assertTrue("Replicas for topic test not created in 1000ms", TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => - res && server.getLogManager().getLog(topicAndPartition).isDefined), 1000)) - var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) + TestUtils.waitUntilTrue(() => servers.forall(_.getLogManager().getLog(topicAndPartition).isDefined), + "Replicas for topic test not created.") + val leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) + val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get).last + follower.shutdown() // start topic deletion AdminUtils.deleteTopic(zkClient, topic) // start partition reassignment at the same time right after delete topic. In this case, reassignment will fail since @@ -243,52 +124,15 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentFailed; - }, 1000) + }, "Partition reassignment shouldn't complete.") val controllerId = ZkUtils.getController(zkClient) val controller = servers.filter(s => s.config.brokerId == controllerId).head assertFalse("Partition reassignment should fail", controller.kafkaController.controllerContext.partitionsBeingReassigned.contains(topicAndPartition)) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, 0) assertEquals("Partition should not be reassigned to 0, 1, 2", oldAssignedReplicas, assignedReplicas) - verifyTopicDeletion(topic, servers) - allServers.foreach(_.shutdown()) - } - - @Test - def testDeleteTopicDuringPartitionReassignment() { - val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) - val topic = "test" - val topicAndPartition = TopicAndPartition(topic, 0) - // create brokers - val allServers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) - val servers = allServers.filter(s => expectedReplicaAssignment(0).contains(s.config.brokerId)) - // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) - // wait until replica log is created on every broker - assertTrue("Replicas for topic test not created in 1000ms", TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => - res && server.getLogManager().getLog(topicAndPartition).isDefined), 1000)) - var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) - assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) - // start partition reassignment at the same time right before delete topic. In this case, reassignment will succeed - // reassign partition 0 - val newReplicas = Seq(1, 2, 3) - val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) - assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) - // start topic deletion - AdminUtils.deleteTopic(zkClient, topic) - // wait until reassignment is completed - TestUtils.waitUntilTrue(() => { - val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); - ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, - Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; - }, 1000) - val controllerId = ZkUtils.getController(zkClient) - val controller = servers.filter(s => s.config.brokerId == controllerId).head - assertFalse("Partition reassignment should complete", - controller.kafkaController.controllerContext.partitionsBeingReassigned.contains(topicAndPartition)) - val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, 0) - assertEquals("Partition should be reassigned to 1,2,3", newReplicas, assignedReplicas) - verifyTopicDeletion(topic, allServers) + follower.startup() + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) allServers.foreach(_.shutdown()) } @@ -296,17 +140,22 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { def testDeleteTopicDuringAddPartition() { val topic = "test" val servers = createTestTopicAndCluster(topic) - // add partitions to topic - val topicAndPartition = TopicAndPartition(topic, 0) + val leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) + assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) + val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get).last val newPartition = TopicAndPartition(topic, 1) - AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2") + follower.shutdown() + // add partitions to topic + AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2", false) // start topic deletion AdminUtils.deleteTopic(zkClient, topic) + follower.startup() // test if topic deletion is resumed - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // verify that new partition doesn't exist on any broker either - assertTrue("Replica logs not for new partition [test,1] not deleted after delete topic is complete", TestUtils.waitUntilTrue(() => - servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(newPartition).isEmpty), 1000)) + TestUtils.waitUntilTrue(() => + servers.forall(_.getLogManager().getLog(newPartition).isEmpty), + "Replica logs not for new partition [test,1] not deleted after delete topic is complete.") servers.foreach(_.shutdown()) } @@ -320,74 +169,114 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // add partitions to topic val newPartition = TopicAndPartition(topic, 1) AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2") - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // verify that new partition doesn't exist on any broker either assertTrue("Replica logs not deleted after delete topic is complete", - servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(newPartition).isEmpty)) + servers.forall(_.getLogManager().getLog(newPartition).isEmpty)) servers.foreach(_.shutdown()) } @Test def testRecreateTopicAfterDeletion() { - val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) + val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" val topicAndPartition = TopicAndPartition(topic, 0) val servers = createTestTopicAndCluster(topic) // start topic deletion AdminUtils.deleteTopic(zkClient, topic) - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // re-create topic on same replicas AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // wait until leader is elected val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) assertTrue("New leader should be elected after re-creating topic test", leaderIdOpt.isDefined) // check if all replica logs are created - assertTrue("Replicas for topic test not created in 1000ms", TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => - res && server.getLogManager().getLog(topicAndPartition).isDefined), 1000)) + TestUtils.waitUntilTrue(() => servers.forall(_.getLogManager().getLog(topicAndPartition).isDefined), + "Replicas for topic test not created.") servers.foreach(_.shutdown()) } @Test - def testTopicConfigChangesDuringDeleteTopic() { - val topic = "test" + def testDeleteNonExistingTopic() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic val servers = createTestTopicAndCluster(topic) - val topicConfigs = new Properties() - topicConfigs.put("segment.ms", "1000000") // start topic deletion - AdminUtils.deleteTopic(zkClient, topic) - verifyTopicDeletion(topic, servers) - // make topic config changes - try { - AdminUtils.changeTopicConfig(zkClient, topic, topicConfigs) - fail("Should fail with AdminOperationException for topic doesn't exist") - } catch { - case e: AdminOperationException => // expected - } + AdminUtils.deleteTopic(zkClient, "test2") + // verify delete topic path for test2 is removed from zookeeper + TestUtils.verifyTopicDeletion(zkClient, "test2", 1, servers) + // verify that topic test is untouched + TestUtils.waitUntilTrue(() => servers.forall(_.getLogManager().getLog(topicAndPartition).isDefined), + "Replicas for topic test not created") + // test the topic path exists + assertTrue("Topic test mistakenly deleted", ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) + // topic test should have a leader + val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + assertTrue("Leader should exist for topic test", leaderIdOpt.isDefined) + servers.foreach(_.shutdown()) + } + + @Test + def testDeleteTopicWithCleaner() { + val topicName = "test" + val topicAndPartition = TopicAndPartition(topicName, 0) + val topic = topicAndPartition.topic + + val brokerConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false) + brokerConfigs(0).setProperty("delete.topic.enable", "true") + brokerConfigs(0).setProperty("log.cleaner.enable","true") + brokerConfigs(0).setProperty("log.cleanup.policy","compact") + brokerConfigs(0).setProperty("log.segment.bytes","100") + brokerConfigs(0).setProperty("log.segment.delete.delay.ms","1000") + brokerConfigs(0).setProperty("log.cleaner.dedupe.buffer.size","1048577") + + val servers = createTestTopicAndCluster(topic,brokerConfigs) + + // for simplicity, we are validating cleaner offsets on a single broker + val server = servers(0) + val log = server.logManager.getLog(topicAndPartition).get + + // write to the topic to activate cleaner + writeDups(numKeys = 100, numDups = 3,log) + + // wait for cleaner to clean + server.logManager.cleaner.awaitCleaned(topicName,0,0) + + // delete topic + AdminUtils.deleteTopic(zkClient, "test") + TestUtils.verifyTopicDeletion(zkClient, "test", 1, servers) + servers.foreach(_.shutdown()) } private def createTestTopicAndCluster(topic: String): Seq[KafkaServer] = { - val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) + + val brokerConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false) + brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true") + ) + createTestTopicAndCluster(topic,brokerConfigs) + } + + private def createTestTopicAndCluster(topic: String, brokerConfigs: Seq[Properties]): Seq[KafkaServer] = { + val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topicAndPartition = TopicAndPartition(topic, 0) // create brokers - val servers = TestUtils.createBrokerConfigs(3).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // wait until replica log is created on every broker - assertTrue("Replicas for topic test not created in 1000ms", TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => - res && server.getLogManager().getLog(topicAndPartition).isDefined), 1000)) + TestUtils.waitUntilTrue(() => servers.forall(_.getLogManager().getLog(topicAndPartition).isDefined), + "Replicas for topic test not created") servers } - private def verifyTopicDeletion(topic: String, servers: Seq[KafkaServer]) { - val topicAndPartition = TopicAndPartition(topic, 0) - // wait until admin path for delete topic is deleted, signaling completion of topic deletion - assertTrue("Admin path /admin/delete_topic/test path not deleted in 1000ms even after a replica is restarted", - TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), 1000)) - assertTrue("Topic path /brokers/topics/test not deleted after /admin/delete_topic/test path is deleted", - TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)), 100)) - // ensure that logs from all replicas are deleted if delete topic is marked successful in zookeeper - assertTrue("Replica logs not deleted after delete topic is complete", - servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isEmpty)) + private def writeDups(numKeys: Int, numDups: Int, log: Log): Seq[(Int, Int)] = { + var counter = 0 + 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) + } } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala new file mode 100644 index 0000000000000..dcd69881445c2 --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -0,0 +1,102 @@ +/** + * 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 junit.framework.Assert._ +import org.junit.Test +import org.scalatest.junit.JUnit3Suite +import kafka.utils.Logging +import kafka.utils.TestUtils +import kafka.zk.ZooKeeperTestHarness +import kafka.admin.TopicCommand.TopicCommandOptions +import kafka.utils.ZkUtils +import kafka.coordinator.ConsumerCoordinator + +class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { + + @Test + def testConfigPreservationAcrossPartitionAlteration() { + val topic = "test" + val numPartitionsOriginal = 1 + val cleanupKey = "cleanup.policy" + val cleanupVal = "compact" + // create brokers + val brokers = List(0, 1, 2) + TestUtils.createBrokersInZk(zkClient, brokers) + // create the topic + val createOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString, + "--replication-factor", "1", + "--config", cleanupKey + "=" + cleanupVal, + "--topic", topic)) + TopicCommand.createTopic(zkClient, createOpts) + val props = AdminUtils.fetchTopicConfig(zkClient, topic) + assertTrue("Properties after creation don't contain " + cleanupKey, props.containsKey(cleanupKey)) + assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal)) + + // pre-create the topic config changes path to avoid a NoNodeException + ZkUtils.createPersistentPath(zkClient, ZkUtils.TopicConfigChangesPath) + + // modify the topic to add new partitions + val numPartitionsModified = 3 + val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, + "--config", cleanupKey + "=" + cleanupVal, + "--topic", topic)) + TopicCommand.alterTopic(zkClient, alterOpts) + val newProps = AdminUtils.fetchTopicConfig(zkClient, topic) + assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey)) + assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal)) + } + + @Test + def testTopicDeletion() { + val normalTopic = "test" + + val numPartitionsOriginal = 1 + + // create brokers + val brokers = List(0, 1, 2) + TestUtils.createBrokersInZk(zkClient, brokers) + + // create the NormalTopic + val createOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString, + "--replication-factor", "1", + "--topic", normalTopic)) + TopicCommand.createTopic(zkClient, createOpts) + + // delete the NormalTopic + val deleteOpts = new TopicCommandOptions(Array("--topic", normalTopic)) + val deletePath = ZkUtils.getDeleteTopicPath(normalTopic) + assertFalse("Delete path for topic shouldn't exist before deletion.", zkClient.exists(deletePath)) + TopicCommand.deleteTopic(zkClient, deleteOpts) + assertTrue("Delete path for topic should exist after deletion.", zkClient.exists(deletePath)) + + // create the offset topic + val createOffsetTopicOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString, + "--replication-factor", "1", + "--topic", ConsumerCoordinator.OffsetsTopicName)) + TopicCommand.createTopic(zkClient, createOffsetTopicOpts) + + // try to delete the OffsetManager.OffsetsTopicName and make sure it doesn't + val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", ConsumerCoordinator.OffsetsTopicName)) + val deleteOffsetTopicPath = ZkUtils.getDeleteTopicPath(ConsumerCoordinator.OffsetsTopicName) + assertFalse("Delete path for topic shouldn't exist before deletion.", zkClient.exists(deleteOffsetTopicPath)) + intercept[AdminOperationException] { + TopicCommand.deleteTopic(zkClient, deleteOffsetTopicOpts) + } + assertFalse("Delete path for topic shouldn't exist after deletion.", zkClient.exists(deleteOffsetTopicPath)) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index eb274d18c716f..5717165f23448 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -17,18 +17,25 @@ package kafka.api + +import kafka.cluster.{BrokerEndPoint, EndPoint, Broker} +import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError} +import kafka.common._ +import kafka.message.{Message, ByteBufferMessageSet} +import kafka.utils.SystemTime + +import kafka.controller.LeaderIsrAndControllerEpoch +import kafka.common.TopicAndPartition + +import java.nio.ByteBuffer + +import org.apache.kafka.common.protocol.SecurityProtocol import org.junit._ import org.scalatest.junit.JUnitSuite import junit.framework.Assert._ -import java.nio.ByteBuffer -import kafka.message.{Message, ByteBufferMessageSet} -import kafka.cluster.Broker -import collection.mutable._ -import kafka.common.{TopicAndPartition, ErrorMapping, OffsetMetadataAndError} -import kafka.controller.LeaderIsrAndControllerEpoch -object SerializationTestUtils{ +object SerializationTestUtils { private val topic1 = "test1" private val topic2 = "test2" private val leader1 = 0 @@ -54,7 +61,7 @@ object SerializationTestUtils{ private val partitionDataMessage3 = new ByteBufferMessageSet(new Message("fourth message".getBytes)) private val partitionDataProducerRequestArray = Array(partitionDataMessage0, partitionDataMessage1, partitionDataMessage2, partitionDataMessage3) - private val topicDataProducerRequest = { + val topicDataProducerRequest = { val groupedData = Array(topic1, topic2).flatMap(topic => partitionDataProducerRequestArray.zipWithIndex.map { @@ -75,21 +82,47 @@ 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 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) - private val partitionMetaData3 = new PartitionMetadata(3, Some(brokers.head), replicas = brokers, isr = brokers.tail.tail, errorCode = 3) + private val brokers = List(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> EndPoint("localhost", 1011, SecurityProtocol.PLAINTEXT))), + new Broker(1, Map(SecurityProtocol.PLAINTEXT -> EndPoint("localhost", 1012, SecurityProtocol.PLAINTEXT))), + new Broker(2, Map(SecurityProtocol.PLAINTEXT -> EndPoint("localhost", 1013, SecurityProtocol.PLAINTEXT)))) + private val brokerEndpoints = brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) + + private val partitionMetaData0 = new PartitionMetadata(0, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints, errorCode = 0) + private val partitionMetaData1 = new PartitionMetadata(1, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints.tail, errorCode = 1) + private val partitionMetaData2 = new PartitionMetadata(2, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints, errorCode = 2) + private val partitionMetaData3 = new PartitionMetadata(3, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints.tail.tail, errorCode = 3) private val partitionMetaDataSeq = Seq(partitionMetaData0, partitionMetaData1, partitionMetaData2, partitionMetaData3) private val topicmetaData1 = new TopicMetadata(topic1, partitionMetaDataSeq) private val topicmetaData2 = new TopicMetadata(topic2, partitionMetaDataSeq) + private val leaderAndIsr0 = new LeaderAndIsr(leader = brokers.head.id, isr = brokers.map(_.id)) + private val leaderAndIsr1 = new LeaderAndIsr(leader = brokers.head.id, isr = brokers.tail.map(_.id)) + private val leaderAndIsr2 = new LeaderAndIsr(leader = brokers.head.id, isr = brokers.map(_.id)) + private val leaderAndIsr3 = new LeaderAndIsr(leader = brokers.head.id, isr = brokers.tail.map(_.id)) + + private val leaderIsrAndControllerEpoch0 = new LeaderIsrAndControllerEpoch(leaderAndIsr0, controllerEpoch = 0) + private val leaderIsrAndControllerEpoch1 = new LeaderIsrAndControllerEpoch(leaderAndIsr1, controllerEpoch = 0) + private val leaderIsrAndControllerEpoch2 = new LeaderIsrAndControllerEpoch(leaderAndIsr2, controllerEpoch = 0) + private val leaderIsrAndControllerEpoch3 = new LeaderIsrAndControllerEpoch(leaderAndIsr3, controllerEpoch = 0) + + private val partitionStateInfo0 = new PartitionStateInfo(leaderIsrAndControllerEpoch0, brokers.map(_.id).toSet) + private val partitionStateInfo1 = new PartitionStateInfo(leaderIsrAndControllerEpoch1, brokers.map(_.id).toSet) + private val partitionStateInfo2 = new PartitionStateInfo(leaderIsrAndControllerEpoch2, brokers.map(_.id).toSet) + private val partitionStateInfo3 = new PartitionStateInfo(leaderIsrAndControllerEpoch3, brokers.map(_.id).toSet) + + private val updateMetadataRequestPartitionStateInfo = collection.immutable.Map( + TopicAndPartition(topic1,0) -> partitionStateInfo0, + TopicAndPartition(topic1,1) -> partitionStateInfo1, + TopicAndPartition(topic1,2) -> partitionStateInfo2, + TopicAndPartition(topic1,3) -> partitionStateInfo3 + ) + def createTestLeaderAndIsrRequest() : LeaderAndIsrRequest = { val leaderAndIsr1 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader1, 1, isr1, 1), 1) val leaderAndIsr2 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader2, 1, isr2, 2), 1) val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, isr1.toSet)), ((topic2, 0), PartitionStateInfo(leaderAndIsr2, isr2.toSet))) - new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 0, 1, 0, "") + new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[BrokerEndPoint](), 0, 1, 0, "") } def createTestLeaderAndIsrResponse() : LeaderAndIsrResponse = { @@ -143,23 +176,44 @@ object SerializationTestUtils{ } def createTestTopicMetadataResponse: TopicMetadataResponse = { - new TopicMetadataResponse(Seq(topicmetaData1, topicmetaData2), 1) + new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)).toSeq, 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 createTestOffsetCommitRequestV2: OffsetCommitRequest = { + new OffsetCommitRequest( + groupId = "group 1", + retentionMs = SystemTime.milliseconds, + requestInfo=collection.immutable.Map( + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata"), + TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata) )) } - def createTestOffsetCommitResponse: OffsetCommitResponse = { - new OffsetCommitResponse(collection.immutable.Map( - TopicAndPartition(topic1, 0) -> ErrorMapping.NoError, - TopicAndPartition(topic1, 1) -> ErrorMapping.UnknownTopicOrPartitionCode + def createTestOffsetCommitRequestV1: OffsetCommitRequest = { + new OffsetCommitRequest( + versionId = 1, + groupId = "group 1", + requestInfo = collection.immutable.Map( + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata", SystemTime.milliseconds), + TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata, SystemTime.milliseconds) )) } + def createTestOffsetCommitRequestV0: OffsetCommitRequest = { + new OffsetCommitRequest( + versionId = 0, + groupId = "group 1", + requestInfo = collection.immutable.Map( + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata"), + TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata) + )) + } + + def createTestOffsetCommitResponse: OffsetCommitResponse = { + new OffsetCommitResponse(collection.immutable.Map(TopicAndPartition(topic1, 0) -> ErrorMapping.NoError, + TopicAndPartition(topic1, 1) -> ErrorMapping.NoError)) + } + def createTestOffsetFetchRequest: OffsetFetchRequest = { new OffsetFetchRequest("group 1", Seq( TopicAndPartition(topic1, 0), @@ -170,11 +224,33 @@ object SerializationTestUtils{ 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) + TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) )) } + def createConsumerMetadataRequest: ConsumerMetadataRequest = { + ConsumerMetadataRequest("group 1", clientId = "client 1") + } + + def createConsumerMetadataResponse: ConsumerMetadataResponse = { + ConsumerMetadataResponse(Some(brokers.head.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), ErrorMapping.NoError, 0) + } + + def createUpdateMetadataRequest(versionId: Short): UpdateMetadataRequest = { + UpdateMetadataRequest( + versionId, + correlationId = 0, + clientId = "client1", + controllerId = 0, + controllerEpoch = 0, + partitionStateInfos = updateMetadataRequestPartitionStateInfo, + brokers.toSet + ) + } + + def createUpdateMetadataResponse: UpdateMetadataResponse = { + UpdateMetadataResponse( correlationId = 0, errorCode = 0) + } } class RequestResponseSerializationTest extends JUnitSuite { @@ -189,118 +265,43 @@ 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 offsetCommitRequestV0 = SerializationTestUtils.createTestOffsetCommitRequestV0 + private val offsetCommitRequestV1 = SerializationTestUtils.createTestOffsetCommitRequestV1 + private val offsetCommitRequestV2 = SerializationTestUtils.createTestOffsetCommitRequestV2 private val offsetCommitResponse = SerializationTestUtils.createTestOffsetCommitResponse private val offsetFetchRequest = SerializationTestUtils.createTestOffsetFetchRequest private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse + private val consumerMetadataRequest = SerializationTestUtils.createConsumerMetadataRequest + private val consumerMetadataResponse = SerializationTestUtils.createConsumerMetadataResponse + private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, 0) + private val updateMetadataRequestV0 = SerializationTestUtils.createUpdateMetadataRequest(0) + private val updateMetadataRequestV1 = SerializationTestUtils.createUpdateMetadataRequest(1) + private val updateMetdataResponse = SerializationTestUtils.createUpdateMetadataResponse @Test def testSerializationAndDeserialization() { - var buffer: ByteBuffer = ByteBuffer.allocate(leaderAndIsrRequest.sizeInBytes()) - leaderAndIsrRequest.writeTo(buffer) - buffer.rewind() - val deserializedLeaderAndIsrRequest = LeaderAndIsrRequest.readFrom(buffer) - assertEquals("The original and deserialzed leaderAndISRRequest should be the same", leaderAndIsrRequest, - deserializedLeaderAndIsrRequest) - - buffer = ByteBuffer.allocate(leaderAndIsrResponse.sizeInBytes()) - leaderAndIsrResponse.writeTo(buffer) - buffer.rewind() - val deserializedLeaderAndIsrResponse = LeaderAndIsrResponse.readFrom(buffer) - assertEquals("The original and deserialzed leaderAndISRResponse should be the same", leaderAndIsrResponse, - deserializedLeaderAndIsrResponse) - - buffer = ByteBuffer.allocate(stopReplicaRequest.sizeInBytes()) - stopReplicaRequest.writeTo(buffer) - buffer.rewind() - val deserializedStopReplicaRequest = StopReplicaRequest.readFrom(buffer) - assertEquals("The original and deserialzed stopReplicaRequest should be the same", stopReplicaRequest, - deserializedStopReplicaRequest) - - buffer = ByteBuffer.allocate(stopReplicaResponse.sizeInBytes()) - stopReplicaResponse.writeTo(buffer) - buffer.rewind() - val deserializedStopReplicaResponse = StopReplicaResponse.readFrom(buffer) - assertEquals("The original and deserialzed stopReplicaResponse should be the same", stopReplicaResponse, - deserializedStopReplicaResponse) - - buffer = ByteBuffer.allocate(producerRequest.sizeInBytes) - producerRequest.writeTo(buffer) - buffer.rewind() - val deserializedProducerRequest = ProducerRequest.readFrom(buffer) - assertEquals("The original and deserialzed producerRequest should be the same", producerRequest, - deserializedProducerRequest) - - buffer = ByteBuffer.allocate(producerResponse.sizeInBytes) - producerResponse.writeTo(buffer) - buffer.rewind() - val deserializedProducerResponse = ProducerResponse.readFrom(buffer) - assertEquals("The original and deserialzed producerResponse should be the same: [%s], [%s]".format(producerResponse, deserializedProducerResponse), producerResponse, - deserializedProducerResponse) - - buffer = ByteBuffer.allocate(fetchRequest.sizeInBytes) - fetchRequest.writeTo(buffer) - buffer.rewind() - val deserializedFetchRequest = FetchRequest.readFrom(buffer) - assertEquals("The original and deserialzed fetchRequest should be the same", fetchRequest, - deserializedFetchRequest) - - buffer = ByteBuffer.allocate(offsetRequest.sizeInBytes) - offsetRequest.writeTo(buffer) - buffer.rewind() - val deserializedOffsetRequest = OffsetRequest.readFrom(buffer) - assertEquals("The original and deserialzed offsetRequest should be the same", offsetRequest, - deserializedOffsetRequest) - - buffer = ByteBuffer.allocate(offsetResponse.sizeInBytes) - offsetResponse.writeTo(buffer) - buffer.rewind() - val deserializedOffsetResponse = OffsetResponse.readFrom(buffer) - assertEquals("The original and deserialzed offsetResponse should be the same", offsetResponse, - deserializedOffsetResponse) - - buffer = ByteBuffer.allocate(topicMetadataRequest.sizeInBytes()) - topicMetadataRequest.writeTo(buffer) - buffer.rewind() - val deserializedTopicMetadataRequest = TopicMetadataRequest.readFrom(buffer) - assertEquals("The original and deserialzed topicMetadataRequest should be the same", topicMetadataRequest, - deserializedTopicMetadataRequest) - - buffer = ByteBuffer.allocate(topicMetadataResponse.sizeInBytes) - topicMetadataResponse.writeTo(buffer) - buffer.rewind() - 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) + val requestsAndResponses = + collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse, stopReplicaRequest, + stopReplicaResponse, producerRequest, producerResponse, + fetchRequest, offsetRequest, offsetResponse, topicMetadataRequest, + topicMetadataResponse, + offsetCommitRequestV0, offsetCommitRequestV1, offsetCommitRequestV2, + offsetCommitResponse, offsetFetchRequest, offsetFetchResponse, + consumerMetadataRequest, consumerMetadataResponse, + updateMetadataRequestV0, updateMetadataRequestV1, updateMetdataResponse, + consumerMetadataResponseNoCoordinator) + + requestsAndResponses.foreach { original => + val buffer = ByteBuffer.allocate(original.sizeInBytes) + original.writeTo(buffer) + buffer.rewind() + val deserializer = original.getClass.getDeclaredMethod("readFrom", classOf[ByteBuffer]) + val deserialized = deserializer.invoke(null, buffer) + assertFalse("All serialized bytes in " + original.getClass.getSimpleName + " should have been consumed", + buffer.hasRemaining) + assertEquals("The original and deserialized for " + original.getClass.getSimpleName + " should be the same.", original, deserialized) + } } } diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala new file mode 100644 index 0000000000000..abe511fc1458b --- /dev/null +++ b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala @@ -0,0 +1,130 @@ +/* + * 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.cluster + +import java.nio.ByteBuffer + +import kafka.utils.Logging +import org.apache.kafka.common.protocol.SecurityProtocol +import org.junit.Test +import org.scalatest.junit.JUnit3Suite + +import scala.collection.mutable + +class BrokerEndPointTest extends JUnit3Suite with Logging { + + @Test + def testSerDe() = { + + val endpoint = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) + val listEndPoints = Map(SecurityProtocol.PLAINTEXT -> endpoint) + val origBroker = new Broker(1, listEndPoints) + val brokerBytes = ByteBuffer.allocate(origBroker.sizeInBytes) + + origBroker.writeTo(brokerBytes) + + val newBroker = Broker.readFrom(brokerBytes.flip().asInstanceOf[ByteBuffer]) + assert(origBroker == newBroker) + } + + @Test + def testHashAndEquals() = { + val endpoint1 = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) + val endpoint2 = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) + val endpoint3 = new EndPoint("myhost", 1111, SecurityProtocol.PLAINTEXT) + val endpoint4 = new EndPoint("other", 1111, SecurityProtocol.PLAINTEXT) + val broker1 = new Broker(1, Map(SecurityProtocol.PLAINTEXT -> endpoint1)) + val broker2 = new Broker(1, Map(SecurityProtocol.PLAINTEXT -> endpoint2)) + val broker3 = new Broker(2, Map(SecurityProtocol.PLAINTEXT -> endpoint3)) + val broker4 = new Broker(1, Map(SecurityProtocol.PLAINTEXT -> endpoint4)) + + assert(broker1 == broker2) + assert(broker1 != broker3) + assert(broker1 != broker4) + assert(broker1.hashCode() == broker2.hashCode()) + assert(broker1.hashCode() != broker3.hashCode()) + assert(broker1.hashCode() != broker4.hashCode()) + + val hashmap = new mutable.HashMap[Broker, Int]() + hashmap.put(broker1, 1) + assert(hashmap.getOrElse(broker1, -1) == 1) + } + + @Test + def testFromJSON() = { + val brokerInfoStr = "{\"version\":2," + + "\"host\":\"localhost\"," + + "\"port\":9092," + + "\"jmx_port\":9999," + + "\"timestamp\":\"1416974968782\"," + + "\"endpoints\":[\"PLAINTEXT://localhost:9092\"]}" + val broker = Broker.createBroker(1, brokerInfoStr) + assert(broker.id == 1) + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host == "localhost") + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port == 9092) + } + + @Test + def testFromOldJSON() = { + val brokerInfoStr = "{\"jmx_port\":-1,\"timestamp\":\"1420485325400\",\"host\":\"172.16.8.243\",\"version\":1,\"port\":9091}" + val broker = Broker.createBroker(1, brokerInfoStr) + assert(broker.id == 1) + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host == "172.16.8.243") + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port == 9091) + } + + @Test + def testBrokerEndpointFromURI() = { + var connectionString = "localhost:9092" + var endpoint = BrokerEndPoint.createBrokerEndPoint(1, connectionString) + assert(endpoint.host == "localhost") + assert(endpoint.port == 9092) + // also test for ipv6 + connectionString = "[::1]:9092" + endpoint = BrokerEndPoint.createBrokerEndPoint(1, connectionString) + assert(endpoint.host == "::1") + assert(endpoint.port == 9092) + } + + @Test + def testEndpointFromURI() = { + var connectionString = "PLAINTEXT://localhost:9092" + var endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == "localhost") + assert(endpoint.port == 9092) + assert(endpoint.connectionString == "PLAINTEXT://localhost:9092") + // also test for default bind + connectionString = "PLAINTEXT://:9092" + endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == null) + assert(endpoint.port == 9092) + assert(endpoint.connectionString == "PLAINTEXT://:9092") + // also test for ipv6 + connectionString = "PLAINTEXT://[::1]:9092" + endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == "::1") + assert(endpoint.port == 9092) + assert(endpoint.connectionString == "PLAINTEXT://[::1]:9092") + // test hostname + connectionString = "PLAINTEXT://MyHostname:9092" + endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == "MyHostname") + assert(endpoint.port == 9092) + assert(endpoint.connectionString == "PLAINTEXT://MyHostname:9092") + } +} diff --git a/core/src/test/scala/unit/kafka/common/ConfigTest.scala b/core/src/test/scala/unit/kafka/common/ConfigTest.scala index 74118f4cbf7fe..0aca9385bff09 100644 --- a/core/src/test/scala/unit/kafka/common/ConfigTest.scala +++ b/core/src/test/scala/unit/kafka/common/ConfigTest.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package unit.kafka.common +package kafka.common import junit.framework.Assert._ import collection.mutable.ArrayBuffer import org.junit.Test -import kafka.common.InvalidConfigException import kafka.producer.ProducerConfig import kafka.consumer.ConsumerConfig @@ -29,7 +28,7 @@ class ConfigTest { @Test def testInvalidClientIds() { val invalidClientIds = new ArrayBuffer[String]() - val badChars = Array('/', '\\', ',', '\0', ':', "\"", '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '=') + val badChars = Array('/', '\\', ',', '\u0000', ':', "\"", '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '=') for (weirdChar <- badChars) { invalidClientIds += "Is" + weirdChar + "illegal" } @@ -59,7 +58,7 @@ class ConfigTest { @Test def testInvalidGroupIds() { val invalidGroupIds = new ArrayBuffer[String]() - val badChars = Array('/', '\\', ',', '\0', ':', "\"", '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '=') + val badChars = Array('/', '\\', ',', '\u0000', ':', "\"", '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '=') for (weirdChar <- badChars) { invalidGroupIds += "Is" + weirdChar + "illegal" } diff --git a/core/src/test/scala/unit/kafka/common/TopicTest.scala b/core/src/test/scala/unit/kafka/common/TopicTest.scala index c8f8f4d8715da..79532c89c4157 100644 --- a/core/src/test/scala/unit/kafka/common/TopicTest.scala +++ b/core/src/test/scala/unit/kafka/common/TopicTest.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package unit.kafka.common +package kafka.common import junit.framework.Assert._ import collection.mutable.ArrayBuffer -import kafka.common.{Topic, InvalidTopicException} import org.junit.Test class TopicTest { @@ -32,7 +31,7 @@ class TopicTest { for (i <- 1 to 6) longName += longName invalidTopicNames += longName - val badChars = Array('/', '\\', ',', '\0', ':', "\"", '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '=') + val badChars = Array('/', '\\', ',', '\u0000', ':', "\"", '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '=') for (weirdChar <- badChars) { invalidTopicNames += "Is" + weirdChar + "illegal" } diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala old mode 100644 new mode 100755 index 9347ea60f2c02..db5302ff02851 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -18,9 +18,9 @@ package kafka.consumer +import java.util.Properties import java.util.concurrent._ import java.util.concurrent.atomic._ -import java.util.Properties import scala.collection._ import junit.framework.Assert._ @@ -28,41 +28,37 @@ import kafka.message._ import kafka.server._ import kafka.utils.TestUtils._ import kafka.utils._ -import kafka.admin.AdminUtils import org.junit.Test import kafka.serializer._ -import kafka.cluster.{Broker, Cluster} import org.scalatest.junit.JUnit3Suite import kafka.integration.KafkaServerTestHarness class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { val numNodes = 1 - val configs = - for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) { - override val zkConnect = TestZKUtils.zookeeperConnect - } + + def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps) + val messages = new mutable.HashMap[Int, Seq[Message]] val topic = "topic" val group = "group1" val consumer0 = "consumer0" val consumedOffset = 5 - val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) val queue = new LinkedBlockingQueue[FetchedDataChunk] - val topicInfos = configs.map(c => new PartitionTopicInfo(topic, - 0, - queue, - new AtomicLong(consumedOffset), - new AtomicLong(0), - new AtomicInteger(0), - "")) - val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0)) + var topicInfos: Seq[PartitionTopicInfo] = null + + def consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0)) override def setUp() { - super.setUp - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(configs.head.brokerId)), new Properties) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + super.setUp() + topicInfos = configs.map(c => new PartitionTopicInfo(topic, + 0, + queue, + new AtomicLong(consumedOffset), + new AtomicLong(0), + new AtomicInteger(0), + "")) + createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) } @Test @@ -83,9 +79,9 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { val receivedMessages = (0 until 5).map(i => iter.next.message).toList assertFalse(iter.hasNext) - assertEquals(1, queue.size) // This is only the shutdown command. + assertEquals(0, queue.size) // Shutdown command has been consumed. assertEquals(5, receivedMessages.size) - val unconsumed = messageSet.filter(_.offset >= consumedOffset).map(m => Utils.readString(m.message.payload)) + val unconsumed = messageSet.filter(_.offset >= consumedOffset).map(m => TestUtils.readString(m.message.payload)) assertEquals(unconsumed, receivedMessages) } diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala new file mode 100644 index 0000000000000..adf08010597b7 --- /dev/null +++ b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala @@ -0,0 +1,207 @@ +/** + * 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.consumer + +import org.scalatest.junit.JUnit3Suite +import org.easymock.EasyMock +import org.I0Itec.zkclient.ZkClient +import org.apache.zookeeper.data.Stat +import kafka.utils.{TestUtils, Logging, ZkUtils, Json} +import junit.framework.Assert._ +import kafka.common.TopicAndPartition +import kafka.consumer.PartitionAssignorTest.StaticSubscriptionInfo +import kafka.consumer.PartitionAssignorTest.Scenario +import kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo + +class PartitionAssignorTest extends JUnit3Suite with Logging { + + def testRoundRobinPartitionAssignor() { + val assignor = new RoundRobinAssignor + + /** various scenarios with only wildcard consumers */ + (1 to PartitionAssignorTest.TestCaseCount).foreach (testCase => { + val consumerCount = 1.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxConsumerCount + 1)) + val topicCount = PartitionAssignorTest.MinTopicCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxTopicCount + 1)) + + val topicPartitionCounts = Map((1 to topicCount).map(topic => { + ("topic-" + topic, PartitionAssignorTest.MinPartitionCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxPartitionCount))) + }).toSeq:_*) + + val subscriptions = Map((1 to consumerCount).map(consumer => { + val streamCount = 1.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxStreamCount + 1)) + ("g1c" + consumer, WildcardSubscriptionInfo(streamCount, ".*", isWhitelist = true)) + }).toSeq:_*) + val scenario = Scenario("g1", topicPartitionCounts, subscriptions) + val zkClient = PartitionAssignorTest.setupZkClientMock(scenario) + EasyMock.replay(zkClient) + PartitionAssignorTest.assignAndVerify(scenario, assignor, zkClient, verifyAssignmentIsUniform = true) + }) + } + + def testRangePartitionAssignor() { + val assignor = new RangeAssignor + (1 to PartitionAssignorTest.TestCaseCount).foreach (testCase => { + val consumerCount = 1.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxConsumerCount + 1)) + val topicCount = PartitionAssignorTest.MinTopicCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxTopicCount + 1)) + + val topicPartitionCounts = Map((1 to topicCount).map(topic => { + ("topic-" + topic, PartitionAssignorTest.MinPartitionCount.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxPartitionCount))) + }).toSeq:_*) + + val subscriptions = Map((1 to consumerCount).map(consumer => { + val streamCounts = Map((1 to topicCount).map(topic => { + val streamCount = 1.max(TestUtils.random.nextInt(PartitionAssignorTest.MaxStreamCount + 1)) + ("topic-" + topic, streamCount) + }).toSeq:_*) + ("g1c" + consumer, StaticSubscriptionInfo(streamCounts)) + }).toSeq:_*) + val scenario = Scenario("g1", topicPartitionCounts, subscriptions) + val zkClient = PartitionAssignorTest.setupZkClientMock(scenario) + EasyMock.replay(zkClient) + + PartitionAssignorTest.assignAndVerify(scenario, assignor, zkClient) + }) + } +} + +private object PartitionAssignorTest extends Logging { + + private val TestCaseCount = 3 + private val MaxConsumerCount = 10 + private val MaxStreamCount = 8 + private val MaxTopicCount = 100 + private val MinTopicCount = 0 + private val MaxPartitionCount = 120 + private val MinPartitionCount = 8 + + private trait SubscriptionInfo { + def registrationString: String + } + + private case class StaticSubscriptionInfo(streamCounts: Map[String, Int]) extends SubscriptionInfo { + def registrationString = + Json.encode(Map("version" -> 1, + "subscription" -> streamCounts, + "pattern" -> "static", + "timestamp" -> 1234.toString)) + + override def toString = { + "Stream counts: " + streamCounts + } + } + + private case class WildcardSubscriptionInfo(streamCount: Int, regex: String, isWhitelist: Boolean) + extends SubscriptionInfo { + def registrationString = + Json.encode(Map("version" -> 1, + "subscription" -> Map(regex -> streamCount), + "pattern" -> (if (isWhitelist) "white_list" else "black_list"))) + + override def toString = { + "\"%s\":%d (%s)".format(regex, streamCount, if (isWhitelist) "whitelist" else "blacklist") + } + } + + private case class Scenario(group: String, + topicPartitionCounts: Map[String, Int], + /* consumerId -> SubscriptionInfo */ + subscriptions: Map[String, SubscriptionInfo]) { + override def toString = { + "\n" + + "Group : %s\n".format(group) + + "Topic partition counts : %s\n".format(topicPartitionCounts) + + "Consumer subscriptions : %s\n".format(subscriptions) + } + } + + private def setupZkClientMock(scenario: Scenario) = { + val consumers = java.util.Arrays.asList(scenario.subscriptions.keys.toSeq:_*) + + val zkClient = EasyMock.createStrictMock(classOf[ZkClient]) + EasyMock.checkOrder(zkClient, false) + + EasyMock.expect(zkClient.getChildren("/consumers/%s/ids".format(scenario.group))).andReturn(consumers) + EasyMock.expectLastCall().anyTimes() + + scenario.subscriptions.foreach { case(consumerId, subscriptionInfo) => + EasyMock.expect(zkClient.readData("/consumers/%s/ids/%s".format(scenario.group, consumerId), new Stat())) + .andReturn(subscriptionInfo.registrationString) + EasyMock.expectLastCall().anyTimes() + } + + scenario.topicPartitionCounts.foreach { case(topic, partitionCount) => + val replicaAssignment = Map((0 until partitionCount).map(partition => (partition.toString, Seq(0))):_*) + EasyMock.expect(zkClient.readData("/brokers/topics/%s".format(topic), new Stat())) + .andReturn(ZkUtils.replicaAssignmentZkData(replicaAssignment)) + EasyMock.expectLastCall().anyTimes() + } + + EasyMock.expect(zkClient.getChildren("/brokers/topics")).andReturn( + java.util.Arrays.asList(scenario.topicPartitionCounts.keys.toSeq:_*)) + EasyMock.expectLastCall().anyTimes() + + zkClient + } + + private def assignAndVerify(scenario: Scenario, assignor: PartitionAssignor, zkClient: ZkClient, + verifyAssignmentIsUniform: Boolean = false) { + val assignments = scenario.subscriptions.map{ case(consumer, subscription) => + val ctx = new AssignmentContext("g1", consumer, excludeInternalTopics = true, zkClient) + assignor.assign(ctx).get(consumer) + } + + // check for uniqueness (i.e., any partition should be assigned to exactly one consumer stream) + val globalAssignment = collection.mutable.Map[TopicAndPartition, ConsumerThreadId]() + assignments.foreach(assignment => { + assignment.foreach { case(topicPartition, owner) => + val previousOwnerOpt = globalAssignment.put(topicPartition, owner) + assertTrue("Scenario %s: %s is assigned to two owners.".format(scenario, topicPartition), previousOwnerOpt.isEmpty) + } + }) + + // check for coverage (i.e., all given partitions are owned) + val assignedPartitions = globalAssignment.keySet + val givenPartitions = scenario.topicPartitionCounts.flatMap{ case (topic, partitionCount) => + (0 until partitionCount).map(partition => TopicAndPartition(topic, partition)) + }.toSet + assertTrue("Scenario %s: the list of given partitions and assigned partitions are different.".format(scenario), + givenPartitions == assignedPartitions) + + // check for uniform assignment + if (verifyAssignmentIsUniform) { + val partitionCountForStream = partitionCountPerStream(globalAssignment) + if (partitionCountForStream.nonEmpty) { + val maxCount = partitionCountForStream.valuesIterator.max + val minCount = partitionCountForStream.valuesIterator.min + assertTrue("Scenario %s: assignment is not uniform (partition counts per stream are in the range [%d, %d])" + .format(scenario, minCount, maxCount), (maxCount - minCount) <= 1) + } + } + } + + /** For each consumer stream, count the number of partitions that it owns. */ + private def partitionCountPerStream(assignment: collection.Map[TopicAndPartition, ConsumerThreadId]) = { + val ownedCounts = collection.mutable.Map[ConsumerThreadId, Int]() + assignment.foreach { case (topicPartition, owner) => + val updatedCount = ownedCounts.getOrElse(owner, 0) + 1 + ownedCounts.put(owner, updatedCount) + } + ownedCounts + } +} + diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index cf2724bb68d39..4b326d090c948 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -21,6 +21,8 @@ package kafka.consumer import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test +import kafka.server.OffsetManager +import kafka.coordinator.ConsumerCoordinator class TopicFilterTest extends JUnitSuite { @@ -29,19 +31,56 @@ class TopicFilterTest extends JUnitSuite { def testWhitelists() { val topicFilter1 = new Whitelist("white1,white2") - assertTrue(topicFilter1.isTopicAllowed("white2")) - assertFalse(topicFilter1.isTopicAllowed("black1")) + assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = false)) + assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) + assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) val topicFilter2 = new Whitelist(".+") - assertTrue(topicFilter2.isTopicAllowed("alltopics")) - + assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true)) + assertFalse(topicFilter2.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter2.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = false)) + val topicFilter3 = new Whitelist("white_listed-topic.+") - assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1")) - assertFalse(topicFilter3.isTopicAllowed("black1")) + assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true)) + assertFalse(topicFilter3.isTopicAllowed("black1", excludeInternalTopics = true)) + + val topicFilter4 = new Whitelist("test-(?!bad\\b)[\\w]+") + assertTrue(topicFilter4.isTopicAllowed("test-good", excludeInternalTopics = true)) + assertFalse(topicFilter4.isTopicAllowed("test-bad", excludeInternalTopics = true)) } @Test def testBlacklists() { val topicFilter1 = new Blacklist("black1") + assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = false)) + assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) + assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) + + assertFalse(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = false)) } + + @Test + def testWildcardTopicCountGetTopicCountMapEscapeJson() { + def getTopicCountMapKey(regex: String): String = { + val topicCount = new WildcardTopicCount(null, "consumerId", new Whitelist(regex), 1, true) + topicCount.getTopicCountMap.head._1 + } + //lets make sure that the JSON strings are escaping as we expect + //if they are not then when they get saved to zookeeper and read back out they will be broken on parse + assertEquals("-\\\"-", getTopicCountMapKey("-\"-")) + assertEquals("-\\\\-", getTopicCountMapKey("-\\-")) + assertEquals("-\\/-", getTopicCountMapKey("-/-")) + assertEquals("-\\\\b-", getTopicCountMapKey("-\\b-")) + assertEquals("-\\\\f-", getTopicCountMapKey("-\\f-")) + assertEquals("-\\\\n-", getTopicCountMapKey("-\\n-")) + assertEquals("-\\\\r-", getTopicCountMapKey("-\\r-")) + assertEquals("-\\\\t-", getTopicCountMapKey("-\\t-")) + assertEquals("-\\\\u0000-", getTopicCountMapKey("-\\u0000-")) + assertEquals("-\\\\u001f-", getTopicCountMapKey("-\\u001f-")) + assertEquals("-\\\\u007f-", getTopicCountMapKey("-\\u007f-")) + assertEquals("-\\\\u009f-", getTopicCountMapKey("-\\u009f-")) + } } \ 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 8fe7259c9abd2..359b0f5d14f82 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -1,4 +1,3 @@ - /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -6,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 @@ -18,35 +17,36 @@ package kafka.consumer +import java.util.{Collections, Properties} + import junit.framework.Assert._ +import kafka.common.MessageStreamsExistException import kafka.integration.KafkaServerTestHarness -import kafka.server._ -import scala.collection._ -import org.scalatest.junit.JUnit3Suite +import kafka.javaapi.consumer.ConsumerRebalanceListener import kafka.message._ import kafka.serializer._ -import kafka.admin.AdminUtils -import org.I0Itec.zkclient.ZkClient -import kafka.utils._ -import kafka.producer.{ProducerConfig, KeyedMessage, Producer} -import java.util.{Collections, Properties} -import org.apache.log4j.{Logger, Level} +import kafka.server._ import kafka.utils.TestUtils._ +import kafka.utils._ +import org.I0Itec.zkclient.ZkClient +import org.apache.log4j.{Level, Logger} +import org.scalatest.junit.JUnit3Suite + +import scala.collection._ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging { val RebalanceBackoffMs = 5000 var dirs : ZKGroupTopicDirs = null - val zookeeperConnect = TestZKUtils.zookeeperConnect val numNodes = 2 val numParts = 2 val topic = "topic1" - val configs = - for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) { - override val zkConnect = zookeeperConnect - override val numPartitions = numParts - } + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) + + override def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect) + .map(KafkaConfig.fromProps(_, overridingProps)) + val group = "group1" val consumer0 = "consumer0" val consumer1 = "consumer1" @@ -79,7 +79,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar // also the iterator should support re-entrant, so loop it twice for (i <- 0 until 2) { try { - getMessages(nMessages*2, topicMessageStreams0) + getMessages(topicMessageStreams0, nMessages * 2) fail("should get an exception") } catch { case e: ConsumerTimeoutException => // this is ok @@ -90,22 +90,22 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar zkConsumerConnector0.shutdown // send some messages to each broker - val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) + val sentMessages1 = sendMessages(servers, topic, nMessages, 0) ++ + sendMessages(servers, topic, nMessages, 1) // wait to make sure the topic and partition have a leader for the successful case - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) // create a consumer val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) + val receivedMessages1 = getMessages(topicMessageStreams1, nMessages * 2) assertEquals(sentMessages1.sorted, receivedMessages1.sorted) // also check partition ownership @@ -115,7 +115,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar assertEquals(expected_1, actual_1) // commit consumed offsets - zkConsumerConnector1.commitOffsets + zkConsumerConnector1.commitOffsets(true) // create a consumer val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { @@ -124,13 +124,13 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) // send some messages to each broker - val sentMessages2 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) + val sentMessages2 = sendMessages(servers, topic, nMessages, 0) ++ + sendMessages(servers, topic, nMessages, 1) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - val receivedMessages2 = getMessages(nMessages, topicMessageStreams1) ++ getMessages(nMessages, topicMessageStreams2) + val receivedMessages2 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages) assertEquals(sentMessages2.sorted, receivedMessages2.sorted) // also check partition ownership @@ -145,19 +145,27 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) // send some messages to each broker - val sentMessages3 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) + val sentMessages3 = sendMessages(servers, topic, nMessages, 0) ++ + sendMessages(servers, topic, nMessages, 1) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - val receivedMessages3 = getMessages(nMessages, topicMessageStreams1) ++ getMessages(nMessages, topicMessageStreams2) + val receivedMessages3 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages) assertEquals(sentMessages3.sorted, receivedMessages3.sorted) // also check partition ownership val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir) assertEquals(expected_2, actual_3) + // call createMesssageStreams twice should throw MessageStreamsExistException + try { + val topicMessageStreams4 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) + fail("Should fail with MessageStreamsExistException") + } catch { + case e: MessageStreamsExistException => // expected + } + zkConsumerConnector1.shutdown zkConsumerConnector2.shutdown zkConsumerConnector3.shutdown @@ -165,26 +173,27 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.ERROR) } + def testCompression() { val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) // send some messages to each broker - val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) + val sentMessages1 = sendMessages(servers, topic, nMessages, 0, GZIPCompressionCodec) ++ + sendMessages(servers, topic, nMessages, 1, GZIPCompressionCodec) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) // create a consumer val consumerConfig1 = new ConsumerConfig( TestUtils.createConsumerProperties(zkConnect, group, consumer1)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) + val receivedMessages1 = getMessages(topicMessageStreams1, nMessages * 2) assertEquals(sentMessages1.sorted, receivedMessages1.sorted) // also check partition ownership @@ -194,7 +203,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar assertEquals(expected_1, actual_1) // commit consumed offsets - zkConsumerConnector1.commitOffsets + zkConsumerConnector1.commitOffsets(true) // create a consumer val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { @@ -203,13 +212,13 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) // send some messages to each broker - val sentMessages2 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) + val sentMessages2 = sendMessages(servers, topic, nMessages, 0, GZIPCompressionCodec) ++ + sendMessages(servers, topic, nMessages, 1, GZIPCompressionCodec) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - val receivedMessages2 = getMessages(nMessages, topicMessageStreams1) ++ getMessages(nMessages, topicMessageStreams2) + val receivedMessages2 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages) assertEquals(sentMessages2.sorted, receivedMessages2.sorted) // also check partition ownership @@ -224,13 +233,13 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int](), new StringDecoder(), new StringDecoder()) // send some messages to each broker - val sentMessages3 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) + val sentMessages3 = sendMessages(servers, topic, nMessages, 0, GZIPCompressionCodec) ++ + sendMessages(servers, topic, nMessages, 1, GZIPCompressionCodec) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) - val receivedMessages3 = getMessages(nMessages, topicMessageStreams1) ++ getMessages(nMessages, topicMessageStreams2) + val receivedMessages3 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages) assertEquals(sentMessages3.sorted, receivedMessages3.sorted) // also check partition ownership @@ -246,16 +255,16 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar def testCompressionSetConsumption() { // send some messages to each broker - val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, 200, DefaultCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, 200, DefaultCompressionCodec) + val sentMessages = sendMessages(servers, topic, 200, 0, DefaultCompressionCodec) ++ + sendMessages(servers, topic, 200, 1, DefaultCompressionCodec) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - val receivedMessages = getMessages(400, topicMessageStreams1) + val receivedMessages = getMessages(topicMessageStreams1, 400) assertEquals(sentMessages.sorted, receivedMessages.sorted) // also check partition ownership @@ -272,16 +281,16 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.FATAL) // send some messages to each broker - val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, NoCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, NoCompressionCodec) + val sentMessages = sendMessages(servers, topic, nMessages, 0, NoCompressionCodec) ++ + sendMessages(servers, topic, nMessages, 1, NoCompressionCodec) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1, 1000) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) val zkConsumerConnector = new ZookeeperConsumerConnector(consumerConfig, true) @@ -307,15 +316,13 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar } def testLeaderSelectionForPartition() { - val zkClient = new ZkClient(zookeeperConnect, 6000, 30000, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(zkConnect, 6000, 30000) // create topic topic1 with 1 partition on broker 0 - AdminUtils.createTopic(zkClient, topic, 1, 1) + createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers) // send some messages to each broker - val sentMessages1 = sendMessages(configs.head, nMessages, "batch1", NoCompressionCodec, 1) - - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) + val sentMessages1 = sendMessages(servers, topic, nMessages) // create a consumer val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) @@ -333,89 +340,102 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val expected_1 = List( ("0", "group1_consumer1-0")) assertEquals(expected_1, actual_1) - val receivedMessages1 = getMessages(nMessages, topicMessageStreams1) + val receivedMessages1 = getMessages(topicMessageStreams1, nMessages) assertEquals(sentMessages1, receivedMessages1) zkConsumerConnector1.shutdown() zkClient.close() } - def sendMessagesToBrokerPartition(config: KafkaConfig, - topic: String, - partition: Int, - numMessages: Int, - compression: CompressionCodec = NoCompressionCodec): List[String] = { - val header = "test-%d-%d".format(config.brokerId, partition) - val props = new Properties() - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) - props.put("partitioner.class", "kafka.utils.FixedValuePartitioner") - props.put("compression.codec", compression.codec.toString) - props.put("key.serializer.class", classOf[IntEncoder].getName.toString) - props.put("serializer.class", classOf[StringEncoder].getName.toString) - val producer: Producer[Int, String] = new Producer[Int, String](new ProducerConfig(props)) - val ms = 0.until(numMessages).map(x => header + config.brokerId + "-" + partition + "-" + x) - producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) - debug("Sent %d messages to broker %d for partition [%s,%d]".format(ms.size, config.brokerId, topic, partition)) - producer.close() - ms.toList - } + def testConsumerRebalanceListener() { + // Send messages to create topic + sendMessages(servers, topic, nMessages, 0) + sendMessages(servers, topic, nMessages, 1) - def sendMessages(config: KafkaConfig, - messagesPerNode: Int, - header: String, - compression: CompressionCodec, - numParts: Int): List[String]= { - var messages: List[String] = Nil - val props = new Properties() - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) - props.put("partitioner.class", "kafka.utils.FixedValuePartitioner") - props.put("key.serializer.class", classOf[IntEncoder].getName.toString) - props.put("serializer.class", classOf[StringEncoder].getName) - val producer: Producer[Int, String] = new Producer[Int, String](new ProducerConfig(props)) - for (partition <- 0 until numParts) { - val ms = 0.until(messagesPerNode).map(x => header + config.brokerId + "-" + partition + "-" + x) - producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) - messages ++= ms - debug("Sent %d messages to broker %d for partition [%s,%d]".format(ms.size, config.brokerId, topic, partition)) - } - producer.close() - messages - } + val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) + val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) + // Register consumer rebalance listener + val rebalanceListener1 = new TestConsumerRebalanceListener() + zkConsumerConnector1.setConsumerRebalanceListener(rebalanceListener1) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - def sendMessages(messagesPerNode: Int, header: String, compression: CompressionCodec = NoCompressionCodec): List[String]= { - var messages: List[String] = Nil - for(conf <- configs) - messages ++= sendMessages(conf, messagesPerNode, header, compression, numParts) - messages - } + // Check if rebalance listener is fired + assertEquals(true, rebalanceListener1.beforeReleasingPartitionsCalled) + assertEquals(true, rebalanceListener1.beforeStartingFetchersCalled) + assertEquals(null, rebalanceListener1.partitionOwnership.get(topic)) + // Check if partition assignment in rebalance listener is correct + assertEquals("group1_consumer1", rebalanceListener1.globalPartitionOwnership.get(topic).get(0).consumer) + assertEquals("group1_consumer1", rebalanceListener1.globalPartitionOwnership.get(topic).get(1).consumer) + assertEquals(0, rebalanceListener1.globalPartitionOwnership.get(topic).get(0).threadId) + assertEquals(0, rebalanceListener1.globalPartitionOwnership.get(topic).get(1).threadId) + assertEquals("group1_consumer1", rebalanceListener1.consumerId) + // reset the flag + rebalanceListener1.beforeReleasingPartitionsCalled = false + rebalanceListener1.beforeStartingFetchersCalled = false - def getMessages(nMessagesPerThread: Int, - topicMessageStreams: Map[String,List[KafkaStream[String, String]]]): List[String]= { - var messages: List[String] = Nil - for((topic, messageStreams) <- topicMessageStreams) { - for (messageStream <- messageStreams) { - val iterator = messageStream.iterator - for(i <- 0 until nMessagesPerThread) { - assertTrue(iterator.hasNext) - val message = iterator.next.message - messages ::= message - debug("received message: " + message) - } - } - } - messages.reverse + val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_1 = List(("0", "group1_consumer1-0"), + ("1", "group1_consumer1-0")) + assertEquals(expected_1, actual_1) + + val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) + val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) + // Register consumer rebalance listener + val rebalanceListener2 = new TestConsumerRebalanceListener() + zkConsumerConnector2.setConsumerRebalanceListener(rebalanceListener2) + val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) + + // Consume messages from consumer 1 to make sure it has finished rebalance + getMessages(topicMessageStreams1, nMessages) + + val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_2 = List(("0", "group1_consumer1-0"), + ("1", "group1_consumer2-0")) + assertEquals(expected_2, actual_2) + + // Check if rebalance listener is fired + assertEquals(true, rebalanceListener1.beforeReleasingPartitionsCalled) + assertEquals(true, rebalanceListener1.beforeStartingFetchersCalled) + assertEquals(Set[Int](0, 1), rebalanceListener1.partitionOwnership.get(topic)) + // Check if global partition ownership in rebalance listener is correct + assertEquals("group1_consumer1", rebalanceListener1.globalPartitionOwnership.get(topic).get(0).consumer) + assertEquals("group1_consumer2", rebalanceListener1.globalPartitionOwnership.get(topic).get(1).consumer) + assertEquals(0, rebalanceListener1.globalPartitionOwnership.get(topic).get(0).threadId) + assertEquals(0, rebalanceListener1.globalPartitionOwnership.get(topic).get(1).threadId) + assertEquals("group1_consumer1", rebalanceListener1.consumerId) + assertEquals("group1_consumer2", rebalanceListener2.consumerId) + assertEquals(rebalanceListener1.globalPartitionOwnership, rebalanceListener2.globalPartitionOwnership) + zkConsumerConnector1.shutdown() + zkConsumerConnector2.shutdown() } def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = { val children = zkClient.getChildren(path) Collections.sort(children) val childrenAsSeq : Seq[java.lang.String] = { - import JavaConversions._ + import scala.collection.JavaConversions._ children.toSeq } childrenAsSeq.map(partition => (partition, zkClient.readData(path + "/" + partition).asInstanceOf[String])) } -} + private class TestConsumerRebalanceListener extends ConsumerRebalanceListener { + var beforeReleasingPartitionsCalled: Boolean = false + var beforeStartingFetchersCalled: Boolean = false + var consumerId: String = ""; + var partitionOwnership: java.util.Map[String, java.util.Set[java.lang.Integer]] = null + var globalPartitionOwnership: java.util.Map[String, java.util.Map[java.lang.Integer, ConsumerThreadId]] = null + + override def beforeReleasingPartitions(partitionOwnership: java.util.Map[String, java.util.Set[java.lang.Integer]]) { + beforeReleasingPartitionsCalled = true + this.partitionOwnership = partitionOwnership + } + override def beforeStartingFetchers(consumerId: String, globalPartitionOwnership: java.util.Map[String, java.util.Map[java.lang.Integer, ConsumerThreadId]]) { + beforeStartingFetchersCalled = true + this.consumerId = consumerId + this.globalPartitionOwnership = globalPartitionOwnership + } + } +} diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala new file mode 100644 index 0000000000000..87a5330e716b9 --- /dev/null +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala @@ -0,0 +1,327 @@ +/** + * 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.coordinator + + +import java.util.concurrent.TimeUnit + +import junit.framework.Assert._ +import kafka.common.TopicAndPartition +import kafka.server.{OffsetManager, ReplicaManager, KafkaConfig} +import kafka.utils.{KafkaScheduler, TestUtils} +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.JoinGroupRequest +import org.easymock.EasyMock +import org.junit.{After, Before, Test} +import org.scalatest.junit.JUnitSuite + +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, Future, Promise} + +/** + * Test ConsumerCoordinator responses + */ +class ConsumerCoordinatorResponseTest extends JUnitSuite { + type JoinGroupCallbackParams = (Set[TopicAndPartition], String, Int, Short) + type JoinGroupCallback = (Set[TopicAndPartition], String, Int, Short) => Unit + type HeartbeatCallbackParams = Short + type HeartbeatCallback = Short => Unit + + val ConsumerMinSessionTimeout = 10 + val ConsumerMaxSessionTimeout = 100 + val DefaultSessionTimeout = 20 + var consumerCoordinator: ConsumerCoordinator = null + var offsetManager : OffsetManager = null + + @Before + def setUp() { + val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") + props.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) + props.setProperty(KafkaConfig.ConsumerMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) + offsetManager = EasyMock.createStrictMock(classOf[OffsetManager]) + consumerCoordinator = ConsumerCoordinator.create(KafkaConfig.fromProps(props), null, offsetManager) + consumerCoordinator.startup() + } + + @After + def tearDown() { + EasyMock.reset(offsetManager) + consumerCoordinator.shutdown() + } + + @Test + def testJoinGroupWrongCoordinator() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = false) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupUnknownPartitionAssignmentStrategy() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "foo" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupSessionTimeoutTooSmall() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, ConsumerMinSessionTimeout - 1, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupSessionTimeoutTooLarge() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, ConsumerMaxSessionTimeout + 1, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupUnknownConsumerNewGroup() { + val groupId = "groupId" + val consumerId = "consumerId" + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, joinGroupErrorCode) + } + + @Test + def testValidJoinGroup() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + } + + @Test + def testJoinGroupInconsistentPartitionAssignmentStrategy() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + val otherPartitionAssignmentStrategy = "roundrobin" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, otherPartitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val otherJoinGroupErrorCode = otherJoinGroupResult._4 + assertEquals(Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code, otherJoinGroupErrorCode) + } + + @Test + def testJoinGroupUnknownConsumerExistingGroup() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = "consumerId" + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val otherJoinGroupErrorCode = otherJoinGroupResult._4 + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, otherJoinGroupErrorCode) + } + + @Test + def testHeartbeatWrongCoordinator() { + val groupId = "groupId" + val consumerId = "consumerId" + + val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = false) + assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.code, heartbeatResult) + } + + @Test + def testHeartbeatUnknownGroup() { + val groupId = "groupId" + val consumerId = "consumerId" + + val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = true) + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, heartbeatResult) + } + + @Test + def testHeartbeatUnknownConsumerExistingGroup() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = "consumerId" + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, otherConsumerId, 1, isCoordinatorForGroup = true) + assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, heartbeatResult) + } + + @Test + def testHeartbeatIllegalGeneration() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val assignedConsumerId = joinGroupResult._2 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, 2, isCoordinatorForGroup = true) + assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult) + } + + @Test + def testValidHeartbeat() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val assignedConsumerId = joinGroupResult._2 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1, isCoordinatorForGroup = true) + assertEquals(Errors.NONE.code, heartbeatResult) + } + + @Test + def testHeartbeatDuringRebalanceCausesIllegalGeneration() { + val groupId = "groupId" + val partitionAssignmentStrategy = "range" + + // First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts) + val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy, + 100, isCoordinatorForGroup = true) + val assignedConsumerId = joinGroupResult._2 + val initialGenerationId = joinGroupResult._3 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + // Then join with a new consumer to trigger a rebalance + EasyMock.reset(offsetManager) + sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy, + DefaultSessionTimeout, isCoordinatorForGroup = true) + + // We should be in the middle of a rebalance, so the heartbeat should return illegal generation + EasyMock.reset(offsetManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true) + assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult) + } + + @Test + def testGenerationIdIncrementsOnRebalance() { + val groupId = "groupId" + val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val otherConsumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID + val partitionAssignmentStrategy = "range" + + val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val initialGenerationId = joinGroupResult._3 + val joinGroupErrorCode = joinGroupResult._4 + assertEquals(1, initialGenerationId) + assertEquals(Errors.NONE.code, joinGroupErrorCode) + + EasyMock.reset(offsetManager) + val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true) + val nextGenerationId = otherJoinGroupResult._3 + val otherJoinGroupErrorCode = otherJoinGroupResult._4 + assertEquals(2, nextGenerationId) + assertEquals(Errors.NONE.code, otherJoinGroupErrorCode) + } + + private def setupJoinGroupCallback: (Future[JoinGroupCallbackParams], JoinGroupCallback) = { + val responsePromise = Promise[JoinGroupCallbackParams] + val responseFuture = responsePromise.future + val responseCallback: JoinGroupCallback = (partitions, consumerId, generationId, errorCode) => + responsePromise.success((partitions, consumerId, generationId, errorCode)) + (responseFuture, responseCallback) + } + + private def setupHeartbeatCallback: (Future[HeartbeatCallbackParams], HeartbeatCallback) = { + val responsePromise = Promise[HeartbeatCallbackParams] + val responseFuture = responsePromise.future + val responseCallback: HeartbeatCallback = errorCode => responsePromise.success(errorCode) + (responseFuture, responseCallback) + } + + private def sendJoinGroup(groupId: String, + consumerId: String, + partitionAssignmentStrategy: String, + sessionTimeout: Int, + isCoordinatorForGroup: Boolean): Future[JoinGroupCallbackParams] = { + val (responseFuture, responseCallback) = setupJoinGroupCallback + EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(offsetManager) + consumerCoordinator.handleJoinGroup(groupId, consumerId, Set.empty, sessionTimeout, partitionAssignmentStrategy, responseCallback) + responseFuture + } + + private def joinGroup(groupId: String, + consumerId: String, + partitionAssignmentStrategy: String, + sessionTimeout: Int, + isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = { + val responseFuture = sendJoinGroup(groupId, consumerId, partitionAssignmentStrategy, sessionTimeout, isCoordinatorForGroup) + Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) + } + + private def heartbeat(groupId: String, + consumerId: String, + generationId: Int, + isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = { + val (responseFuture, responseCallback) = setupHeartbeatCallback + EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(offsetManager) + consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback) + Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) + } +} diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala new file mode 100644 index 0000000000000..b69c993a3fa49 --- /dev/null +++ b/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala @@ -0,0 +1,172 @@ +/** + * 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.coordinator + +import junit.framework.Assert._ +import org.junit.{Before, Test} +import org.scalatest.junit.JUnitSuite + +/** + * Test group state transitions + */ +class ConsumerGroupMetadataTest extends JUnitSuite { + var group: ConsumerGroupMetadata = null + + @Before + def setUp() { + group = new ConsumerGroupMetadata("test", "range") + } + + @Test + def testCanRebalanceWhenStable() { + assertTrue(group.canRebalance) + } + + @Test + def testCannotRebalanceWhenPreparingRebalance() { + group.transitionTo(PreparingRebalance) + assertFalse(group.canRebalance) + } + + @Test + def testCannotRebalanceWhenRebalancing() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Rebalancing) + assertFalse(group.canRebalance) + } + + @Test + def testCannotRebalanceWhenDead() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Dead) + assertFalse(group.canRebalance) + } + + @Test + def testStableToPreparingRebalanceTransition() { + group.transitionTo(PreparingRebalance) + assertState(group, PreparingRebalance) + } + + @Test + def testPreparingRebalanceToRebalancingTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Rebalancing) + assertState(group, Rebalancing) + } + + @Test + def testPreparingRebalanceToDeadTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Dead) + assertState(group, Dead) + } + + @Test + def testRebalancingToStableTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Rebalancing) + group.transitionTo(Stable) + assertState(group, Stable) + } + + @Test(expected = classOf[IllegalStateException]) + def testStableToStableIllegalTransition() { + group.transitionTo(Stable) + } + + @Test(expected = classOf[IllegalStateException]) + def testStableToRebalancingIllegalTransition() { + group.transitionTo(Rebalancing) + } + + @Test(expected = classOf[IllegalStateException]) + def testStableToDeadIllegalTransition() { + group.transitionTo(Dead) + } + + @Test(expected = classOf[IllegalStateException]) + def testPreparingRebalanceToPreparingRebalanceIllegalTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(PreparingRebalance) + } + + @Test(expected = classOf[IllegalStateException]) + def testPreparingRebalanceToStableIllegalTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Stable) + } + + @Test(expected = classOf[IllegalStateException]) + def testRebalancingToRebalancingIllegalTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Rebalancing) + group.transitionTo(Rebalancing) + } + + @Test(expected = classOf[IllegalStateException]) + def testRebalancingToPreparingRebalanceTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Rebalancing) + group.transitionTo(PreparingRebalance) + } + + @Test(expected = classOf[IllegalStateException]) + def testRebalancingToDeadIllegalTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Rebalancing) + group.transitionTo(Dead) + } + + @Test(expected = classOf[IllegalStateException]) + def testDeadToDeadIllegalTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Dead) + group.transitionTo(Dead) + } + + @Test(expected = classOf[IllegalStateException]) + def testDeadToStableIllegalTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Dead) + group.transitionTo(Stable) + } + + @Test(expected = classOf[IllegalStateException]) + def testDeadToPreparingRebalanceIllegalTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Dead) + group.transitionTo(PreparingRebalance) + } + + @Test(expected = classOf[IllegalStateException]) + def testDeadToRebalancingIllegalTransition() { + group.transitionTo(PreparingRebalance) + group.transitionTo(Dead) + group.transitionTo(Rebalancing) + } + + private def assertState(group: ConsumerGroupMetadata, targetState: GroupState) { + val states: Set[GroupState] = Set(Stable, PreparingRebalance, Rebalancing, Dead) + val otherStates = states - targetState + otherStates.foreach { otherState => + assertFalse(group.is(otherState)) + } + assertTrue(group.is(targetState)) + } +} diff --git a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala new file mode 100644 index 0000000000000..2cbf6e251adbf --- /dev/null +++ b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala @@ -0,0 +1,213 @@ +/** + * 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.coordinator + +import kafka.server.KafkaConfig +import kafka.utils.{ZkUtils, TestUtils} + +import junit.framework.Assert._ +import org.I0Itec.zkclient.{IZkDataListener, ZkClient} +import org.apache.zookeeper.data.Stat +import org.easymock.EasyMock +import org.junit.{Before, Test} +import org.scalatest.junit.JUnitSuite + +/** + * Test coordinator group and topic metadata management + */ +class CoordinatorMetadataTest extends JUnitSuite { + val DefaultNumPartitions = 8 + val DefaultNumReplicas = 2 + var zkClient: ZkClient = null + var coordinatorMetadata: CoordinatorMetadata = null + + @Before + def setUp() { + val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") + zkClient = EasyMock.createStrictMock(classOf[ZkClient]) + coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props).brokerId, zkClient, null) + } + + @Test + def testGetNonexistentGroup() { + assertNull(coordinatorMetadata.getGroup("group")) + } + + @Test + def testGetGroup() { + val groupId = "group" + val expected = coordinatorMetadata.addGroup(groupId, "range") + val actual = coordinatorMetadata.getGroup(groupId) + assertEquals(expected, actual) + } + + @Test + def testAddGroupReturnsPreexistingGroupIfItAlreadyExists() { + val groupId = "group" + val group1 = coordinatorMetadata.addGroup(groupId, "range") + val group2 = coordinatorMetadata.addGroup(groupId, "range") + assertEquals(group1, group2) + } + + @Test(expected = classOf[IllegalArgumentException]) + def testBindNonexistentGroupToTopics() { + val groupId = "group" + val topics = Set("a") + coordinatorMetadata.bindGroupToTopics(groupId, topics) + } + + @Test + def testBindGroupToTopicsNotListenedOn() { + val groupId = "group" + val topics = Set("a") + coordinatorMetadata.addGroup(groupId, "range") + + expectZkClientSubscribeDataChanges(zkClient, topics) + EasyMock.replay(zkClient) + coordinatorMetadata.bindGroupToTopics(groupId, topics) + assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic) + } + + @Test + def testBindGroupToTopicsAlreadyListenedOn() { + val group1 = "group1" + val group2 = "group2" + val topics = Set("a") + coordinatorMetadata.addGroup(group1, "range") + coordinatorMetadata.addGroup(group2, "range") + + expectZkClientSubscribeDataChanges(zkClient, topics) + EasyMock.replay(zkClient) + coordinatorMetadata.bindGroupToTopics(group1, topics) + coordinatorMetadata.bindGroupToTopics(group2, topics) + assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic) + } + + @Test(expected = classOf[IllegalArgumentException]) + def testUnbindNonexistentGroupFromTopics() { + val groupId = "group" + val topics = Set("a") + coordinatorMetadata.unbindGroupFromTopics(groupId, topics) + } + + @Test + def testUnbindGroupFromTopicsNotListenedOn() { + val groupId = "group" + val topics = Set("a") + coordinatorMetadata.addGroup(groupId, "range") + + expectZkClientSubscribeDataChanges(zkClient, topics) + EasyMock.replay(zkClient) + coordinatorMetadata.bindGroupToTopics(groupId, topics) + coordinatorMetadata.unbindGroupFromTopics(groupId, Set("b")) + assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic) + } + + @Test + def testUnbindGroupFromTopicsListenedOnByOtherGroups() { + val group1 = "group1" + val group2 = "group2" + val topics = Set("a") + coordinatorMetadata.addGroup(group1, "range") + coordinatorMetadata.addGroup(group2, "range") + + expectZkClientSubscribeDataChanges(zkClient, topics) + EasyMock.replay(zkClient) + coordinatorMetadata.bindGroupToTopics(group1, topics) + coordinatorMetadata.bindGroupToTopics(group2, topics) + coordinatorMetadata.unbindGroupFromTopics(group1, topics) + assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic) + } + + @Test + def testUnbindGroupFromTopicsListenedOnByNoOtherGroup() { + val groupId = "group" + val topics = Set("a") + coordinatorMetadata.addGroup(groupId, "range") + + expectZkClientSubscribeDataChanges(zkClient, topics) + expectZkClientUnsubscribeDataChanges(zkClient, topics) + EasyMock.replay(zkClient) + coordinatorMetadata.bindGroupToTopics(groupId, topics) + coordinatorMetadata.unbindGroupFromTopics(groupId, topics) + assertEquals(Map.empty[String, Int], coordinatorMetadata.partitionsPerTopic) + } + + @Test(expected = classOf[IllegalArgumentException]) + def testRemoveNonexistentGroup() { + val groupId = "group" + val topics = Set("a") + coordinatorMetadata.removeGroup(groupId, topics) + } + + @Test + def testRemoveGroupWithOtherGroupsBoundToItsTopics() { + val group1 = "group1" + val group2 = "group2" + val topics = Set("a") + coordinatorMetadata.addGroup(group1, "range") + coordinatorMetadata.addGroup(group2, "range") + + expectZkClientSubscribeDataChanges(zkClient, topics) + EasyMock.replay(zkClient) + coordinatorMetadata.bindGroupToTopics(group1, topics) + coordinatorMetadata.bindGroupToTopics(group2, topics) + coordinatorMetadata.removeGroup(group1, topics) + assertNull(coordinatorMetadata.getGroup(group1)) + assertNotNull(coordinatorMetadata.getGroup(group2)) + assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic) + } + + @Test + def testRemoveGroupWithNoOtherGroupsBoundToItsTopics() { + val groupId = "group" + val topics = Set("a") + coordinatorMetadata.addGroup(groupId, "range") + + expectZkClientSubscribeDataChanges(zkClient, topics) + expectZkClientUnsubscribeDataChanges(zkClient, topics) + EasyMock.replay(zkClient) + coordinatorMetadata.bindGroupToTopics(groupId, topics) + coordinatorMetadata.removeGroup(groupId, topics) + assertNull(coordinatorMetadata.getGroup(groupId)) + assertEquals(Map.empty[String, Int], coordinatorMetadata.partitionsPerTopic) + } + + private def expectZkClientSubscribeDataChanges(zkClient: ZkClient, topics: Set[String]) { + topics.foreach(topic => expectZkClientSubscribeDataChange(zkClient, topic)) + } + + private def expectZkClientUnsubscribeDataChanges(zkClient: ZkClient, topics: Set[String]) { + topics.foreach(topic => expectZkClientUnsubscribeDataChange(zkClient, topic)) + } + + private def expectZkClientSubscribeDataChange(zkClient: ZkClient, topic: String) { + val replicaAssignment = + (0 until DefaultNumPartitions) + .map(partition => partition.toString -> (0 until DefaultNumReplicas).toSeq).toMap + val topicPath = ZkUtils.getTopicPath(topic) + EasyMock.expect(zkClient.readData(topicPath, new Stat())) + .andReturn(ZkUtils.replicaAssignmentZkData(replicaAssignment)) + zkClient.subscribeDataChanges(EasyMock.eq(topicPath), EasyMock.isA(classOf[IZkDataListener])) + } + + private def expectZkClientUnsubscribeDataChange(zkClient: ZkClient, topic: String) { + val topicPath = ZkUtils.getTopicPath(topic) + zkClient.unsubscribeDataChanges(EasyMock.eq(topicPath), EasyMock.isA(classOf[IZkDataListener])) + } +} diff --git a/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala new file mode 100644 index 0000000000000..887cee5a582b5 --- /dev/null +++ b/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala @@ -0,0 +1,305 @@ +/** + * 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.coordinator + +import kafka.common.TopicAndPartition + +import junit.framework.Assert._ +import org.junit.Test +import org.scalatest.junit.JUnitSuite + +class PartitionAssignorTest extends JUnitSuite { + + @Test + def testRangeAssignorOneConsumerNoTopic() { + val consumer = "consumer" + val assignor = new RangeAssignor() + val topicsPerConsumer = Map(consumer -> Set.empty[String]) + val partitionsPerTopic = Map.empty[String, Int] + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> Set.empty[TopicAndPartition]) + assertEquals(expected, actual) + } + + @Test + def testRangeAssignorOneConsumerNonexistentTopic() { + val topic = "topic" + val consumer = "consumer" + val assignor = new RangeAssignor() + val topicsPerConsumer = Map(consumer -> Set(topic)) + val partitionsPerTopic = Map(topic -> 0) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> Set.empty[TopicAndPartition]) + assertEquals(expected, actual) + } + + @Test + def testRangeAssignorOneConsumerOneTopic() { + val topic = "topic" + val consumer = "consumer" + val numPartitions = 3 + val assignor = new RangeAssignor() + val topicsPerConsumer = Map(consumer -> Set(topic)) + val partitionsPerTopic = Map(topic -> numPartitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> topicAndPartitions(Map(topic -> Set(0, 1, 2)))) + assertEquals(expected, actual) + } + + @Test + def testRangeAssignorOnlyAssignsPartitionsFromSubscribedTopics() { + val subscribedTopic = "topic" + val otherTopic = "other" + val consumer = "consumer" + val subscribedTopicNumPartitions = 3 + val otherTopicNumPartitions = 3 + val assignor = new RangeAssignor() + val topicsPerConsumer = Map(consumer -> Set(subscribedTopic)) + val partitionsPerTopic = Map(subscribedTopic -> subscribedTopicNumPartitions, otherTopic -> otherTopicNumPartitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> topicAndPartitions(Map(subscribedTopic -> Set(0, 1, 2)))) + assertEquals(expected, actual) + } + + @Test + def testRangeAssignorOneConsumerMultipleTopics() { + val topic1 = "topic1" + val topic2 = "topic2" + val consumer = "consumer" + val numTopic1Partitions = 1 + val numTopic2Partitions = 2 + val assignor = new RangeAssignor() + val topicsPerConsumer = Map(consumer -> Set(topic1, topic2)) + val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> topicAndPartitions(Map(topic1 -> Set(0), topic2 -> Set(0, 1)))) + assertEquals(expected, actual) + } + + @Test + def testRangeAssignorTwoConsumersOneTopicOnePartition() { + val topic = "topic" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val numPartitions = 1 + val assignor = new RangeAssignor() + val topicsPerConsumer = Map(consumer1 -> Set(topic), consumer2 -> Set(topic)) + val partitionsPerTopic = Map(topic -> numPartitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map( + consumer1 -> topicAndPartitions(Map(topic -> Set(0))), + consumer2 -> Set.empty[TopicAndPartition]) + assertEquals(expected, actual) + } + + @Test + def testRangeAssignorTwoConsumersOneTopicTwoPartitions() { + val topic = "topic" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val numPartitions = 2 + val assignor = new RangeAssignor() + val topicsPerConsumer = Map(consumer1 -> Set(topic), consumer2 -> Set(topic)) + val partitionsPerTopic = Map(topic -> numPartitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map( + consumer1 -> topicAndPartitions(Map(topic -> Set(0))), + consumer2 -> topicAndPartitions(Map(topic -> Set(1)))) + assertEquals(expected, actual) + } + + @Test + def testRangeAssignorMultipleConsumersMixedTopics() { + val topic1 = "topic1" + val topic2 = "topic2" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val consumer3 = "consumer3" + val numTopic1Partitions = 3 + val numTopic2Partitions = 2 + val assignor = new RangeAssignor() + val topicsPerConsumer = Map(consumer1 -> Set(topic1), consumer2 -> Set(topic1, topic2), consumer3 -> Set(topic1)) + val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map( + consumer1 -> topicAndPartitions(Map(topic1 -> Set(0))), + consumer2 -> topicAndPartitions(Map(topic1 -> Set(1), topic2 -> Set(0, 1))), + consumer3 -> topicAndPartitions(Map(topic1 -> Set(2)))) + assertEquals(expected, actual) + } + + @Test + def testRangeAssignorTwoConsumersTwoTopicsSixPartitions() { + val topic1 = "topic1" + val topic2 = "topic2" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val numTopic1Partitions = 3 + val numTopic2Partitions = 3 + val assignor = new RangeAssignor() + val topicsPerConsumer = Map(consumer1 -> Set(topic1, topic2), consumer2 -> Set(topic1, topic2)) + val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map( + consumer1 -> topicAndPartitions(Map(topic1 -> Set(0, 1), topic2 -> Set(0, 1))), + consumer2 -> topicAndPartitions(Map(topic1 -> Set(2), topic2 -> Set(2)))) + assertEquals(expected, actual) + } + + @Test + def testRoundRobinAssignorOneConsumerNoTopic() { + val consumer = "consumer" + val assignor = new RoundRobinAssignor() + val topicsPerConsumer = Map(consumer -> Set.empty[String]) + val partitionsPerTopic = Map.empty[String, Int] + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> Set.empty[TopicAndPartition]) + assertEquals(expected, actual) + } + + @Test + def testRoundRobinAssignorOneConsumerNonexistentTopic() { + val topic = "topic" + val consumer = "consumer" + val assignor = new RoundRobinAssignor() + val topicsPerConsumer = Map(consumer -> Set(topic)) + val partitionsPerTopic = Map(topic -> 0) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> Set.empty[TopicAndPartition]) + assertEquals(expected, actual) + } + + @Test + def testRoundRobinAssignorOneConsumerOneTopic() { + val topic = "topic" + val consumer = "consumer" + val numPartitions = 3 + val assignor = new RoundRobinAssignor() + val topicsPerConsumer = Map(consumer -> Set(topic)) + val partitionsPerTopic = Map(topic -> numPartitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> topicAndPartitions(Map(topic -> Set(0, 1, 2)))) + assertEquals(expected, actual) + } + + @Test + def testRoundRobinAssignorOnlyAssignsPartitionsFromSubscribedTopics() { + val subscribedTopic = "topic" + val otherTopic = "other" + val consumer = "consumer" + val subscribedTopicNumPartitions = 3 + val otherTopicNumPartitions = 3 + val assignor = new RoundRobinAssignor() + val topicsPerConsumer = Map(consumer -> Set(subscribedTopic)) + val partitionsPerTopic = Map(subscribedTopic -> subscribedTopicNumPartitions, otherTopic -> otherTopicNumPartitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> topicAndPartitions(Map(subscribedTopic -> Set(0, 1, 2)))) + assertEquals(expected, actual) + } + + @Test + def testRoundRobinAssignorOneConsumerMultipleTopics() { + val topic1 = "topic1" + val topic2 = "topic2" + val consumer = "consumer" + val numTopic1Partitions = 1 + val numTopic2Partitions = 2 + val assignor = new RoundRobinAssignor() + val topicsPerConsumer = Map(consumer -> Set(topic1, topic2)) + val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map(consumer -> topicAndPartitions(Map(topic1 -> Set(0), topic2 -> Set(0, 1)))) + assertEquals(expected, actual) + } + + @Test + def testRoundRobinAssignorTwoConsumersOneTopicOnePartition() { + val topic = "topic" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val numPartitions = 1 + val assignor = new RoundRobinAssignor() + val topicsPerConsumer = Map(consumer1 -> Set(topic), consumer2 -> Set(topic)) + val partitionsPerTopic = Map(topic -> numPartitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map( + consumer1 -> topicAndPartitions(Map(topic -> Set(0))), + consumer2 -> Set.empty[TopicAndPartition]) + assertEquals(expected, actual) + } + + @Test + def testRoundRobinAssignorTwoConsumersOneTopicTwoPartitions() { + val topic = "topic" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val numPartitions = 2 + val assignor = new RoundRobinAssignor() + val topicsPerConsumer = Map(consumer1 -> Set(topic), consumer2 -> Set(topic)) + val partitionsPerTopic = Map(topic -> numPartitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map( + consumer1 -> topicAndPartitions(Map(topic -> Set(0))), + consumer2 -> topicAndPartitions(Map(topic -> Set(1)))) + assertEquals(expected, actual) + } + + @Test + def testRoundRobinAssignorMultipleConsumersMixedTopics() { + val topic1 = "topic1" + val topic2 = "topic2" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val consumer3 = "consumer3" + val numTopic1Partitions = 3 + val numTopic2Partitions = 2 + val assignor = new RoundRobinAssignor() + val topicsPerConsumer = Map(consumer1 -> Set(topic1), consumer2 -> Set(topic1, topic2), consumer3 -> Set(topic1)) + val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map( + consumer1 -> topicAndPartitions(Map(topic1 -> Set(0))), + consumer2 -> topicAndPartitions(Map(topic1 -> Set(1), topic2 -> Set(0, 1))), + consumer3 -> topicAndPartitions(Map(topic1 -> Set(2)))) + assertEquals(expected, actual) + } + + @Test + def testRoundRobinAssignorTwoConsumersTwoTopicsSixPartitions() { + val topic1 = "topic1" + val topic2 = "topic2" + val consumer1 = "consumer1" + val consumer2 = "consumer2" + val numTopic1Partitions = 3 + val numTopic2Partitions = 3 + val assignor = new RoundRobinAssignor() + val topicsPerConsumer = Map(consumer1 -> Set(topic1, topic2), consumer2 -> Set(topic1, topic2)) + val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions) + val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic) + val expected = Map( + consumer1 -> topicAndPartitions(Map(topic1 -> Set(0, 2), topic2 -> Set(1))), + consumer2 -> topicAndPartitions(Map(topic1 -> Set(1), topic2 -> Set(0, 2)))) + assertEquals(expected, actual) + } + + private def topicAndPartitions(topicPartitions: Map[String, Set[Int]]): Set[TopicAndPartition] = { + topicPartitions.flatMap { case (topic, partitions) => + partitions.map(partition => TopicAndPartition(topic, partition)) + }.toSet + } +} diff --git a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala index e5703bc16b2ce..139dc9a104c02 100644 --- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala +++ b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala @@ -17,22 +17,25 @@ package kafka.integration -import junit.framework.Assert._ import kafka.utils.{ZKGroupTopicDirs, Logging} import kafka.consumer.{ConsumerTimeoutException, ConsumerConfig, ConsumerConnector, Consumer} import kafka.server._ -import org.apache.log4j.{Level, Logger} -import org.scalatest.junit.JUnit3Suite import kafka.utils.TestUtils import kafka.serializer._ import kafka.producer.{Producer, KeyedMessage} +import org.junit.Test +import org.apache.log4j.{Level, Logger} +import org.scalatest.junit.JUnit3Suite +import junit.framework.Assert._ + class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with Logging { + def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) + val topic = "test_topic" val group = "default_group" val testConsumer = "consumer" - val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0))) val NumMessages = 10 val LargeOffset = 10000 val SmallOffset = -1 @@ -50,37 +53,40 @@ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with L requestHandlerLogger.setLevel(Level.ERROR) super.tearDown } - - def testResetToEarliestWhenOffsetTooHigh() = + + @Test + def testResetToEarliestWhenOffsetTooHigh() = assertEquals(NumMessages, resetAndConsume(NumMessages, "smallest", LargeOffset)) - + + @Test def testResetToEarliestWhenOffsetTooLow() = assertEquals(NumMessages, resetAndConsume(NumMessages, "smallest", SmallOffset)) + @Test def testResetToLatestWhenOffsetTooHigh() = assertEquals(0, resetAndConsume(NumMessages, "largest", LargeOffset)) + @Test def testResetToLatestWhenOffsetTooLow() = assertEquals(0, resetAndConsume(NumMessages, "largest", SmallOffset)) - + /* Produce the given number of messages, create a consumer with the given offset policy, * then reset the offset to the given value and consume until we get no new messages. * Returns the count of messages received. */ def resetAndConsume(numMessages: Int, resetTo: String, offset: Long): Int = { - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + TestUtils.createTopic(zkClient, topic, 1, 1, servers) - val producer: Producer[String, Array[Byte]] = TestUtils.createProducer(TestUtils.getBrokerListStrFromConfigs(configs), - new DefaultEncoder(), new StringEncoder()) + val producer: Producer[String, Array[Byte]] = TestUtils.createProducer( + TestUtils.getBrokerListStrFromServers(servers), + keyEncoder = classOf[StringEncoder].getName) for(i <- 0 until numMessages) producer.send(new KeyedMessage[String, Array[Byte]](topic, topic, "test".getBytes)) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) - // update offset in zookeeper for consumer to jump "forward" in time val dirs = new ZKGroupTopicDirs(group, topic) - var consumerProps = TestUtils.createConsumerProperties(zkConnect, group, testConsumer) + val consumerProps = TestUtils.createConsumerProperties(zkConnect, group, testConsumer) consumerProps.put("auto.offset.reset", resetTo) consumerProps.put("consumer.timeout.ms", "2000") consumerProps.put("fetch.wait.max.ms", "0") diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 47130d33c36fa..facebd8f81c67 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -28,37 +28,34 @@ import org.scalatest.junit.JUnit3Suite import kafka.consumer._ import kafka.serializer._ import kafka.producer.{KeyedMessage, Producer} -import kafka.utils.TestUtils._ import kafka.utils.TestUtils -import kafka.admin.AdminUtils class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { - val numNodes = 1 - val configs = - for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) + def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps) + 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 shutdown = ZookeeperConsumerConnector.shutdownCommand val queue = new LinkedBlockingQueue[FetchedDataChunk] - val topicInfos = configs.map(c => new PartitionTopicInfo(topic, - 0, - queue, - new AtomicLong(0), - new AtomicLong(0), - new AtomicInteger(0), - "")) var fetcher: ConsumerFetcherManager = null override def setUp() { super.setUp - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(configs.head.brokerId))) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) + + val cluster = new Cluster(servers.map(s => new Broker(s.config.brokerId, "localhost", s.boundPort()))) + fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient) fetcher.stopConnections() + val topicInfos = configs.map(c => + new PartitionTopicInfo(topic, + 0, + queue, + new AtomicLong(0), + new AtomicLong(0), + new AtomicInteger(0), + "")) fetcher.startConnections(topicInfos, cluster) } @@ -69,32 +66,17 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { def testFetcher() { val perNode = 2 - var count = sendMessages(perNode) + var count = TestUtils.sendMessages(servers, topic, perNode).size fetch(count) assertQueueEmpty() - count = sendMessages(perNode) + count = TestUtils.sendMessages(servers, topic, perNode).size fetch(count) assertQueueEmpty() } def assertQueueEmpty(): Unit = assertEquals(0, queue.size) - def sendMessages(messagesPerNode: Int): Int = { - var count = 0 - for(conf <- configs) { - val producer: Producer[String, Array[Byte]] = TestUtils.createProducer(TestUtils.getBrokerListStrFromConfigs(configs), - new DefaultEncoder(), - 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 => new KeyedMessage[String, Array[Byte]](topic, topic, m)):_*) - producer.close() - count += ms.size - } - count - } - def fetch(expected: Int) { var count = 0 while(true) { diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala old mode 100644 new mode 100755 index 194dd70919a5f..87c631573aa1e --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -17,8 +17,11 @@ package kafka.integration +import java.util.Arrays + +import scala.collection.mutable.Buffer import kafka.server._ -import kafka.utils.{Utils, TestUtils} +import kafka.utils.{CoreUtils, TestUtils} import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import kafka.common.KafkaException @@ -27,20 +30,65 @@ import kafka.common.KafkaException * A test harness that brings up some number of broker nodes */ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { + var instanceConfigs: Seq[KafkaConfig] = null + var servers: Buffer[KafkaServer] = null + var brokerList: String = null + var alive: Array[Boolean] = null + + /** + * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every + * test and should not reuse previous configurations unless they select their ports randomly when servers are started. + */ + def generateConfigs(): Seq[KafkaConfig] + + def configs: Seq[KafkaConfig] = { + if (instanceConfigs == null) + instanceConfigs = generateConfigs() + instanceConfigs + } - val configs: List[KafkaConfig] - var servers: List[KafkaServer] = null + def serverForId(id: Int) = servers.find(s => s.config.brokerId == id) + def bootstrapUrl = servers.map(s => s.config.hostName + ":" + s.boundPort()).mkString(",") + + override def setUp() { super.setUp if(configs.size <= 0) - throw new KafkaException("Must suply at least one server config.") - servers = configs.map(TestUtils.createServer(_)) + throw new KafkaException("Must supply at least one server config.") + servers = configs.map(TestUtils.createServer(_)).toBuffer + brokerList = TestUtils.getBrokerListStrFromServers(servers) + alive = new Array[Boolean](servers.length) + Arrays.fill(alive, true) } override def tearDown() { - servers.map(server => server.shutdown()) - servers.map(server => server.config.logDirs.map(Utils.rm(_))) + servers.foreach(_.shutdown()) + servers.foreach(_.config.logDirs.foreach(CoreUtils.rm(_))) super.tearDown } + + /** + * Pick a broker at random and kill it if it isn't already dead + * Return the id of the broker killed + */ + def killRandomBroker(): Int = { + val index = TestUtils.random.nextInt(servers.length) + if(alive(index)) { + servers(index).shutdown() + servers(index).awaitShutdown() + alive(index) = false + } + index + } + + /** + * Restart any dead brokers + */ + def restartDeadBrokers() { + for(i <- 0 until servers.length if !alive(i)) { + servers(i).startup() + alive(i) = true + } + } } diff --git a/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala b/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala deleted file mode 100644 index c3c7631659fbd..0000000000000 --- a/core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala +++ /dev/null @@ -1,173 +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.integration - -import kafka.api.FetchRequestBuilder -import kafka.message.ByteBufferMessageSet -import kafka.server.{KafkaRequestHandler, KafkaConfig} -import org.apache.log4j.{Level, Logger} -import org.scalatest.junit.JUnit3Suite -import scala.collection._ -import kafka.utils._ -import kafka.common.{ErrorMapping, KafkaException, OffsetOutOfRangeException} -import kafka.producer.KeyedMessage -import org.junit.Assert.assertEquals - -/** - * End to end tests of the primitive apis against a local server - */ -class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness { - - val port = TestUtils.choosePort - val props = TestUtils.createBrokerConfig(0, port) - val config = new KafkaConfig(props) - val configs = List(config) - val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) - - override def setUp() { - super.setUp - if(configs.size <= 0) - throw new KafkaException("Must suply at least one server config.") - - // 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 testProduceAndFetch() { - // send some messages - val topic = "test" - val sentMessages = List("hello", "there") - val producerData = sentMessages.map(m => new KeyedMessage[String, String](topic, topic, m)) - - producer.send(producerData:_*) - - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) - - var fetchedMessage: ByteBufferMessageSet = null - while(fetchedMessage == null || fetchedMessage.validBytes == 0) { - val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) - fetchedMessage = fetched.messageSet(topic, 0) - } - assertEquals(sentMessages, fetchedMessage.map(m => Utils.readString(m.message.payload)).toList) - - // send an invalid offset - try { - val fetchedWithError = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, -1, 10000).build()) - fetchedWithError.data.values.foreach(pdata => ErrorMapping.maybeThrowException(pdata.error)) - fail("Expected an OffsetOutOfRangeException exception to be thrown") - } catch { - case e: OffsetOutOfRangeException => - } - } - - def testProduceAndMultiFetch() { - // send some messages, with non-ordered topics - val topicOffsets = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - { - val messages = new mutable.HashMap[String, Seq[String]] - val builder = new FetchRequestBuilder() - for( (topic, offset) <- topicOffsets) { - val producedData = List("a_" + topic, "b_" + topic) - messages += topic -> producedData - producer.send(producedData.map(m => new KeyedMessage[String, String](topic, topic, m)):_*) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) - builder.addFetch(topic, offset, 0, 10000) - } - - // wait a bit for produced message to be available - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, offset) <- topicOffsets) { - val fetched = response.messageSet(topic, offset) - assertEquals(messages(topic), fetched.map(m => Utils.readString(m.message.payload))) - } - } - - // send some invalid offsets - val builder = new FetchRequestBuilder() - for((topic, offset) <- topicOffsets) - builder.addFetch(topic, offset, -1, 10000) - - val request = builder.build() - val responses = consumer.fetch(request) - responses.data.values.foreach(pd => { - try { - ErrorMapping.maybeThrowException(pd.error) - fail("Expected an OffsetOutOfRangeException exception to be thrown") - } catch { - case e: OffsetOutOfRangeException => // this is good - } - }) - } - - def testMultiProduce() { - // send some messages - val topics = List("test1", "test2", "test3"); - val messages = new mutable.HashMap[String, Seq[String]] - val builder = new FetchRequestBuilder() - var produceList: List[KeyedMessage[String, String]] = Nil - for(topic <- topics) { - val set = List("a_" + topic, "b_" + topic) - messages += topic -> set - produceList ++= set.map(new KeyedMessage[String, String](topic, topic, _)) - builder.addFetch(topic, 0, 0, 10000) - } - producer.send(produceList: _*) - topics.foreach(topic => TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)) - - // wait a bit for produced message to be available - val request = builder.build() - val response = consumer.fetch(request) - for(topic <- topics) { - val fetched = response.messageSet(topic, 0) - assertEquals(messages(topic), fetched.map(m => Utils.readString(m.message.payload))) - } - } - - def testMultiProduceResend() { - // send some messages - val topics = List("test1", "test2", "test3"); - val messages = new mutable.HashMap[String, Seq[String]] - val builder = new FetchRequestBuilder() - var produceList: List[KeyedMessage[String, String]] = Nil - for(topic <- topics) { - val set = List("a_" + topic, "b_" + topic) - messages += topic -> set - produceList ++= set.map(new KeyedMessage[String, String](topic, topic, _)) - builder.addFetch(topic, 0, 0, 10000) - } - producer.send(produceList: _*) - topics.foreach(topic => TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000)) - - producer.send(produceList: _*) - // wait a bit for produced message to be available - val request = builder.build() - val response = consumer.fetch(request) - for(topic <- topics) { - val topicMessages = response.messageSet(topic, 0) - assertEquals(messages(topic) ++ messages(topic), topicMessages.map(m => Utils.readString(m.message.payload))) - } - } -} diff --git a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala new file mode 100644 index 0000000000000..a2c97134d85c6 --- /dev/null +++ b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala @@ -0,0 +1,36 @@ +/* + * 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.integration + +import java.util.Properties + +import kafka.server.KafkaConfig +import kafka.utils.TestUtils +import org.scalatest.junit.JUnit3Suite + +class MinIsrConfigTest extends JUnit3Suite with KafkaServerTestHarness { + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.MinInSyncReplicasProp, "5") + def generateConfigs() = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps(_, overridingProps)) + + def testDefaultKafkaConfig() { + assert(servers.head.getLogManager().defaultConfig.minInSyncReplicas == 5) + } + +} diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala old mode 100644 new mode 100755 index 5f331d22bc99b..6a758a7db71bd --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -21,27 +21,24 @@ import java.nio.ByteBuffer import junit.framework.Assert._ import kafka.api.{PartitionFetchInfo, FetchRequest, FetchRequestBuilder} import kafka.server.{KafkaRequestHandler, KafkaConfig} -import kafka.producer.{KeyedMessage, Producer, ProducerConfig} +import kafka.producer.{KeyedMessage, Producer} import org.apache.log4j.{Level, Logger} -import org.I0Itec.zkclient.ZkClient import kafka.zk.ZooKeeperTestHarness import org.scalatest.junit.JUnit3Suite import scala.collection._ -import kafka.admin.AdminUtils import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionException, OffsetOutOfRangeException} -import kafka.utils.{TestUtils, Utils} +import kafka.utils.{StaticPartitioner, TestUtils, CoreUtils} +import kafka.serializer.StringEncoder +import java.util.Properties /** * End to end tests of the primitive apis against a local server */ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with ZooKeeperTestHarness { - - val port = TestUtils.choosePort - val props = TestUtils.createBrokerConfig(0, port) - val config = new KafkaConfig(props) - val configs = List(config) val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) + def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) + def testFetchRequestCanProperlySerialize() { val request = new FetchRequestBuilder() .clientId("test-client") @@ -67,15 +64,12 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with def testDefaultEncoderProducerAndFetch() { val topic = "test-topic" - val props = producer.config.props.props - val config = new ProducerConfig(props) - val stringProducer1 = new Producer[String, String](config) - stringProducer1.send(new KeyedMessage[String, String](topic, "test-message")) + producer.send(new KeyedMessage[String, String](topic, "test-message")) val replica = servers.head.replicaManager.getReplica(topic, 0).get assertTrue("HighWatermark should equal logEndOffset with just 1 replica", - replica.logEndOffset > 0 && replica.logEndOffset == replica.highWatermark) + replica.logEndOffset.messageOffset > 0 && replica.logEndOffset.equals(replica.highWatermark)) val request = new FetchRequestBuilder() .clientId("test-client") @@ -88,28 +82,34 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with assertTrue(messageSet.iterator.hasNext) val fetchedMessageAndOffset = messageSet.head - assertEquals("test-message", Utils.readString(fetchedMessageAndOffset.message.payload, "UTF-8")) + assertEquals("test-message", TestUtils.readString(fetchedMessageAndOffset.message.payload, "UTF-8")) } def testDefaultEncoderProducerAndFetchWithCompression() { val topic = "test-topic" - val props = producer.config.props.props - props.put("compression", "true") - val config = new ProducerConfig(props) + val props = new Properties() + props.put("compression.codec", "gzip") + + val stringProducer1 = TestUtils.createProducer[String, String]( + TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName, + producerProps = props) - val stringProducer1 = new Producer[String, String](config) stringProducer1.send(new KeyedMessage[String, String](topic, "test-message")) - var fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) + val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) val messageSet = fetched.messageSet(topic, 0) assertTrue(messageSet.iterator.hasNext) val fetchedMessageAndOffset = messageSet.head - assertEquals("test-message", Utils.readString(fetchedMessageAndOffset.message.payload, "UTF-8")) + assertEquals("test-message", TestUtils.readString(fetchedMessageAndOffset.message.payload, "UTF-8")) } - def testProduceAndMultiFetch() { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4"), config.brokerId) + private def produceAndMultiFetch(producer: Producer[String, String]) { + for(topic <- List("test1", "test2", "test3", "test4")) + TestUtils.createTopic(zkClient, topic, servers = servers) // send some messages val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); @@ -128,7 +128,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val response = consumer.fetch(request) for((topic, partition) <- topics) { val fetched = response.messageSet(topic, partition) - assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) + assertEquals(messages(topic), fetched.map(messageAndOffset => TestUtils.readString(messageAndOffset.message.payload))) } } @@ -171,139 +171,60 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with requestHandlerLogger.setLevel(Level.ERROR) } - def testProduceAndMultiFetchWithCompression() { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4"), config.brokerId) - - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - { - val messages = new mutable.HashMap[String, Seq[String]] - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) { - val messageList = List("a_" + topic, "b_" + topic) - val producerData = messageList.map(new KeyedMessage[String, String](topic, topic, _)) - messages += topic -> messageList - producer.send(producerData:_*) - builder.addFetch(topic, partition, 0, 10000) - } - - // wait a bit for produced message to be available - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val fetched = response.messageSet(topic, partition) - assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) - } - } - - // temporarily set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.FATAL) - - { - // send some invalid offsets - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) - builder.addFetch(topic, partition, -1, 10000) - - try { - val request = builder.build() - val response = consumer.fetch(request) - response.data.values.foreach(pdata => ErrorMapping.maybeThrowException(pdata.error)) - fail("Expected exception when fetching message with invalid offset") - } catch { - case e: OffsetOutOfRangeException => "this is good" - } - } - - { - // send some invalid partitions - val builder = new FetchRequestBuilder() - for( (topic, _) <- topics) - builder.addFetch(topic, -1, 0, 10000) - - try { - val request = builder.build() - val response = consumer.fetch(request) - response.data.values.foreach(pdata => ErrorMapping.maybeThrowException(pdata.error)) - fail("Expected exception when fetching message with invalid partition") - } catch { - case e: UnknownTopicOrPartitionException => "this is good" - } - } - - // restore set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.ERROR) + def testProduceAndMultiFetch() { + produceAndMultiFetch(producer) } - def testMultiProduce() { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4"), config.brokerId) + private def multiProduce(producer: Producer[String, String]) { + val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) + topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers)) - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); val messages = new mutable.HashMap[String, Seq[String]] val builder = new FetchRequestBuilder() - var produceList: List[KeyedMessage[String, String]] = Nil - for( (topic, partition) <- topics) { + for((topic, partition) <- topics) { val messageList = List("a_" + topic, "b_" + topic) val producerData = messageList.map(new KeyedMessage[String, String](topic, topic, _)) messages += topic -> messageList producer.send(producerData:_*) builder.addFetch(topic, partition, 0, 10000) } - producer.send(produceList: _*) val request = builder.build() val response = consumer.fetch(request) - for( (topic, partition) <- topics) { + for((topic, partition) <- topics) { val fetched = response.messageSet(topic, partition) - assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) + assertEquals(messages(topic), fetched.map(messageAndOffset => TestUtils.readString(messageAndOffset.message.payload))) } } - def testMultiProduceWithCompression() { - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - val messages = new mutable.HashMap[String, Seq[String]] - val builder = new FetchRequestBuilder() - var produceList: List[KeyedMessage[String, String]] = Nil - for( (topic, partition) <- topics) { - val messageList = List("a_" + topic, "b_" + topic) - val producerData = messageList.map(new KeyedMessage[String, String](topic, topic, _)) - messages += topic -> messageList - producer.send(producerData:_*) - builder.addFetch(topic, partition, 0, 10000) - } - producer.send(produceList: _*) - - // wait a bit for produced message to be available - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val fetched = response.messageSet(topic, 0) - assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) - } + def testMultiProduce() { + multiProduce(producer) } def testConsumerEmptyTopic() { val newTopic = "new-topic" - AdminUtils.createTopic(zkClient, newTopic, 1, 1) - TestUtils.waitUntilMetadataIsPropagated(servers, newTopic, 0, 1000) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, newTopic, 0, 500) + TestUtils.createTopic(zkClient, newTopic, numPartitions = 1, replicationFactor = 1, servers = servers) + val fetchResponse = consumer.fetch(new FetchRequestBuilder().addFetch(newTopic, 0, 0, 10000).build()) assertFalse(fetchResponse.messageSet(newTopic, 0).iterator.hasNext) } def testPipelinedProduceRequests() { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4"), config.brokerId) - val props = producer.config.props.props + val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) + topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers)) + val props = new Properties() props.put("request.required.acks", "0") - val pipelinedProducer: Producer[String, String] = new Producer(new ProducerConfig(props)) + val pipelinedProducer: Producer[String, String] = + TestUtils.createProducer[String, String]( + TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName, + producerProps = props) // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); val messages = new mutable.HashMap[String, Seq[String]] val builder = new FetchRequestBuilder() - var produceList: List[KeyedMessage[String, String]] = Nil for( (topic, partition) <- topics) { val messageList = List("a_" + topic, "b_" + topic) val producerData = messageList.map(new KeyedMessage[String, String](topic, topic, _)) @@ -313,35 +234,31 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with } // wait until the messages are published - 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) + TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog(TopicAndPartition("test1", 0)).get.logEndOffset == 2 }, + "Published messages should be in the log") + TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog(TopicAndPartition("test2", 0)).get.logEndOffset == 2 }, + "Published messages should be in the log") + TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog(TopicAndPartition("test3", 0)).get.logEndOffset == 2 }, + "Published messages should be in the log") + TestUtils.waitUntilTrue(() => { servers.head.logManager.getLog(TopicAndPartition("test4", 0)).get.logEndOffset == 2 }, + "Published messages should be in the log") val replicaId = servers.head.config.brokerId - val hwWaitMs = config.replicaHighWatermarkCheckpointIntervalMs - TestUtils.waitUntilTrue(() => { servers.head.replicaManager.getReplica("test1", 0, replicaId).get.highWatermark == 2 }, hwWaitMs) - TestUtils.waitUntilTrue(() => { servers.head.replicaManager.getReplica("test2", 0, replicaId).get.highWatermark == 2 }, hwWaitMs) - TestUtils.waitUntilTrue(() => { servers.head.replicaManager.getReplica("test3", 0, replicaId).get.highWatermark == 2 }, hwWaitMs) - TestUtils.waitUntilTrue(() => { servers.head.replicaManager.getReplica("test4", 0, replicaId).get.highWatermark == 2 }, hwWaitMs) + TestUtils.waitUntilTrue(() => { servers.head.replicaManager.getReplica("test1", 0, replicaId).get.highWatermark.messageOffset == 2 }, + "High watermark should equal to log end offset") + TestUtils.waitUntilTrue(() => { servers.head.replicaManager.getReplica("test2", 0, replicaId).get.highWatermark.messageOffset == 2 }, + "High watermark should equal to log end offset") + TestUtils.waitUntilTrue(() => { servers.head.replicaManager.getReplica("test3", 0, replicaId).get.highWatermark.messageOffset == 2 }, + "High watermark should equal to log end offset") + TestUtils.waitUntilTrue(() => { servers.head.replicaManager.getReplica("test4", 0, replicaId).get.highWatermark.messageOffset == 2 }, + "High watermark should equal to log end offset") // test if the consumer received the messages in the correct order when producer has enabled request pipelining val request = builder.build() val response = consumer.fetch(request) for( (topic, partition) <- topics) { val fetched = response.messageSet(topic, partition) - assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) - } - } - - /** - * For testing purposes, just create these topics each with one partition and one replica for - * which the provided broker should the leader for. Create and wait for broker to lead. Simple. - */ - def createSimpleTopicsAndAwaitLeader(zkClient: ZkClient, topics: Seq[String], brokerId: Int) { - for( topic <- topics ) { - AdminUtils.createTopic(zkClient, topic, 1, 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + assertEquals(messages(topic), fetched.map(messageAndOffset => TestUtils.readString(messageAndOffset.message.payload))) } } } diff --git a/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala index 731ee59e67e30..4614a922e7390 100644 --- a/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala @@ -19,19 +19,22 @@ package kafka.integration import kafka.consumer.SimpleConsumer import org.scalatest.junit.JUnit3Suite -import kafka.producer.{ProducerConfig, Producer} -import kafka.utils.TestUtils +import kafka.producer.Producer +import kafka.utils.{StaticPartitioner, TestUtils} +import kafka.serializer.StringEncoder + trait ProducerConsumerTestHarness extends JUnit3Suite with KafkaServerTestHarness { - val port: Int val host = "localhost" var producer: Producer[String, String] = null var consumer: SimpleConsumer = null override def setUp() { super.setUp - val props = TestUtils.getProducerConfig(TestUtils.getBrokerListStrFromConfigs(configs), "kafka.utils.StaticPartitioner") - producer = new Producer(new ProducerConfig(props)) - consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "") + producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName) + consumer = new SimpleConsumer(host, servers(0).boundPort(), 1000000, 64*1024, "") } override def tearDown() { diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala old mode 100644 new mode 100755 index b585f0ec0b1c4..12d0733f5edf4 --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -15,59 +15,33 @@ * limitations under the License. */ -package kafka.server +package kafka.integration import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import junit.framework.Assert._ -import kafka.utils.{ZkUtils, Utils, TestUtils} -import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} -import kafka.cluster.Broker -import kafka.common.ErrorMapping -import kafka.api._ -import kafka.admin.AdminUtils +import kafka.utils.{CoreUtils, TestUtils} +import kafka.server.{KafkaConfig, KafkaServer} class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { - val brokerId1 = 0 - val brokerId2 = 1 - val brokerId3 = 2 - val brokerId4 = 3 - - val port1 = TestUtils.choosePort() - val port2 = TestUtils.choosePort() - val port3 = TestUtils.choosePort() - val port4 = TestUtils.choosePort() - - val enableShutdown = true - val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) - configProps1.put("controlled.shutdown.enable", "true") - val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) - configProps2.put("controlled.shutdown.enable", "true") - val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3) - configProps3.put("controlled.shutdown.enable", "true") - val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4) - configProps4.put("controlled.shutdown.enable", "true") - configProps4.put("controlled.shutdown.retry.backoff.ms", "100") - - var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] val partitionId = 0 + var servers: Seq[KafkaServer] = null override def setUp() { super.setUp() + // controlled.shutdown.enable is true by default + val configs = (0 until 4).map(i => TestUtils.createBrokerConfig(i, zkConnect)) + configs(3).put("controlled.shutdown.retry.backoff.ms", "100") + // start all the servers - val server1 = TestUtils.createServer(new KafkaConfig(configProps1)) - val server2 = TestUtils.createServer(new KafkaConfig(configProps2)) - val server3 = TestUtils.createServer(new KafkaConfig(configProps3)) - val server4 = TestUtils.createServer(new KafkaConfig(configProps4)) - - servers ++= List(server1, server2, server3, server4) + servers = configs.map(c => TestUtils.createServer(KafkaConfig.fromProps(c))) } override def tearDown() { - servers.map(server => server.shutdown()) - servers.map(server => Utils.rm(server.config.logDirs)) + servers.foreach(_.shutdown()) + servers.foreach(server => CoreUtils.rm(server.config.logDirs)) super.tearDown() } @@ -79,31 +53,10 @@ class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { val topic4 = "new-topic4" // create topics with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic1, Map(0->Seq(0,1))) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic2, Map(0->Seq(1,2))) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic3, Map(0->Seq(2,3))) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic4, Map(0->Seq(0,3))) - - // wait until leader is elected - var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partitionId, 500) - var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, partitionId, 500) - var leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, partitionId, 500) - var leader4 = waitUntilLeaderIsElectedOrChanged(zkClient, topic4, partitionId, 500) - - debug("Leader for " + topic1 + " is elected to be: %s".format(leader1.getOrElse(-1))) - debug("Leader for " + topic2 + " is elected to be: %s".format(leader1.getOrElse(-1))) - debug("Leader for " + topic3 + "is elected to be: %s".format(leader1.getOrElse(-1))) - debug("Leader for " + topic4 + "is elected to be: %s".format(leader1.getOrElse(-1))) - - assertTrue("Leader should get elected", leader1.isDefined) - assertTrue("Leader should get elected", leader2.isDefined) - assertTrue("Leader should get elected", leader3.isDefined) - assertTrue("Leader should get elected", leader4.isDefined) - - assertTrue("Leader could be broker 0 or broker 1 for " + topic1, (leader1.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 1)) - assertTrue("Leader could be broker 1 or broker 2 for " + topic2, (leader2.getOrElse(-1) == 1) || (leader1.getOrElse(-1) == 2)) - assertTrue("Leader could be broker 2 or broker 3 for " + topic3, (leader3.getOrElse(-1) == 2) || (leader1.getOrElse(-1) == 3)) - assertTrue("Leader could be broker 3 or broker 4 for " + topic4, (leader4.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 3)) + createTopic(zkClient, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) + createTopic(zkClient, topic2, partitionReplicaAssignment = Map(0->Seq(1,2)), servers = servers) + createTopic(zkClient, topic3, partitionReplicaAssignment = Map(0->Seq(2,3)), servers = servers) + createTopic(zkClient, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers) // Do a rolling bounce and check if leader transitions happen correctly @@ -130,10 +83,10 @@ class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { servers((startIndex + 1) % 4).shutdown() prevLeader = (startIndex + 1) % 4 } - var newleader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 1500) + var newleader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) // Ensure the new leader is different from the old assertTrue("Leader transition did not happen for " + topic, newleader.getOrElse(-1) != -1 && (newleader.getOrElse(-1) != prevLeader)) // Start the server back up again servers(prevLeader).startup() } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 9998a1156d41d..5b6c9d60d2943 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -17,28 +17,34 @@ package kafka.integration -import org.scalatest.junit.JUnit3Suite -import kafka.zk.ZooKeeperTestHarness -import kafka.admin.AdminUtils import java.nio.ByteBuffer + import junit.framework.Assert._ -import kafka.cluster.Broker +import kafka.admin.AdminUtils +import kafka.api.{TopicMetadataResponse, TopicMetadataRequest} +import kafka.client.ClientUtils +import kafka.cluster.{Broker, BrokerEndPoint} +import kafka.common.ErrorMapping +import kafka.server.{KafkaConfig, KafkaServer, NotRunning} import kafka.utils.TestUtils import kafka.utils.TestUtils._ -import kafka.server.{KafkaServer, KafkaConfig} -import kafka.api.TopicMetadataRequest -import kafka.common.ErrorMapping -import kafka.client.ClientUtils +import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.protocol.SecurityProtocol +import org.scalatest.junit.JUnit3Suite 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)) + var brokerEndPoints: Seq[BrokerEndPoint] = null + var adHocConfigs: Seq[KafkaConfig] = null + val numConfigs: Int = 4 override def setUp() { super.setUp() + val props = createBrokerConfigs(numConfigs, zkConnect) + val configs: Seq[KafkaConfig] = props.map(KafkaConfig.fromProps) + adHocConfigs = configs.takeRight(configs.size - 1) // Started and stopped by individual test cases server1 = TestUtils.createServer(configs.head) + brokerEndPoints = Seq(new Broker(server1.config.brokerId, server1.config.hostName, server1.boundPort()).getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) } override def tearDown() { @@ -65,10 +71,9 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { def testBasicTopicMetadata { // create topic val topic = "test" - 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", + createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) + + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", 2000,0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) @@ -84,15 +89,11 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { // create topic val topic1 = "testGetAllTopicMetadata1" val topic2 = "testGetAllTopicMetadata2" - AdminUtils.createTopic(zkClient, topic1, 1, 1) - AdminUtils.createTopic(zkClient, topic2, 1, 1) - - // wait for leader to be elected for both topics - TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic1, 0, 1000) - TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic2, 0, 1000) + createTopic(zkClient, topic1, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) + createTopic(zkClient, topic2, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) // issue metadata request with empty list of topics - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokers, "TopicMetadataTest-testGetAllTopicMetadata", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokerEndPoints, "TopicMetadataTest-testGetAllTopicMetadata", 2000, 0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(2, topicsMetadata.size) @@ -111,7 +112,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { def testAutoCreateTopic { // auto create topic val topic = "testAutoCreateTopic" - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testAutoCreateTopic", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testAutoCreateTopic", 2000,0).topicsMetadata assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode) assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) @@ -119,11 +120,11 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(0, topicsMetadata.head.partitionsMetadata.size) // wait for leader to be elected - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) - TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0, 1000) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0) // retry the metadata for the auto created topic - topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", + topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", 2000,0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) @@ -133,4 +134,120 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(1, partitionMetadata.head.replicas.size) assertTrue(partitionMetadata.head.leader.isDefined) } + + private def checkIsr(servers: Seq[KafkaServer]): Unit = { + val activeBrokers: Seq[KafkaServer] = servers.filter(x => x.brokerState.currentState != NotRunning.state) + val expectedIsr: Seq[BrokerEndPoint] = activeBrokers.map( + x => new BrokerEndPoint(x.config.brokerId, + if (x.config.hostName.nonEmpty) x.config.hostName else "localhost", + x.boundPort()) + ) + + // Assert that topic metadata at new brokers is updated correctly + activeBrokers.foreach(x => { + var metadata: TopicMetadataResponse = new TopicMetadataResponse(Seq(), Seq(), -1) + waitUntilTrue(() => { + metadata = ClientUtils.fetchTopicMetadata( + Set.empty, + Seq(new BrokerEndPoint( + x.config.brokerId, + if (x.config.hostName.nonEmpty) x.config.hostName else "localhost", + x.boundPort())), + "TopicMetadataTest-testBasicTopicMetadata", + 2000, 0) + metadata.topicsMetadata.nonEmpty && + metadata.topicsMetadata.head.partitionsMetadata.nonEmpty && + expectedIsr == metadata.topicsMetadata.head.partitionsMetadata.head.isr + }, + "Topic metadata is not correctly updated for broker " + x + ".\n" + + "Expected ISR: " + expectedIsr + "\n" + + "Actual ISR : " + (if (metadata.topicsMetadata.nonEmpty && + metadata.topicsMetadata.head.partitionsMetadata.nonEmpty) + metadata.topicsMetadata.head.partitionsMetadata.head.isr + else + "")) + }) + } + + + def testIsrAfterBrokerShutDownAndJoinsBack { + val numBrokers = 2 //just 2 brokers are enough for the test + + // start adHoc brokers + val adHocServers = adHocConfigs.take(numBrokers - 1).map(p => createServer(p)) + val allServers: Seq[KafkaServer] = Seq(server1) ++ adHocServers + + // create topic + val topic: String = "test" + AdminUtils.createTopic(zkClient, topic, 1, numBrokers) + + // shutdown a broker + adHocServers.last.shutdown() + adHocServers.last.awaitShutdown() + + // startup a broker + adHocServers.last.startup() + + // check metadata is still correct and updated at all brokers + checkIsr(allServers) + + // shutdown adHoc brokers + adHocServers.map(p => p.shutdown()) + } + + private def checkMetadata(servers: Seq[KafkaServer], expectedBrokersCount: Int): Unit = { + var topicMetadata: TopicMetadataResponse = new TopicMetadataResponse(Seq(), Seq(), -1) + + // Get topic metadata from old broker + // Wait for metadata to get updated by checking metadata from a new broker + waitUntilTrue(() => { + topicMetadata = ClientUtils.fetchTopicMetadata( + Set.empty, brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", 2000, 0) + topicMetadata.brokers.size == expectedBrokersCount}, + "Alive brokers list is not correctly propagated by coordinator to brokers" + ) + + // Assert that topic metadata at new brokers is updated correctly + servers.filter(x => x.brokerState.currentState != NotRunning.state).foreach(x => + waitUntilTrue(() => + topicMetadata == ClientUtils.fetchTopicMetadata( + Set.empty, + Seq(new Broker(x.config.brokerId, + x.config.hostName, + x.boundPort()).getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), + "TopicMetadataTest-testBasicTopicMetadata", + 2000, 0), "Topic metadata is not correctly updated")) + } + + + def testAliveBrokerListWithNoTopics { + checkMetadata(Seq(server1), 1) + } + + def testAliveBrokersListWithNoTopicsAfterNewBrokerStartup { + var adHocServers = adHocConfigs.takeRight(adHocConfigs.size - 1).map(p => createServer(p)) + + checkMetadata(adHocServers, numConfigs - 1) + + // Add a broker + adHocServers = adHocServers ++ Seq(createServer(adHocConfigs.head)) + + checkMetadata(adHocServers, numConfigs) + adHocServers.map(p => p.shutdown()) + } + + + def testAliveBrokersListWithNoTopicsAfterABrokerShutdown { + val adHocServers = adHocConfigs.map(p => createServer(p)) + + checkMetadata(adHocServers, numConfigs) + + // Shutdown a broker + adHocServers.last.shutdown() + adHocServers.last.awaitShutdown() + + checkMetadata(adHocServers, numConfigs - 1) + + adHocServers.map(p => p.shutdown()) + } } diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala new file mode 100755 index 0000000000000..e4bf2df48dd59 --- /dev/null +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -0,0 +1,268 @@ +/** + * 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.integration + +import org.apache.kafka.common.config.ConfigException + +import scala.collection.mutable.MutableList +import scala.util.Random +import org.apache.log4j.{Level, Logger} +import org.scalatest.junit.JUnit3Suite +import java.util.Properties +import junit.framework.Assert._ +import kafka.admin.AdminUtils +import kafka.common.FailedToSendMessageException +import kafka.consumer.{Consumer, ConsumerConfig, ConsumerTimeoutException} +import kafka.producer.{KeyedMessage, Producer} +import kafka.serializer.StringDecoder +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.CoreUtils +import kafka.utils.TestUtils._ +import kafka.zk.ZooKeeperTestHarness + +class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { + val brokerId1 = 0 + val brokerId2 = 1 + + // controlled shutdown is needed for these tests, but we can trim the retry count and backoff interval to + // reduce test execution time + val enableControlledShutdown = true + + var configProps1: Properties = null + var configProps2: Properties = null + + var configs: Seq[KafkaConfig] = Seq.empty[KafkaConfig] + var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] + + val random = new Random() + val topic = "topic" + random.nextLong + val partitionId = 0 + + val kafkaApisLogger = Logger.getLogger(classOf[kafka.server.KafkaApis]) + val networkProcessorLogger = Logger.getLogger(classOf[kafka.network.Processor]) + val syncProducerLogger = Logger.getLogger(classOf[kafka.producer.SyncProducer]) + val eventHandlerLogger = Logger.getLogger(classOf[kafka.producer.async.DefaultEventHandler[Object, Object]]) + + override def setUp() { + super.setUp() + + configProps1 = createBrokerConfig(brokerId1, zkConnect) + configProps2 = createBrokerConfig(brokerId2, zkConnect) + + for (configProps <- List(configProps1, configProps2)) { + configProps.put("controlled.shutdown.enable", String.valueOf(enableControlledShutdown)) + configProps.put("controlled.shutdown.max.retries", String.valueOf(1)) + configProps.put("controlled.shutdown.retry.backoff.ms", String.valueOf(1000)) + } + + // temporarily set loggers to a higher level so that tests run quietly + kafkaApisLogger.setLevel(Level.FATAL) + networkProcessorLogger.setLevel(Level.FATAL) + syncProducerLogger.setLevel(Level.FATAL) + eventHandlerLogger.setLevel(Level.FATAL) + } + + override def tearDown() { + servers.foreach(server => shutdownServer(server)) + servers.foreach(server => CoreUtils.rm(server.config.logDirs)) + + // restore log levels + kafkaApisLogger.setLevel(Level.ERROR) + networkProcessorLogger.setLevel(Level.ERROR) + syncProducerLogger.setLevel(Level.ERROR) + eventHandlerLogger.setLevel(Level.ERROR) + + super.tearDown() + } + + private def startBrokers(cluster: Seq[Properties]) { + for (props <- cluster) { + val config = KafkaConfig.fromProps(props) + val server = createServer(config) + configs ++= List(config) + servers ++= List(server) + } + } + + def testUncleanLeaderElectionEnabled { + // unclean leader election is enabled by default + startBrokers(Seq(configProps1, configProps2)) + + // create topic with 1 partition, 2 replicas, one on each broker + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2))) + + verifyUncleanLeaderElectionEnabled + } + + def testUncleanLeaderElectionDisabled { + // disable unclean leader election + configProps1.put("unclean.leader.election.enable", String.valueOf(false)) + configProps2.put("unclean.leader.election.enable", String.valueOf(false)) + startBrokers(Seq(configProps1, configProps2)) + + // create topic with 1 partition, 2 replicas, one on each broker + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2))) + + verifyUncleanLeaderElectionDisabled + } + + def testUncleanLeaderElectionEnabledByTopicOverride { + // disable unclean leader election globally, but enable for our specific test topic + configProps1.put("unclean.leader.election.enable", String.valueOf(false)) + configProps2.put("unclean.leader.election.enable", String.valueOf(false)) + startBrokers(Seq(configProps1, configProps2)) + + // create topic with 1 partition, 2 replicas, one on each broker, and unclean leader election enabled + val topicProps = new Properties() + topicProps.put("unclean.leader.election.enable", String.valueOf(true)) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), + topicProps) + + verifyUncleanLeaderElectionEnabled + } + + def testCleanLeaderElectionDisabledByTopicOverride { + // enable unclean leader election globally, but disable for our specific test topic + configProps1.put("unclean.leader.election.enable", String.valueOf(true)) + configProps2.put("unclean.leader.election.enable", String.valueOf(true)) + startBrokers(Seq(configProps1, configProps2)) + + // create topic with 1 partition, 2 replicas, one on each broker, and unclean leader election disabled + val topicProps = new Properties() + topicProps.put("unclean.leader.election.enable", String.valueOf(false)) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)), + topicProps) + + verifyUncleanLeaderElectionDisabled + } + + def testUncleanLeaderElectionInvalidTopicOverride { + startBrokers(Seq(configProps1)) + + // create topic with an invalid value for unclean leader election + val topicProps = new Properties() + topicProps.put("unclean.leader.election.enable", "invalid") + + intercept[ConfigException] { + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1)), topicProps) + } + } + + def verifyUncleanLeaderElectionEnabled { + // wait until leader is elected + val leaderIdOpt = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) + assertTrue("Leader should get elected", leaderIdOpt.isDefined) + val leaderId = leaderIdOpt.get + debug("Leader for " + topic + " is elected to be: %s".format(leaderId)) + assertTrue("Leader id is set to expected value for topic: " + topic, leaderId == brokerId1 || leaderId == brokerId2) + + // the non-leader broker is the follower + val followerId = if (leaderId == brokerId1) brokerId2 else brokerId1 + debug("Follower for " + topic + " is: %s".format(followerId)) + + sendMessage(servers, topic, "first") + waitUntilMetadataIsPropagated(servers, topic, partitionId) + assertEquals(List("first"), consumeAllMessages(topic)) + + // shutdown follower server + servers.filter(server => server.config.brokerId == followerId).map(server => shutdownServer(server)) + + sendMessage(servers, topic, "second") + assertEquals(List("first", "second"), consumeAllMessages(topic)) + + // shutdown leader and then restart follower + servers.filter(server => server.config.brokerId == leaderId).map(server => shutdownServer(server)) + servers.filter(server => server.config.brokerId == followerId).map(server => server.startup()) + + // wait until new leader is (uncleanly) elected + waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(followerId)) + + sendMessage(servers, topic, "third") + + // second message was lost due to unclean election + assertEquals(List("first", "third"), consumeAllMessages(topic)) + } + + def verifyUncleanLeaderElectionDisabled { + // wait until leader is elected + val leaderIdOpt = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) + assertTrue("Leader should get elected", leaderIdOpt.isDefined) + val leaderId = leaderIdOpt.get + debug("Leader for " + topic + " is elected to be: %s".format(leaderId)) + assertTrue("Leader id is set to expected value for topic: " + topic, leaderId == brokerId1 || leaderId == brokerId2) + + // the non-leader broker is the follower + val followerId = if (leaderId == brokerId1) brokerId2 else brokerId1 + debug("Follower for " + topic + " is: %s".format(followerId)) + + sendMessage(servers, topic, "first") + waitUntilMetadataIsPropagated(servers, topic, partitionId) + assertEquals(List("first"), consumeAllMessages(topic)) + + // shutdown follower server + servers.filter(server => server.config.brokerId == followerId).map(server => shutdownServer(server)) + + sendMessage(servers, topic, "second") + assertEquals(List("first", "second"), consumeAllMessages(topic)) + + // shutdown leader and then restart follower + servers.filter(server => server.config.brokerId == leaderId).map(server => shutdownServer(server)) + servers.filter(server => server.config.brokerId == followerId).map(server => server.startup()) + + // verify that unclean election to non-ISR follower does not occur + waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(-1)) + + // message production and consumption should both fail while leader is down + intercept[FailedToSendMessageException] { + sendMessage(servers, topic, "third") + } + assertEquals(List.empty[String], consumeAllMessages(topic)) + + // restart leader temporarily to send a successfully replicated message + servers.filter(server => server.config.brokerId == leaderId).map(server => server.startup()) + waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(leaderId)) + + sendMessage(servers, topic, "third") + waitUntilMetadataIsPropagated(servers, topic, partitionId) + servers.filter(server => server.config.brokerId == leaderId).map(server => shutdownServer(server)) + + // verify clean leader transition to ISR follower + waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(followerId)) + + // verify messages can be consumed from ISR follower that was just promoted to leader + assertEquals(List("first", "second", "third"), consumeAllMessages(topic)) + } + + private def shutdownServer(server: KafkaServer) = { + server.shutdown() + server.awaitShutdown() + } + + private def consumeAllMessages(topic: String) : List[String] = { + // use a fresh consumer group every time so that we don't need to mess with disabling auto-commit or + // resetting the ZK offset + val consumerProps = createConsumerProperties(zkConnect, "group" + random.nextLong, "id", 1000) + val consumerConnector = Consumer.create(new ConsumerConfig(consumerProps)) + val messageStream = consumerConnector.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) + + val messages = getMessages(messageStream) + consumerConnector.shutdown + + messages + } +} 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 43af649f32976..74c761dec7afc 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.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 @@ -17,34 +17,37 @@ package kafka.javaapi.consumer -import junit.framework.Assert._ -import kafka.integration.KafkaServerTestHarness +import java.util.Properties + import kafka.server._ -import org.scalatest.junit.JUnit3Suite -import scala.collection.JavaConversions -import org.apache.log4j.{Level, Logger} import kafka.message._ import kafka.serializer._ +import kafka.integration.KafkaServerTestHarness import kafka.producer.KeyedMessage import kafka.javaapi.producer.Producer import kafka.utils.IntEncoder -import kafka.utils.TestUtils._ import kafka.utils.{Logging, TestUtils} import kafka.consumer.{KafkaStream, ConsumerConfig} import kafka.zk.ZooKeeperTestHarness +import kafka.common.MessageStreamsExistException -class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { +import scala.collection.JavaConversions + +import org.scalatest.junit.JUnit3Suite +import org.apache.log4j.{Level, Logger} +import junit.framework.Assert._ - val zookeeperConnect = zkConnect + +class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { val numNodes = 2 val numParts = 2 val topic = "topic1" - val configs = - for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) { - override val numPartitions = numParts - override val zkConnect = zookeeperConnect - } + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) + + def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps(_, overridingProps)) + val group = "group1" val consumer1 = "consumer1" val nMessages = 2 @@ -52,70 +55,60 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar def testBasic() { val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) requestHandlerLogger.setLevel(Level.FATAL) - var actualMessages: List[Message] = Nil - // send some messages to each broker - val sentMessages1 = sendMessages(nMessages, "batch1") + // create the topic + TestUtils.createTopic(zkClient, topic, numParts, 1, servers) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) - waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) + // send some messages to each broker + val sentMessages1 = sendMessages(servers, nMessages, "batch1") // create a consumer - val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zookeeperConnect, group, consumer1)) + val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(toJavaMap(Map(topic -> numNodes*numParts/2)), new StringDecoder(), new StringDecoder()) val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) assertEquals(sentMessages1.sorted, receivedMessages1.sorted) + // call createMesssageStreams twice should throw MessageStreamsExistException + try { + val topicMessageStreams2 = zkConsumerConnector1.createMessageStreams(toJavaMap(Map(topic -> numNodes*numParts/2)), new StringDecoder(), new StringDecoder()) + fail("Should fail with MessageStreamsExistException") + } catch { + case e: MessageStreamsExistException => // expected + } zkConsumerConnector1.shutdown info("all consumer connectors stopped") requestHandlerLogger.setLevel(Level.ERROR) } - def sendMessages(conf: KafkaConfig, - messagesPerNode: Int, - header: String, - compressed: CompressionCodec): List[String] = { + def sendMessages(servers: Seq[KafkaServer], + messagesPerNode: Int, + header: String): List[String] = { var messages: List[String] = Nil - val producer: kafka.producer.Producer[Int, String] = - TestUtils.createProducer(TestUtils.getBrokerListStrFromConfigs(configs), new StringEncoder(), new IntEncoder()) - val javaProducer: Producer[Int, String] = new kafka.javaapi.producer.Producer(producer) - for (partition <- 0 until numParts) { - val ms = 0.until(messagesPerNode).map(x => header + conf.brokerId + "-" + partition + "-" + x) - messages ++= ms - import JavaConversions._ - javaProducer.send(ms.map(new KeyedMessage[Int, String](topic, partition, _)): java.util.List[KeyedMessage[Int, String]]) + for(server <- servers) { + val producer: kafka.producer.Producer[Int, String] = + TestUtils.createProducer(TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[IntEncoder].getName) + val javaProducer: Producer[Int, String] = new kafka.javaapi.producer.Producer(producer) + for (partition <- 0 until numParts) { + val ms = 0.until(messagesPerNode).map(x => header + server.config.brokerId + "-" + partition + "-" + x) + messages ++= ms + import JavaConversions._ + javaProducer.send(ms.map(new KeyedMessage[Int, String](topic, partition, _)): java.util.List[KeyedMessage[Int, String]]) + } + javaProducer.close } - javaProducer.close messages } - def sendMessages(messagesPerNode: Int, - header: String, - compressed: CompressionCodec = NoCompressionCodec): List[String] = { - var messages: List[String] = Nil - for(conf <- configs) - messages ++= sendMessages(conf, messagesPerNode, header, compressed) - messages - } - - def getMessages(nMessagesPerThread: Int, + def getMessages(nMessagesPerThread: Int, jTopicMessageStreams: java.util.Map[String, java.util.List[KafkaStream[String, String]]]): List[String] = { var messages: List[String] = Nil import scala.collection.JavaConversions._ - val topicMessageStreams: collection.mutable.Map[String, java.util.List[KafkaStream[String, String]]] = jTopicMessageStreams - for ((topic, messageStreams) <- topicMessageStreams) { - for (messageStream <- messageStreams) { - val iterator = messageStream.iterator - for (i <- 0 until nMessagesPerThread) { - assertTrue(iterator.hasNext) - val message = iterator.next.message - messages ::= message - debug("received message: " + message) - } - } - } + val topicMessageStreams = jTopicMessageStreams.mapValues(_.toList) + messages = TestUtils.getMessages(topicMessageStreams, nMessagesPerThread) messages } @@ -123,5 +116,5 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val javaMap = new java.util.HashMap[String, java.lang.Integer]() scalaMap.foreach(m => javaMap.put(m._1, m._2.asInstanceOf[java.lang.Integer])) javaMap - } + } } diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala new file mode 100755 index 0000000000000..6180b871025ae --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala @@ -0,0 +1,85 @@ +/** + * 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._ +import kafka.message._ +import org.scalatest.junit.JUnitSuite +import org.junit._ +import org.junit.Assert._ +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.runners.Parameterized.Parameters +import java.util.{Properties, Collection, ArrayList} +import kafka.server.KafkaConfig +import org.apache.kafka.common.record.CompressionType +import scala.collection.JavaConversions._ + +@RunWith(value = classOf[Parameterized]) +class BrokerCompressionTest(messageCompression: String, brokerCompression: String) extends JUnitSuite { + + var logDir: File = null + val time = new MockTime(0) + val logConfig = LogConfig() + + @Before + def setUp() { + logDir = TestUtils.tempDir() + } + + @After + def tearDown() { + CoreUtils.rm(logDir) + } + + /** + * Test broker-side compression configuration + */ + @Test + def testBrokerSideCompression() { + val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression) + val logProps = new Properties() + logProps.put(LogConfig.CompressionTypeProp,brokerCompression) + /*configure broker-side compression */ + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) + + /* append two messages */ + log.append(new ByteBufferMessageSet(messageCompressionCode, new Message("hello".getBytes), new Message("there".getBytes))) + + def readMessage(offset: Int) = log.read(offset, 4096).messageSet.head.message + + if (!brokerCompression.equals("producer")) { + val brokerCompressionCode = BrokerCompressionCodec.getCompressionCodec(brokerCompression) + assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode, readMessage(0).compressionCodec) + } + else + assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode, readMessage(0).compressionCodec) + + } + +} + +object BrokerCompressionTest { + @Parameters + def parameters: Collection[Array[String]] = { + for (brokerCompression <- BrokerCompressionCodec.brokerCompressionOptions; + messageCompression <- CompressionType.values + ) yield Array(messageCompression.name, brokerCompression) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala old mode 100644 new mode 100755 index 51cd94b316764..0e2a6a1e8e6d0 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -17,6 +17,8 @@ package kafka.log +import java.util.Properties + import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Test} @@ -26,6 +28,8 @@ import scala.collection._ import kafka.common._ import kafka.utils._ import kafka.message._ +import java.util.concurrent.atomic.AtomicLong +import org.apache.kafka.common.utils.Utils /** * Unit tests for the log cleaning logic @@ -33,13 +37,17 @@ import kafka.message._ class CleanerTest extends JUnitSuite { val dir = TestUtils.tempDir() - val logConfig = LogConfig(segmentSize=1024, maxIndexSize=1024, dedupe=true) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + val logConfig = LogConfig(logProps) val time = new MockTime() val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time) @After def teardown() { - Utils.rm(dir) + CoreUtils.rm(dir) } /** @@ -48,8 +56,11 @@ class CleanerTest extends JUnitSuite { @Test def testCleanSegments() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 1024)) - + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + // append messages to the log until we have four segments while(log.numberOfSegments < 4) log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) @@ -70,7 +81,10 @@ class CleanerTest extends JUnitSuite { @Test def testCleaningWithDeletes() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 1024)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append messages with the keys 0 through N while(log.numberOfSegments < 2) @@ -90,10 +104,48 @@ class CleanerTest extends JUnitSuite { assertTrue("None of the keys we deleted should still exist.", (0 until leo.toInt by 2).forall(!keys.contains(_))) } + + @Test + def testCleaningWithUnkeyedMessages { + val cleaner = makeCleaner(Int.MaxValue) + + // create a log with compaction turned off so we can append unkeyed messages + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Delete) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + + // append unkeyed messages + while(log.numberOfSegments < 2) + log.append(unkeyedMessage(log.logEndOffset.toInt)) + val numInvalidMessages = unkeyedMessageCountInLog(log) + + val sizeWithUnkeyedMessages = log.size + + // append keyed messages + while(log.numberOfSegments < 3) + log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) + + val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages + + // turn on compaction and compact the log + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val compactedLog = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0)) + + assertEquals("Log should only contain keyed messages after cleaning.", 0, unkeyedMessageCountInLog(log)) + assertEquals("Log should only contain keyed messages after cleaning.", expectedSizeAfterCleaning, log.size) + assertEquals("Cleaner should have seen %d invalid messages.", numInvalidMessages, cleaner.stats.invalidMessagesRead) + } /* 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 keysInLog(log: Log): Iterable[Int] = + log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => TestUtils.readString(m.message.key).toInt)) + + def unkeyedMessageCountInLog(log: Log) = + log.logSegments.map(s => s.log.filter(!_.message.isNull).count(m => !m.message.hasKey)).sum def abortCheckDone(topicAndPartition: TopicAndPartition) { throw new LogCleaningAbortedException() @@ -105,7 +157,10 @@ class CleanerTest extends JUnitSuite { @Test def testCleanSegmentsWithAbort() { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) - val log = makeLog(config = logConfig.copy(segmentSize = 1024)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append messages to the log until we have four segments while(log.numberOfSegments < 4) @@ -125,12 +180,16 @@ class CleanerTest extends JUnitSuite { @Test def testSegmentGrouping() { val cleaner = makeCleaner(Int.MaxValue) - val log = makeLog(config = logConfig.copy(segmentSize = 300, indexInterval = 1)) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append some messages to the log var i = 0 while(log.numberOfSegments < 10) { - log.append(TestUtils.singleMessageSet("hello".getBytes)) + log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) i += 1 } @@ -165,6 +224,56 @@ class CleanerTest extends JUnitSuite { assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize)) } + /** + * Validate the logic for grouping log segments together for cleaning when only a small number of + * messages are retained, but the range of offsets is greater than Int.MaxValue. A group should not + * contain a range of offsets greater than Int.MaxValue to ensure that relative offsets can be + * stored in 4 bytes. + */ + @Test + def testSegmentGroupingWithSparseOffsets() { + val cleaner = makeCleaner(Int.MaxValue) + + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + + // fill up first segment + while (log.numberOfSegments == 1) + log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) + + // forward offset and append message to next segment at offset Int.MaxValue + val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new AtomicLong(Int.MaxValue-1), new Message("hello".getBytes, "hello".getBytes)) + log.append(messageSet, assignOffsets = false) + log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) + assertEquals(Int.MaxValue, log.activeSegment.index.lastOffset) + + // grouping should result in a single group with maximum relative offset of Int.MaxValue + var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue) + assertEquals(1, groups.size) + + // append another message, making last offset of second segment > Int.MaxValue + log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) + + // grouping should not group the two segments to ensure that maximum relative offset in each group <= Int.MaxValue + groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue) + assertEquals(2, groups.size) + checkSegmentOrder(groups) + + // append more messages, creating new segments, further grouping should still occur + while (log.numberOfSegments < 4) + log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) + + groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue) + assertEquals(log.numberOfSegments-1, groups.size) + for (group <- groups) + assertTrue("Relative offset greater than Int.MaxValue", group.last.index.lastOffset - group.head.index.baseOffset <= Int.MaxValue) + checkSegmentOrder(groups) + + } + private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]) { val offsets = groups.flatMap(_.map(_.baseOffset)) assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets) @@ -196,6 +305,110 @@ class CleanerTest extends JUnitSuite { checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt) } + + /** + * Tests recovery if broker crashes at the following stages during the cleaning sequence + *
              + *
            1. Cleaner has created .cleaned log containing multiple segments, swap sequence not yet started + *
            2. .cleaned log renamed to .swap, old segment files not yet renamed to .deleted + *
            3. .cleaned log renamed to .swap, old segment files renamed to .deleted, but not yet deleted + *
            4. .swap suffix removed, completing the swap, but async delete of .deleted files not yet complete + *
            + */ + @Test + def testRecoveryAfterCrash() { + val cleaner = makeCleaner(Int.MaxValue) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer) + + val config = LogConfig.fromProps(logConfig.originals, logProps) + + def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = { + // Recover log file and check that after recovery, keys are as expected + // and all temporary files have been deleted + val recoveredLog = makeLog(config = config) + time.sleep(config.fileDeleteDelayMs + 1) + for (file <- dir.listFiles) { + assertFalse("Unexpected .deleted file after recovery", file.getName.endsWith(Log.DeletedFileSuffix)) + assertFalse("Unexpected .cleaned file after recovery", file.getName.endsWith(Log.CleanedFileSuffix)) + assertFalse("Unexpected .swap file after recovery", file.getName.endsWith(Log.SwapFileSuffix)) + } + assertEquals(expectedKeys, keysInLog(recoveredLog)) + recoveredLog + } + + // create a log and append some messages + var log = makeLog(config = config) + var messageCount = 0 + while(log.numberOfSegments < 10) { + log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) + messageCount += 1 + } + val allKeys = keysInLog(log) + + // pretend we have odd-numbered keys + val offsetMap = new FakeOffsetMap(Int.MaxValue) + for (k <- 1 until messageCount by 2) + offsetMap.put(key(k), Long.MaxValue) + + // clean the log + cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L) + var cleanedKeys = keysInLog(log) + + // 1) Simulate recovery just after .cleaned file is created, before rename to .swap + // On recovery, clean operation is aborted. All messages should be present in the log + log.logSegments.head.changeFileSuffixes("", Log.CleanedFileSuffix) + for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) { + file.renameTo(new File(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + } + log = recoverAndCheck(config, allKeys) + + // clean again + cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L) + cleanedKeys = keysInLog(log) + + // 2) Simulate recovery just after swap file is created, before old segment files are + // renamed to .deleted. Clean operation is resumed during recovery. + log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix) + for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) { + file.renameTo(new File(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + } + log = recoverAndCheck(config, cleanedKeys) + + // add some more messages and clean the log again + while(log.numberOfSegments < 10) { + log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) + messageCount += 1 + } + for (k <- 1 until messageCount by 2) + offsetMap.put(key(k), Long.MaxValue) + cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L) + cleanedKeys = keysInLog(log) + + // 3) Simulate recovery after swap file is created and old segments files are renamed + // to .deleted. Clean operation is resumed during recovery. + log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix) + log = recoverAndCheck(config, cleanedKeys) + + // add some more messages and clean the log again + while(log.numberOfSegments < 10) { + log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) + messageCount += 1 + } + for (k <- 1 until messageCount by 2) + offsetMap.put(key(k), Long.MaxValue) + cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L) + cleanedKeys = keysInLog(log) + + // 4) Simulate recovery after swap is complete, but async deletion + // is not yet complete. Clean operation is resumed during recovery. + recoverAndCheck(config, cleanedKeys) + + } + + def makeLog(dir: File = dir, config: LogConfig = logConfig) = new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time) @@ -220,7 +433,10 @@ class CleanerTest extends JUnitSuite { def message(key: Int, value: Int) = new ByteBufferMessageSet(new Message(key=key.toString.getBytes, bytes=value.toString.getBytes)) - + + def unkeyedMessage(value: Int) = + new ByteBufferMessageSet(new Message(bytes=value.toString.getBytes)) + def deleteMessage(key: Int) = new ByteBufferMessageSet(new Message(key=key.toString.getBytes, bytes=null)) diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index cec1caecc5150..02cf66882f406 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -17,6 +17,7 @@ package kafka.log +import java.io._ import java.nio._ import java.util.concurrent.atomic._ import junit.framework.Assert._ @@ -146,5 +147,57 @@ class FileMessageSetTest extends BaseMessageSetTestCases { assertEquals(List(message), messageSet.toList) assertEquals(MessageSet.entrySize(message.message), messageSet.sizeInBytes) } - + + /** + * Test the new FileMessageSet with pre allocate as true + */ + @Test + def testPreallocateTrue() { + val temp = tempFile() + val set = new FileMessageSet(temp, false, 512 *1024 *1024, true) + val position = set.channel.position + val size = set.sizeInBytes() + assertEquals(0, position) + assertEquals(0, size) + assertEquals(512 *1024 *1024, temp.length) + } + + /** + * Test the new FileMessageSet with pre allocate as false + */ + @Test + def testPreallocateFalse() { + val temp = tempFile() + val set = new FileMessageSet(temp, false, 512 *1024 *1024, false) + val position = set.channel.position + val size = set.sizeInBytes() + assertEquals(0, position) + assertEquals(0, size) + assertEquals(0, temp.length) + } + + /** + * Test the new FileMessageSet with pre allocate as true and file has been clearly shut down, the file will be truncate to end of valid data. + */ + @Test + def testPreallocateClearShutdown() { + val temp = tempFile() + val set = new FileMessageSet(temp, false, 512 *1024 *1024, true) + set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*)) + val oldposition = set.channel.position + val oldsize = set.sizeInBytes() + assertEquals(messageSet.sizeInBytes, oldposition) + assertEquals(messageSet.sizeInBytes, oldsize) + set.close() + + val tempReopen = new File(temp.getAbsolutePath()) + val setReopen = new FileMessageSet(tempReopen, true, 512 *1024 *1024, true) + val position = setReopen.channel.position + val size = setReopen.sizeInBytes() + + assertEquals(oldposition, position) + assertEquals(oldposition, size) + assertEquals(oldposition, tempReopen.length) + } + } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala old mode 100644 new mode 100755 index 1de3ef0435d76..381e9aa6fe611 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -18,19 +18,29 @@ package kafka.log import java.io.File -import scala.collection._ -import org.junit._ +import java.util.Properties + import kafka.common.TopicAndPartition -import kafka.utils._ import kafka.message._ -import org.scalatest.junit.JUnitSuite -import junit.framework.Assert._ +import kafka.server.OffsetCheckpoint +import kafka.utils._ +import org.apache.kafka.common.record.CompressionType +import org.junit.Assert._ +import org.junit._ +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.runners.Parameterized.Parameters +import org.scalatest.junit.JUnit3Suite + +import scala.collection._ + /** * This is an integration test that tests the fully integrated log cleaner */ -class LogCleanerIntegrationTest extends JUnitSuite { - +@RunWith(value = classOf[Parameterized]) +class LogCleanerIntegrationTest(compressionCodec: String) extends JUnit3Suite { + val time = new MockTime() val segmentSize = 100 val deleteDelay = 1000 @@ -38,16 +48,16 @@ class LogCleanerIntegrationTest extends JUnitSuite { 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 appends = writeDups(numKeys = 100, numDups = 3, log, CompressionCodec.getCompressionCodec(compressionCodec)) 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) @@ -55,29 +65,47 @@ class LogCleanerIntegrationTest extends JUnitSuite { 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 appends2 = appends ++ writeDups(numKeys = 100, numDups = 3, log, CompressionCodec.getCompressionCodec(compressionCodec)) 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) + + // simulate deleting a partition, by removing it from logs + // force a checkpoint + // and make sure its gone from checkpoint file + + cleaner.logs.remove(topics(0)) + + cleaner.updateCheckpoints(logDir) + val checkpoints = new OffsetCheckpoint(new File(logDir,cleaner.cleanerManager.offsetCheckpointFile)).read() + + // we expect partition 0 to be gone + assert(!checkpoints.contains(topics(0))) 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 + for (segment <- log.logSegments; entry <- segment.log; messageAndOffset <- { + // create single message iterator or deep iterator depending on compression codec + if (entry.message.compressionCodec == NoCompressionCodec) + Stream.cons(entry, Stream.empty).iterator + else + ByteBufferMessageSet.deepIterator(entry.message) + }) yield { + val key = TestUtils.readString(messageAndOffset.message.key).toInt + val value = TestUtils.readString(messageAndOffset.message.payload).toInt key -> value } } - - def writeDups(numKeys: Int, numDups: Int, log: Log): Seq[(Int, Int)] = { + + def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec): 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) + log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, codec = codec, key = key.toString.getBytes), assignOffsets = true) counter += 1 (key, count) } @@ -85,14 +113,14 @@ class LogCleanerIntegrationTest extends JUnitSuite { @After def teardown() { - Utils.rm(logDir) + CoreUtils.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", + defaultPolicy: String = "compact", policyOverrides: Map[String, String] = Map()): LogCleaner = { // create partitions and add them to the pool @@ -100,8 +128,13 @@ class LogCleanerIntegrationTest extends JUnitSuite { for(i <- 0 until parts) { val dir = new File(logDir, "log-" + i) dir.mkdirs() + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 100*1024: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, deleteDelay: java.lang.Integer) + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) val log = new Log(dir = dir, - LogConfig(segmentSize = segmentSize, maxIndexSize = 100*1024, fileDeleteDelayMs = deleteDelay, dedupe = true), + LogConfig(logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time) @@ -114,4 +147,14 @@ class LogCleanerIntegrationTest extends JUnitSuite { time = time) } +} + +object LogCleanerIntegrationTest { + @Parameters + def parameters: java.util.Collection[Array[String]] = { + val list = new java.util.ArrayList[Array[String]]() + for (codec <- CompressionType.values) + list.add(Array(codec.name)) + list + } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala new file mode 100644 index 0000000000000..19dcb47f3f406 --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -0,0 +1,86 @@ +/** + * 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 org.apache.kafka.common.config.ConfigException +import org.junit.{Assert, Test} +import org.scalatest.junit.JUnit3Suite + +class LogConfigTest extends JUnit3Suite { + + @Test + def testFromPropsEmpty() { + val p = new Properties() + val config = LogConfig(p) + Assert.assertEquals(LogConfig(), config) + } + + @Test + def testFromPropsToProps() { + import scala.util.Random._ + val expected = new Properties() + LogConfig.configNames().foreach((name) => { + name match { + case LogConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) + case LogConfig.CompressionTypeProp => expected.setProperty(name, randFrom("producer", "uncompressed", "gzip")) + case LogConfig.CleanupPolicyProp => expected.setProperty(name, randFrom(LogConfig.Compact, LogConfig.Delete)) + case LogConfig.MinCleanableDirtyRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) + case LogConfig.MinInSyncReplicasProp => expected.setProperty(name, (nextInt(Int.MaxValue - 1) + 1).toString) + case LogConfig.RetentionBytesProp => expected.setProperty(name, nextInt().toString) + case LogConfig.RetentionMsProp => expected.setProperty(name, nextLong().toString) + case LogConfig.PreAllocateEnableProp => expected.setProperty(name, randFrom("true", "false")) + case positiveIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString) + } + }) + + val actual = LogConfig(expected).originals + Assert.assertEquals(expected, actual) + } + + @Test + def testFromPropsInvalid() { + LogConfig.configNames().foreach((name) => { + name match { + case LogConfig.UncleanLeaderElectionEnableProp => return + case LogConfig.RetentionBytesProp => assertPropertyInvalid(name, "not_a_number") + case LogConfig.RetentionMsProp => assertPropertyInvalid(name, "not_a_number" ) + case LogConfig.CleanupPolicyProp => assertPropertyInvalid(name, "true", "foobar"); + case LogConfig.MinCleanableDirtyRatioProp => assertPropertyInvalid(name, "not_a_number", "-0.1", "1.2") + case LogConfig.MinInSyncReplicasProp => assertPropertyInvalid(name, "not_a_number", "0", "-1") + case positiveIntProperty => assertPropertyInvalid(name, "not_a_number", "-1") + } + }) + } + + private def assertPropertyInvalid(name: String, values: AnyRef*) { + values.foreach((value) => { + val props = new Properties + props.setProperty(name, value.toString) + intercept[ConfigException] { + LogConfig(props) + } + }) + } + + private def randFrom[T](choices: T*): T = { + import scala.util.Random + choices(Random.nextInt(choices.size)) + } +} diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala old mode 100644 new mode 100755 index b4bee33191ebc..a13f2bef8ee8c --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -18,38 +18,33 @@ package kafka.log import java.io._ +import java.util.Properties import junit.framework.Assert._ import org.junit.Test import org.scalatest.junit.JUnit3Suite -import kafka.server.KafkaConfig +import kafka.server.{BrokerState, OffsetCheckpoint} import kafka.common._ import kafka.utils._ -import kafka.server.OffsetCheckpoint class LogManagerTest extends JUnit3Suite { val time: MockTime = new MockTime() val maxRollInterval = 100 val maxLogAgeMs = 10*60*60*1000 - val logConfig = LogConfig(segmentSize = 1024, maxIndexSize = 4096, retentionMs = maxLogAgeMs) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) + logProps.put(LogConfig.RetentionMsProp, maxLogAgeMs: java.lang.Integer) + val logConfig = LogConfig(logProps) var logDir: File = null var logManager: LogManager = null val name = "kafka" val veryLargeLogFlushInterval = 10000000L - val cleanerConfig = CleanerConfig(enableCleaner = false) override def setUp() { super.setUp() 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 = createLogManager() logManager.startup logDir = logManager.logDirs(0) } @@ -57,8 +52,8 @@ class LogManagerTest extends JUnit3Suite { override def tearDown() { if(logManager != null) logManager.shutdown() - Utils.rm(logDir) - logManager.logDirs.map(Utils.rm(_)) + CoreUtils.rm(logDir) + logManager.logDirs.foreach(CoreUtils.rm(_)) super.tearDown() } @@ -104,7 +99,7 @@ class LogManagerTest extends JUnit3Suite { 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) + assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).messageSet.sizeInBytes) try { log.read(0, 1024) @@ -123,9 +118,12 @@ class LogManagerTest extends JUnit3Suite { def testCleanupSegmentsToMaintainSize() { val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes logManager.shutdown() + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer) + logProps.put(LogConfig.RetentionBytesProp, 5L * 10L * setSize + 10L: java.lang.Long) + val config = LogConfig.fromProps(logConfig.originals, logProps) - 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 = createLogManager() logManager.startup // create a log @@ -147,7 +145,7 @@ class LogManagerTest extends JUnit3Suite { 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) + assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).messageSet.sizeInBytes) try { log.read(0, 1024) fail("Should get exception from fetching earlier.") @@ -164,8 +162,11 @@ class LogManagerTest extends JUnit3Suite { @Test def testTimeBasedFlush() { logManager.shutdown() - val config = logConfig.copy(flushMs = 1000) - logManager = new LogManager(Array(logDir), Map(), config, cleanerConfig, 1000L, 10000L, 1000L, time.scheduler, time) + val logProps = new Properties() + logProps.put(LogConfig.FlushMsProp, 1000: java.lang.Integer) + val config = LogConfig.fromProps(logConfig.originals, logProps) + + logManager = createLogManager() logManager.startup val log = logManager.createLog(TopicAndPartition(name, 0), config) val lastFlush = log.lastFlushTime @@ -187,8 +188,8 @@ class LogManagerTest extends JUnit3Suite { TestUtils.tempDir(), TestUtils.tempDir()) logManager.shutdown() - logManager = new LogManager(dirs, Map(), logConfig, cleanerConfig, 1000L, 10000L, 1000L, time.scheduler, time) - + logManager = createLogManager() + // verify that logs are always assigned to the least loaded partition for(partition <- 0 until 20) { logManager.createLog(TopicAndPartition("test", partition), logConfig) @@ -201,32 +202,77 @@ class LogManagerTest extends JUnit3Suite { /** * Test that it is not possible to open two log managers using the same data directory */ + @Test def testTwoLogManagersUsingSameDirFails() { try { - new LogManager(Array(logDir), Map(), logConfig, cleanerConfig, 1000L, 10000L, 1000L, time.scheduler, time) + createLogManager() 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 */ + @Test 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() + verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1), TopicAndPartition("test-b", 1)), logManager) + } + + /** + * Test that recovery points directory checking works with trailing slash + */ + @Test + def testRecoveryDirectoryMappingWithTrailingSlash() { + logManager.shutdown() + logDir = TestUtils.tempDir() + logManager = TestUtils.createLogManager( + logDirs = Array(new File(logDir.getAbsolutePath + File.separator))) + logManager.startup + verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager) + } + + /** + * Test that recovery points directory checking works with relative directory + */ + @Test + def testRecoveryDirectoryMappingWithRelativeDirectory() { + logManager.shutdown() + logDir = new File("data" + File.separator + logDir.getName) + logDir.mkdirs() + logDir.deleteOnExit() + logManager = createLogManager() + logManager.startup + verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager) + } + + + private def verifyCheckpointRecovery(topicAndPartitions: Seq[TopicAndPartition], + logManager: LogManager) { + val logs = topicAndPartitions.map(this.logManager.createLog(_, logConfig)) + logs.foreach(log => { + for(i <- 0 until 50) + log.append(TestUtils.singleMessageSet("test".getBytes())) + + log.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) + + topicAndPartitions.zip(logs).foreach { + case(tp, log) => { + assertEquals("Recovery point should equal checkpoint", checkpoints(tp), log.recoveryPoint) + } + } + } + + + private def createLogManager(logDirs: Array[File] = Array(this.logDir)): LogManager = { + TestUtils.createLogManager( + defaultConfig = logConfig, + logDirs = logDirs, + time = this.time) } } diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 6b7603728ae52..abcd1f07444c6 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -39,7 +39,7 @@ class LogSegmentTest extends JUnit3Suite { val idxFile = TestUtils.tempFile() idxFile.delete() val idx = new OffsetIndex(idxFile, offset, 1000) - val seg = new LogSegment(ms, idx, offset, 10, SystemTime) + val seg = new LogSegment(ms, idx, offset, 10, 0, SystemTime) segments += seg seg } @@ -78,7 +78,7 @@ class LogSegmentTest extends JUnit3Suite { val seg = createSegment(40) val ms = messages(50, "hello", "there", "little", "bee") seg.append(50, ms) - val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None) + val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).messageSet assertEquals(ms.toList, read.toList) } @@ -94,7 +94,7 @@ class LogSegmentTest extends JUnit3Suite { 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) + seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).messageSet.toList) validate(50) validate(51) validate(52) @@ -109,7 +109,7 @@ class LogSegmentTest extends JUnit3Suite { val ms = messages(50, "hello", "there") seg.append(50, ms) val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None) - assertNull("Read beyond the last offset in the segment should give null", null) + assertNull("Read beyond the last offset in the segment should give null", read) } /** @@ -124,7 +124,7 @@ class LogSegmentTest extends JUnit3Suite { val ms2 = messages(60, "alpha", "beta") seg.append(60, ms2) val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) - assertEquals(ms2.toList, read.toList) + assertEquals(ms2.toList, read.messageSet.toList) } /** @@ -142,12 +142,12 @@ class LogSegmentTest extends JUnit3Suite { seg.append(offset+1, ms2) // check that we can read back both messages val read = seg.read(offset, None, 10000) - assertEquals(List(ms1.head, ms2.head), read.toList) + assertEquals(List(ms1.head, ms2.head), read.messageSet.toList) // now truncate off the last message seg.truncateTo(offset + 1) val read2 = seg.read(offset, None, 10000) - assertEquals(1, read2.size) - assertEquals(ms1.head, read2.head) + assertEquals(1, read2.messageSet.size) + assertEquals(ms1.head, read2.messageSet.head) offset += 1 } } @@ -204,7 +204,7 @@ class LogSegmentTest extends JUnit3Suite { 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) + assertEquals(i, seg.read(i, Some(i+1), 1024).messageSet.head.offset) } /** @@ -226,5 +226,57 @@ class LogSegmentTest extends JUnit3Suite { seg.delete() } } - + + /* create a segment with pre allocate */ + def createSegment(offset: Long, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): LogSegment = { + val tempDir = TestUtils.tempDir() + val seg = new LogSegment(tempDir, offset, 10, 1000, 0, SystemTime, fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate) + segments += seg + seg + } + + /* create a segment with pre allocate, put message to it and verify */ + @Test + def testCreateWithInitFileSizeAppendMessage() { + val seg = createSegment(40, false, 512*1024*1024, true) + val ms = messages(50, "hello", "there") + seg.append(50, ms) + val ms2 = messages(60, "alpha", "beta") + seg.append(60, ms2) + val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, read.messageSet.toList) + } + + /* create a segment with pre allocate and clearly shut down*/ + @Test + def testCreateWithInitFileSizeClearShutdown() { + val tempDir = TestUtils.tempDir() + val seg = new LogSegment(tempDir, 40, 10, 1000, 0, SystemTime, false, 512*1024*1024, true) + + val ms = messages(50, "hello", "there") + seg.append(50, ms) + val ms2 = messages(60, "alpha", "beta") + seg.append(60, ms2) + val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, read.messageSet.toList) + val oldSize = seg.log.sizeInBytes() + val oldPosition = seg.log.channel.position + val oldFileSize = seg.log.file.length + assertEquals(512*1024*1024, oldFileSize) + seg.close() + //After close, file should be trimed + assertEquals(oldSize, seg.log.file.length) + + val segReopen = new LogSegment(tempDir, 40, 10, 1000, 0, SystemTime, true, 512*1024*1024, true) + segments += segReopen + + val readAgain = segReopen.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, readAgain.messageSet.toList) + val size = segReopen.log.sizeInBytes() + val position = segReopen.log.channel.position + val fileSize = segReopen.log.file.length + assertEquals(oldPosition, position) + assertEquals(oldSize, size) + assertEquals(size, fileSize) + } } \ 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 old mode 100644 new mode 100755 index 1da1393983d4b..9e26190d99537 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -18,15 +18,14 @@ package kafka.log import java.io._ -import java.util.ArrayList +import java.util.Properties import java.util.concurrent.atomic._ import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} import kafka.message._ -import kafka.common.{MessageSizeTooLargeException, OffsetOutOfRangeException} +import kafka.common.{MessageSizeTooLargeException, OffsetOutOfRangeException, MessageSetSizeTooLargeException} import kafka.utils._ -import scala.Some import kafka.server.KafkaConfig class LogTest extends JUnitSuite { @@ -34,18 +33,18 @@ class LogTest extends JUnitSuite { var logDir: File = null val time = new MockTime(0) var config: KafkaConfig = null - val logConfig = LogConfig() + val logConfig = LogConfig() @Before def setUp() { logDir = TestUtils.tempDir() - val props = TestUtils.createBrokerConfig(0, -1) - config = new KafkaConfig(props) + val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1) + config = KafkaConfig.fromProps(props) } @After def tearDown() { - Utils.rm(logDir) + CoreUtils.rm(logDir) } def createEmptyLogs(dir: File, offsets: Int*) { @@ -63,11 +62,14 @@ class LogTest extends JUnitSuite { def testTimeBasedLogRoll() { val set = TestUtils.singleMessageSet("test".getBytes()) + val logProps = new Properties() + logProps.put(LogConfig.SegmentMsProp, 1 * 60 * 60: java.lang.Long) + // create a log - val log = new Log(logDir, - logConfig.copy(segmentMs = 1 * 60 * 60L), - recoveryPoint = 0L, - scheduler = time.scheduler, + val log = new Log(logDir, + LogConfig(logProps), + recoveryPoint = 0L, + scheduler = time.scheduler, time = time) assertEquals("Log begins with a single empty segment.", 1, log.numberOfSegments) time.sleep(log.config.segmentMs + 1) @@ -89,6 +91,35 @@ class LogTest extends JUnitSuite { assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments) } + /** + * Test for jitter s 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 testTimeBasedLogRollJitter() { + val set = TestUtils.singleMessageSet("test".getBytes()) + val maxJitter = 20 * 60L + + val logProps = new Properties() + logProps.put(LogConfig.SegmentMsProp, 1 * 60 * 60: java.lang.Long) + logProps.put(LogConfig.SegmentJitterMsProp, maxJitter: java.lang.Long) + // create a log + val log = new Log(logDir, + LogConfig(logProps), + recoveryPoint = 0L, + scheduler = time.scheduler, + time = time) + assertEquals("Log begins with a single empty segment.", 1, log.numberOfSegments) + log.append(set) + + time.sleep(log.config.segmentMs - maxJitter) + log.append(set) + assertEquals("Log does not roll on this append because it occurs earlier than max jitter", 1, log.numberOfSegments) + time.sleep(maxJitter - log.activeSegment.rollJitterMs + 1) + log.append(set) + assertEquals("Log should roll after segmentMs adjusted by random jitter", 2, log.numberOfSegments) + } + /** * Test that appending more than the maximum segment size rolls the log */ @@ -99,8 +130,10 @@ class LogTest extends JUnitSuite { val msgPerSeg = 10 val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) // create a log - val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), recoveryPoint = 0L, time.scheduler, time = time) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) // segments expire in size @@ -125,60 +158,66 @@ class LogTest extends JUnitSuite { */ @Test def testAppendAndReadWithSequentialOffsets() { - val log = new Log(logDir, logConfig.copy(segmentSize = 71), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), 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 + val read = log.read(i, 100, Some(i+1)).messageSet.head assertEquals("Offset read should match order appended.", i, read.offset) assertEquals("Message should match appended.", messages(i), read.message) } - assertEquals("Reading beyond the last message returns nothing.", 0, log.read(messages.length, 100, None).size) + assertEquals("Reading beyond the last message returns nothing.", 0, log.read(messages.length, 100, None).messageSet.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 logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), 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 + val read = log.read(i, 100, None).messageSet.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 + * 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) - + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), 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))) - + 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) + + assertEquals("A read should now return the last message in the log", log.logEndOffset-1, log.read(1, 200, None).messageSet.head.offset) } - + /** * Test reading at the boundary of the log, specifically * - reading from the logEndOffset should give an empty message set @@ -187,8 +226,10 @@ class LogTest extends JUnitSuite { @Test def testReadOutOfRange() { createEmptyLogs(logDir, 1024) - 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) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) + assertEquals("Reading just beyond end of log should produce 0 byte read.", 0, log.read(1024, 1000).messageSet.sizeInBytes) try { log.read(0, 1024) fail("Expected exception on invalid read.") @@ -210,7 +251,9 @@ class LogTest extends JUnitSuite { @Test def testLogRolls() { /* create a multipart log with 100 messages */ - val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val numMessages = 100 val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes)) messageSets.foreach(log.append(_)) @@ -219,41 +262,43 @@ class LogTest extends JUnitSuite { /* do successive reads to ensure all our messages are there */ var offset = 0L for(i <- 0 until numMessages) { - val messages = log.read(offset, 1024*1024) + val messages = log.read(offset, 1024*1024).messageSet assertEquals("Offsets not equal", offset, messages.head.offset) assertEquals("Messages not equal at offset " + offset, messageSets(i).head.message, messages.head.message) offset = messages.head.offset + 1 } - val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)) + val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).messageSet 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 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, logConfig.copy(segmentSize = 10), recoveryPoint = 0L, time.scheduler, time = time) - + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), 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) - + + def read(offset: Int) = ByteBufferMessageSet.deepIterator(log.read(offset, 4096).messageSet.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) + assertEquals("Read at offset 0 should produce 0", 0, read(0).next().offset) + assertEquals("Read at offset 1 should produce 0", 0, read(1).next().offset) + assertEquals("Read at offset 2 should produce 2", 2, read(2).next().offset) + assertEquals("Read at offset 3 should produce 2", 2, read(3).next().offset) } - + /** * Test garbage collecting old segments */ @@ -262,10 +307,12 @@ class LogTest extends JUnitSuite { 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) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), 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) @@ -276,17 +323,82 @@ class LogTest extends JUnitSuite { 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", + 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) - + log.append(TestUtils.singleMessageSet("hello".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 + * MessageSet size shouldn't exceed the config.segmentSize, check that it is properly enforced by + * appending a message set larger than the config.segmentSize setting and checking that an exception is thrown. + */ + @Test + def testMessageSetSizeCheck() { + val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes)) + // append messages to log + val configSegmentSize = messageSet.sizeInBytes - 1 + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, configSegmentSize: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) + + try { + log.append(messageSet) + fail("message set should throw MessageSetSizeTooLargeException.") + } catch { + case e: MessageSetSizeTooLargeException => // this is good + } + } + + @Test + def testCompactedTopicConstraints() { + val keyedMessage = new Message(bytes = "this message has a key".getBytes, key = "and here it is".getBytes) + val anotherKeyedMessage = new Message(bytes = "this message also has a key".getBytes, key ="another key".getBytes) + val unkeyedMessage = new Message(bytes = "this message does not have a key".getBytes) + + val messageSetWithUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage, keyedMessage) + val messageSetWithOneUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage) + val messageSetWithCompressedKeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage) + val messageSetWithCompressedUnkeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage, unkeyedMessage) + + val messageSetWithKeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage) + val messageSetWithKeyedMessages = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage, anotherKeyedMessage) + + val logProps = new Properties() + logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time) + + try { + log.append(messageSetWithUnkeyedMessage) + fail("Compacted topics cannot accept a message without a key.") + } catch { + case e: InvalidMessageException => // this is good + } + try { + log.append(messageSetWithOneUnkeyedMessage) + fail("Compacted topics cannot accept a message without a key.") + } catch { + case e: InvalidMessageException => // this is good + } + try { + log.append(messageSetWithCompressedUnkeyedMessage) + fail("Compacted topics cannot accept a message without a key.") + } catch { + case e: InvalidMessageException => // this is good + } + + // the following should succeed without any InvalidMessageException + log.append(messageSetWithKeyedMessage) + log.append(messageSetWithKeyedMessages) + log.append(messageSetWithCompressedKeyedMessage) + } + + /** + * 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 @@ -296,7 +408,9 @@ class LogTest extends JUnitSuite { // append messages to log val maxMessageSize = second.sizeInBytes - 1 - val log = new Log(logDir, logConfig.copy(maxMessageSize = maxMessageSize), recoveryPoint = 0L, time.scheduler, time = time) + val logProps = new Properties() + logProps.put(LogConfig.MaxMessageBytesProp, maxMessageSize: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) // should be able to append the small message log.append(first) @@ -305,10 +419,9 @@ 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. */ @@ -318,7 +431,11 @@ class LogTest extends JUnitSuite { val messageSize = 100 val segmentSize = 7 * messageSize val indexInterval = 3 * messageSize - val config = logConfig.copy(segmentSize = segmentSize, indexInterval = indexInterval, maxIndexSize = 4096) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, indexInterval: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) + val config = LogConfig(logProps) var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) for(i <- 0 until numMessages) log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(messageSize))) @@ -327,13 +444,13 @@ class LogTest extends JUnitSuite { val numIndexEntries = log.activeSegment.index.entries val lastOffset = log.logEndOffset log.close() - + 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.activeSegment.index.lastOffset) assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries) log.close() - + // 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) @@ -341,7 +458,7 @@ class LogTest extends JUnitSuite { 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 */ @@ -349,21 +466,58 @@ class LogTest extends JUnitSuite { def testIndexRebuild() { // publish the messages and close the log val numMessages = 200 - val config = logConfig.copy(segmentSize = 200, indexInterval = 1) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 200: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val config = LogConfig(logProps) 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).messageSet.head.offset) + log.close() + } + + /** + * Test that if we have corrupted an index segment it is rebuilt when the log is re-opened + */ + @Test + def testCorruptIndexRebuild() { + // publish the messages and close the log + val numMessages = 200 + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 200: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + + val config = LogConfig(logProps) + 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() + + // corrupt all the index files + for( file <- indexFiles) { + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(" ") + bw.close() + } + // reopen the log - log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) + log = new Log(logDir, config, recoveryPoint = 200L, 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) + assertEquals(i, log.read(i, 100, None).messageSet.head.offset) log.close() } @@ -375,18 +529,21 @@ class LogTest extends JUnitSuite { val set = TestUtils.singleMessageSet("test".getBytes()) val setSize = set.sizeInBytes val msgPerSeg = 10 - val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages + val segmentSize = msgPerSeg * setSize // each segment will be 10 messages + + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) // create a log - val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), recoveryPoint = 0L, scheduler = time.scheduler, time = time) + val log = new Log(logDir, LogConfig(logProps), 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 @@ -404,7 +561,7 @@ 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.truncateFullyAndStartAt(log.logEndOffset - (msgPerSeg - 1)) @@ -429,8 +586,10 @@ class LogTest extends JUnitSuite { val set = TestUtils.singleMessageSet("test".getBytes()) val setSize = set.sizeInBytes val msgPerSeg = 10 - val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages - val config = logConfig.copy(segmentSize = segmentSize) + val segmentSize = msgPerSeg * setSize // each segment will be 10 messages + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) + val config = LogConfig(logProps) 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) @@ -455,23 +614,25 @@ 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, - logConfig.copy(segmentSize = set.sizeInBytes * 5, - maxIndexSize = 1000, - indexInterval = 1), + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + val log = new Log(logDir, + LogConfig(logProps), 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() } @@ -481,22 +642,24 @@ class LogTest extends JUnitSuite { @Test def testReopenThenTruncate() { val set = TestUtils.singleMessageSet("test".getBytes()) - val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, - maxIndexSize = 1000, - indexInterval = 10000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 10000: java.lang.Integer) + val config = LogConfig(logProps) // create a log - var log = new Log(logDir, + 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, + log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, @@ -505,7 +668,7 @@ class LogTest extends JUnitSuite { assertEquals("All but one segment should be deleted.", 1, log.numberOfSegments) assertEquals("Log end offset should be 3.", 3, log.logEndOffset) } - + /** * Test that deleted files are deleted after the appropriate time. */ @@ -513,83 +676,93 @@ class LogTest extends JUnitSuite { 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 logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 10000: java.lang.Integer) + logProps.put(LogConfig.FileDeleteDelayMsProp, asyncDeleteMs: java.lang.Integer) + val config = LogConfig(logProps) + val log = new Log(logDir, config, - recoveryPoint = 0L, + 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)) && + 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) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) + val config = LogConfig(logProps) 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, + + 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, + val 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) + val messageSet = log.read(0, 4096, None).messageSet + assertEquals(0, messageSet.head.offset) + assertTrue("Message payload should be null.", messageSet.head.message.isNull) } - + @Test - def testCorruptLog() { + def testCorruptLog() { // append some messages to create some segments - val config = logConfig.copy(indexInterval = 1, maxMessageSize = 64*1024, segmentSize = 1000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer) + val config = LogConfig(logProps) val set = TestUtils.singleMessageSet("test".getBytes()) val recoveryPoint = 50L for(iteration <- 0 until 50) { @@ -605,23 +778,27 @@ class LogTest extends JUnitSuite { 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) + CoreUtils.rm(logDir) } } @Test def testCleanShutdownFile() { // append some messages to create some segments - val config = logConfig.copy(indexInterval = 1, maxMessageSize = 64*1024, segmentSize = 1000) + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 1000: java.lang.Integer) + logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + val config = LogConfig(logProps) val set = TestUtils.singleMessageSet("test".getBytes()) val parentLogDir = logDir.getParentFile assertTrue("Data directory %s must exist", parentLogDir.isDirectory) @@ -646,4 +823,79 @@ class LogTest extends JUnitSuite { assertEquals(recoveryPoint, log.logEndOffset) cleanShutdownFile.delete() } + + @Test + def testParseTopicPartitionName() { + val topic: String = "test_topic" + val partition:String = "143" + val dir: File = new File(logDir + topicPartitionName(topic, partition)) + val topicAndPartition = Log.parseTopicPartitionName(dir) + assertEquals(topic, topicAndPartition.asTuple._1) + assertEquals(partition.toInt, topicAndPartition.asTuple._2) + } + + @Test + def testParseTopicPartitionNameForEmptyName() { + try { + val dir: File = new File("") + val topicAndPartition = Log.parseTopicPartitionName(dir) + fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + } catch { + case e: Exception => // its GOOD! + } + } + + @Test + def testParseTopicPartitionNameForNull() { + try { + val dir: File = null + val topicAndPartition = Log.parseTopicPartitionName(dir) + fail("KafkaException should have been thrown for dir: " + dir) + } catch { + case e: Exception => // its GOOD! + } + } + + @Test + def testParseTopicPartitionNameForMissingSeparator() { + val topic: String = "test_topic" + val partition:String = "1999" + val dir: File = new File(logDir + File.separator + topic + partition) + try { + val topicAndPartition = Log.parseTopicPartitionName(dir) + fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + } catch { + case e: Exception => // its GOOD! + } + } + + @Test + def testParseTopicPartitionNameForMissingTopic() { + val topic: String = "" + val partition:String = "1999" + val dir: File = new File(logDir + topicPartitionName(topic, partition)) + try { + val topicAndPartition = Log.parseTopicPartitionName(dir) + fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + } catch { + case e: Exception => // its GOOD! + } + } + + @Test + def testParseTopicPartitionNameForMissingPartition() { + val topic: String = "test_topic" + val partition:String = "" + val dir: File = new File(logDir + topicPartitionName(topic, partition)) + try { + val topicAndPartition = Log.parseTopicPartitionName(dir) + fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + } catch { + case e: Exception => // its GOOD! + } + } + + def topicPartitionName(topic: String, partition: String): String = { + File.separator + topic + "-" + partition + } } diff --git a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala deleted file mode 100644 index 67497dd042dfd..0000000000000 --- a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala +++ /dev/null @@ -1,177 +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.log4j - -import java.util.Properties -import java.io.File -import kafka.consumer.SimpleConsumer -import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.{TestUtils, Utils, Logging} -import junit.framework.Assert._ -import kafka.api.FetchRequestBuilder -import kafka.producer.async.MissingConfigException -import kafka.serializer.Encoder -import kafka.zk.ZooKeeperTestHarness -import org.apache.log4j.spi.LoggingEvent -import org.apache.log4j.{PropertyConfigurator, Logger} -import org.junit.{After, Before, Test} -import org.scalatest.junit.JUnit3Suite - -class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { - - var logDirZk: File = null - var config: KafkaConfig = null - var server: KafkaServer = null - - var simpleConsumerZk: SimpleConsumer = null - - val tLogger = Logger.getLogger(getClass()) - - private val brokerZk = 0 - - private val ports = TestUtils.choosePorts(2) - private val portZk = ports(0) - - @Before - override def setUp() { - super.setUp() - - val propsZk = TestUtils.createBrokerConfig(brokerZk, portZk) - val logDirZkPath = propsZk.getProperty("log.dir") - logDirZk = new File(logDirZkPath) - config = new KafkaConfig(propsZk) - server = TestUtils.createServer(config); - simpleConsumerZk = new SimpleConsumer("localhost", portZk, 1000000, 64*1024, "") - } - - @After - override def tearDown() { - simpleConsumerZk.close - server.shutdown - Utils.rm(logDirZk) - super.tearDown() - } - - @Test - def testKafkaLog4jConfigs() { - var props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.appender.KAFKA.SerializerClass", "kafka.log4j.AppenderStringEncoder") - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - - // port missing - try { - PropertyConfigurator.configure(props) - fail("Missing properties exception was expected !") - }catch { - case e: MissingConfigException => - } - - props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.appender.KAFKA.SerializerClass", "kafka.log4j.AppenderStringEncoder") - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - - // host missing - try { - PropertyConfigurator.configure(props) - fail("Missing properties exception was expected !") - }catch { - case e: MissingConfigException => - } - - props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.SerializerClass", "kafka.log4j.AppenderStringEncoder") - props.put("log4j.appender.KAFKA.brokerList", TestUtils.getBrokerListStrFromConfigs(Seq(config))) - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - - // topic missing - try { - PropertyConfigurator.configure(props) - fail("Missing properties exception was expected !") - }catch { - case e: MissingConfigException => - } - - props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.brokerList", TestUtils.getBrokerListStrFromConfigs(Seq(config))) - props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") - - // serializer missing - try { - PropertyConfigurator.configure(props) - }catch { - case e: MissingConfigException => fail("should default to kafka.serializer.StringEncoder") - } - } - - @Test - def testLog4jAppends() { - PropertyConfigurator.configure(getLog4jConfig) - - for(i <- 1 to 5) - info("test") - - val response = simpleConsumerZk.fetch(new FetchRequestBuilder().addFetch("test-topic", 0, 0L, 1024*1024).build()) - val fetchMessage = response.messageSet("test-topic", 0) - - var count = 0 - for(message <- fetchMessage) { - count = count + 1 - } - - assertEquals(5, count) - } - - private def getLog4jConfig: Properties = { - var props = new Properties() - props.put("log4j.rootLogger", "INFO") - props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") - props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") - props.put("log4j.appender.KAFKA.brokerList", TestUtils.getBrokerListStrFromConfigs(Seq(config))) - props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.logger.kafka.log4j", "INFO,KAFKA") - props.put("log4j.appender.KAFKA.requiredNumAcks", "1") - props - } -} - -class AppenderStringEncoder(encoding: String = "UTF-8") extends Encoder[LoggingEvent] { - def toBytes(event: LoggingEvent): Array[Byte] = { - event.getMessage.toString.getBytes(encoding) - } -} - diff --git a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala index 6db245c956d21..dd8847f5f709a 100644 --- a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala +++ b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala @@ -31,7 +31,7 @@ trait BaseMessageSetTestCases extends JUnitSuite { def createMessageSet(messages: Seq[Message]): MessageSet @Test - def testWrittenEqualsRead { + def testWrittenEqualsRead() { val messageSet = createMessageSet(messages) checkEquals(messages.iterator, messageSet.map(m => m.message).iterator) } diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 4e45d965bc423..07bc317bcd40c 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -147,11 +147,11 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { // check uncompressed offsets checkOffsets(messages, 0) var offset = 1234567 - checkOffsets(messages.assignOffsets(new AtomicLong(offset), NoCompressionCodec), offset) - + checkOffsets(messages.validateMessagesAndAssignOffsets(new AtomicLong(offset), NoCompressionCodec, NoCompressionCodec), offset) + // check compressed messages checkOffsets(compressedMessages, 0) - checkOffsets(compressedMessages.assignOffsets(new AtomicLong(offset), DefaultCompressionCodec), offset) + checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(new AtomicLong(offset), DefaultCompressionCodec, DefaultCompressionCodec), offset) } /* check that offsets are assigned based on byte offset from the given base offset */ diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala index ed22931f24fed..76987d4fa68fd 100644 --- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala @@ -30,6 +30,8 @@ class MessageCompressionTest extends JUnitSuite { val codecs = mutable.ArrayBuffer[CompressionCodec](GZIPCompressionCodec) if(isSnappyAvailable) codecs += SnappyCompressionCodec + if(isLZ4Available) + codecs += LZ4CompressionCodec for(codec <- codecs) testSimpleCompressDecompress(codec) } @@ -42,15 +44,6 @@ class MessageCompressionTest extends JUnitSuite { assertEquals(messages, decompressed) } - @Test - def testComplexCompressDecompress() { - val messages = List(new Message("hi there".getBytes), new Message("I am fine".getBytes), new Message("I am not so well today".getBytes)) - val message = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, messages = messages.slice(0, 2):_*) - val complexMessages = List(message.shallowIterator.next.message):::messages.slice(2,3) - val complexMessage = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, messages = complexMessages:_*) - val decompressedMessages = complexMessage.iterator.map(_.message).toList - assertEquals(messages, decompressedMessages) - } def isSnappyAvailable(): Boolean = { try { @@ -61,4 +54,13 @@ class MessageCompressionTest extends JUnitSuite { case e: org.xerial.snappy.SnappyError => false } } + + def isLZ4Available(): Boolean = { + try { + val lz4 = new net.jpountz.lz4.LZ4BlockOutputStream(new ByteArrayOutputStream()) + true + } catch { + case e: UnsatisfiedLinkError => false + } + } } diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala old mode 100644 new mode 100755 index 4837585d03535..11c0f817ca06e --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -24,7 +24,8 @@ import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{Before, Test} import kafka.utils.TestUtils -import kafka.utils.Utils +import kafka.utils.CoreUtils +import org.apache.kafka.common.utils.Utils case class MessageTestVal(val key: Array[Byte], val payload: Array[Byte], @@ -39,7 +40,7 @@ class MessageTest extends JUnitSuite { def setUp(): Unit = { val keys = Array(null, "key".getBytes, "".getBytes) val vals = Array("value".getBytes, "".getBytes, null) - val codecs = Array(NoCompressionCodec, GZIPCompressionCodec) + val codecs = Array(NoCompressionCodec, GZIPCompressionCodec, SnappyCompressionCodec, LZ4CompressionCodec) for(k <- keys; v <- vals; codec <- codecs) messages += new MessageTestVal(k, v, codec, new Message(v, k, codec)) } diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala new file mode 100644 index 0000000000000..b08a3432ad1ae --- /dev/null +++ b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala @@ -0,0 +1,130 @@ +/** + * 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.message + +import java.io.{InputStream, ByteArrayInputStream, ByteArrayOutputStream} +import java.nio.ByteBuffer +import java.util.Random +import junit.framework.Assert._ +import org.junit.Test +import org.scalatest.junit.JUnitSuite + +class MessageWriterTest extends JUnitSuite { + + private val rnd = new Random() + + private def mkRandomArray(size: Int): Array[Byte] = { + (0 until size).map(_ => rnd.nextInt(10).toByte).toArray + } + + private def mkMessageWithWriter(key: Array[Byte] = null, bytes: Array[Byte], codec: CompressionCodec): Message = { + val writer = new MessageWriter(100) + writer.write(key = key, codec = codec) { output => + val out = if (codec == NoCompressionCodec) output else CompressionFactory(codec, output) + try { + val p = rnd.nextInt(bytes.length) + out.write(bytes, 0, p) + out.write(bytes, p, bytes.length - p) + } finally { + out.close() + } + } + val bb = ByteBuffer.allocate(writer.size) + writer.writeTo(bb) + bb.rewind() + new Message(bb) + } + + private def compress(bytes: Array[Byte], codec: CompressionCodec): Array[Byte] = { + val baos = new ByteArrayOutputStream() + val out = CompressionFactory(codec, baos) + out.write(bytes) + out.close() + baos.toByteArray + } + + private def decompress(compressed: Array[Byte], codec: CompressionCodec): Array[Byte] = { + toArray(CompressionFactory(codec, new ByteArrayInputStream(compressed))) + } + + private def toArray(in: InputStream): Array[Byte] = { + val out = new ByteArrayOutputStream() + val buf = new Array[Byte](100) + var amount = in.read(buf) + while (amount >= 0) { + out.write(buf, 0, amount) + amount = in.read(buf) + } + out.toByteArray + } + + private def toArray(bb: ByteBuffer): Array[Byte] = { + val arr = new Array[Byte](bb.limit()) + bb.get(arr) + bb.rewind() + arr + } + + @Test + def testBufferingOutputStream(): Unit = { + val out = new BufferingOutputStream(50) + out.write(0) + out.write(1) + out.write(2) + val r = out.reserve(100) + out.write((103 until 200).map(_.toByte).toArray) + r.write((3 until 103).map(_.toByte).toArray) + + val buf = ByteBuffer.allocate(out.size) + out.writeTo(buf) + buf.rewind() + + assertEquals((0 until 200).map(_.toByte), buf.array.toSeq) + } + + @Test + def testWithNoCompressionAttribute(): Unit = { + val bytes = mkRandomArray(4096) + val actual = mkMessageWithWriter(bytes = bytes, codec = NoCompressionCodec) + val expected = new Message(bytes, NoCompressionCodec) + assertEquals(expected.buffer, actual.buffer) + } + + @Test + def testWithCompressionAttribute(): Unit = { + val bytes = mkRandomArray(4096) + val actual = mkMessageWithWriter(bytes = bytes, codec = SnappyCompressionCodec) + val expected = new Message(compress(bytes, SnappyCompressionCodec), SnappyCompressionCodec) + + assertEquals( + decompress(toArray(expected.payload), SnappyCompressionCodec).toSeq, + decompress(toArray(actual.payload), SnappyCompressionCodec).toSeq + ) + } + + @Test + def testWithKey(): Unit = { + val key = mkRandomArray(123) + val bytes = mkRandomArray(4096) + val actual = mkMessageWithWriter(bytes = bytes, key = key, codec = NoCompressionCodec) + val expected = new Message(bytes = bytes, key = key, codec = NoCompressionCodec) + + assertEquals(expected.buffer, actual.buffer) + } + +} diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala new file mode 100644 index 0000000000000..b42101b85fa4a --- /dev/null +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -0,0 +1,99 @@ +/** + * 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.consumer + +import java.util.Properties + +import com.yammer.metrics.Metrics +import com.yammer.metrics.core.MetricPredicate +import org.junit.Test +import junit.framework.Assert._ +import kafka.integration.KafkaServerTestHarness +import kafka.server._ +import kafka.message._ +import kafka.serializer._ +import kafka.utils._ +import kafka.admin.AdminUtils +import kafka.utils.TestUtils._ +import scala.collection._ +import scala.collection.JavaConversions._ +import scala.util.matching.Regex +import org.scalatest.junit.JUnit3Suite + +class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { + val numNodes = 2 + val numParts = 2 + val topic = "topic1" + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) + + def generateConfigs() = + TestUtils.createBrokerConfigs(numNodes, zkConnect, enableDeleteTopic=true).map(KafkaConfig.fromProps(_, overridingProps)) + + val nMessages = 2 + + override def tearDown() { + super.tearDown() + } + + @Test + def testMetricsLeak() { + // create topic topic1 with 1 partition on broker 0 + createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers) + // force creation not client's specific metrics. + createAndShutdownStep("group0", "consumer0", "producer0") + + val countOfStaticMetrics = Metrics.defaultRegistry().allMetrics().keySet().size + + for (i <- 0 to 5) { + createAndShutdownStep("group" + i % 3, "consumer" + i % 2, "producer" + i % 2) + assertEquals(countOfStaticMetrics, Metrics.defaultRegistry().allMetrics().keySet().size) + } + } + + @Test + def testMetricsReporterAfterDeletingTopic() { + val topic = "test-topic-metric" + AdminUtils.createTopic(zkClient, topic, 1, 1) + AdminUtils.deleteTopic(zkClient, topic) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) + assertFalse("Topic metrics exists after deleteTopic", checkTopicMetricsExists(topic)) + } + + def createAndShutdownStep(group: String, consumerId: String, producerId: String): Unit = { + val sentMessages1 = sendMessages(servers, topic, nMessages) + // create a consumer + val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumerId)) + val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) + val receivedMessages1 = getMessages(topicMessageStreams1, nMessages) + + zkConsumerConnector1.shutdown() + } + + private def checkTopicMetricsExists(topic: String): Boolean = { + val topicMetricRegex = new Regex(".*("+topic+")$") + val metricGroups = Metrics.defaultRegistry().groupedMetrics(MetricPredicate.ALL).entrySet() + for(metricGroup <- metricGroups) { + if (topicMetricRegex.pattern.matcher(metricGroup.getKey()).matches) + return true + } + false + } +} diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 4ff6f55914909..7dc2fad542ea5 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.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 @@ -19,6 +19,11 @@ package kafka.network; import java.net._ import java.io._ +import kafka.cluster.EndPoint +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.network.NetworkSend +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.utils.SystemTime import org.junit._ import org.scalatest.junit.JUnitSuite import java.util.Random @@ -30,18 +35,23 @@ import kafka.common.TopicAndPartition import kafka.message.ByteBufferMessageSet import java.nio.channels.SelectionKey import kafka.utils.TestUtils - +import scala.collection.Map class SocketServerTest extends JUnitSuite { val server: SocketServer = new SocketServer(0, - host = null, - port = kafka.utils.TestUtils.choosePort, + Map(SecurityProtocol.PLAINTEXT -> EndPoint(null, 0, SecurityProtocol.PLAINTEXT), + SecurityProtocol.TRACE -> EndPoint(null, 0, SecurityProtocol.TRACE)), numProcessorThreads = 1, maxQueuedRequests = 50, sendBufferSize = 300000, recvBufferSize = 300000, - maxRequestSize = 50) + maxRequestSize = 50, + maxConnectionsPerIp = 5, + connectionsMaxIdleMs = 60*1000, + maxConnectionsPerIpOverrides = Map.empty[String,Int], + new SystemTime(), + new Metrics()) server.startup() def sendRequest(socket: Socket, id: Short, request: Array[Byte]) { @@ -52,7 +62,7 @@ class SocketServerTest extends JUnitSuite { outgoing.flush() } - def receiveResponse(socket: Socket): Array[Byte] = { + def receiveResponse(socket: Socket): Array[Byte] = { val incoming = new DataInputStream(socket.getInputStream) val len = incoming.readInt() val response = new Array[Byte](len) @@ -66,20 +76,23 @@ class SocketServerTest extends JUnitSuite { val byteBuffer = ByteBuffer.allocate(request.requestObj.sizeInBytes) request.requestObj.writeTo(byteBuffer) byteBuffer.rewind() - val send = new BoundedByteBufferSend(byteBuffer) + val send = new NetworkSend(request.connectionId, byteBuffer) channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) } - def connect() = new Socket("localhost", server.port) + def connect(s:SocketServer = server, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { + new Socket("localhost", server.boundPort(protocol)) + } + @After def cleanup() { server.shutdown() } - @Test def simpleRequest() { - val socket = connect() + val plainSocket = connect(protocol = SecurityProtocol.PLAINTEXT) + val traceSocket = connect(protocol = SecurityProtocol.TRACE) val correlationId = -1 val clientId = SyncProducerConfig.DefaultClientId val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs @@ -93,49 +106,97 @@ class SocketServerTest extends JUnitSuite { val serializedBytes = new Array[Byte](byteBuffer.remaining) byteBuffer.get(serializedBytes) - sendRequest(socket, 0, serializedBytes) + // Test PLAINTEXT socket + sendRequest(plainSocket, 0, serializedBytes) + processRequest(server.requestChannel) + assertEquals(serializedBytes.toSeq, receiveResponse(plainSocket).toSeq) + + // Test TRACE socket + sendRequest(traceSocket, 0, serializedBytes) processRequest(server.requestChannel) - assertEquals(serializedBytes.toSeq, receiveResponse(socket).toSeq) + assertEquals(serializedBytes.toSeq, receiveResponse(traceSocket).toSeq) } - @Test(expected=classOf[IOException]) + @Test def tooBigRequestIsRejected() { val tooManyBytes = new Array[Byte](server.maxRequestSize + 1) new Random().nextBytes(tooManyBytes) val socket = connect() sendRequest(socket, 0, tooManyBytes) - receiveResponse(socket) + try { + receiveResponse(socket) + } catch { + case e: IOException => // thats fine + } } @Test - def testSocketSelectionKeyState() { - val socket = connect() - val correlationId = -1 - val clientId = SyncProducerConfig.DefaultClientId - val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs - val ack: Short = 0 - val emptyRequest = - new ProducerRequest(correlationId, clientId, ack, ackTimeoutMs, collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]()) - - val byteBuffer = ByteBuffer.allocate(emptyRequest.sizeInBytes) - emptyRequest.writeTo(byteBuffer) - byteBuffer.rewind() - val serializedBytes = new Array[Byte](byteBuffer.remaining) - byteBuffer.get(serializedBytes) + def testSocketsCloseOnShutdown() { + // open a connection + val plainSocket = connect(protocol = SecurityProtocol.PLAINTEXT) + val traceSocket = connect(protocol = SecurityProtocol.TRACE) + val bytes = new Array[Byte](40) + // send a request first to make sure the connection has been picked up by the socket server + sendRequest(plainSocket, 0, bytes) + sendRequest(traceSocket, 0, bytes) + processRequest(server.requestChannel) - sendRequest(socket, 0, serializedBytes) + // make sure the sockets are open + server.acceptors.values.map(acceptor => assertFalse(acceptor.serverChannel.socket.isClosed)) + // then shutdown the server + server.shutdown() - 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) + val largeChunkOfBytes = new Array[Byte](1000000) + // doing a subsequent send should throw an exception as the connection should be closed. + // send a large chunk of bytes to trigger a socket flush + try { + sendRequest(plainSocket, 0, largeChunkOfBytes) + fail("expected exception when writing to closed plain socket") + } catch { + case e: IOException => // expected + } + + try { + sendRequest(traceSocket, 0, largeChunkOfBytes) + fail("expected exception when writing to closed trace socket") + } catch { + case e: IOException => // expected + } + } - server.requestChannel.sendResponse(new RequestChannel.Response(0, request, null)) + @Test + def testMaxConnectionsPerIp() { + // make the maximum allowable number of connections and then leak them + val conns = (0 until server.maxConnectionsPerIp).map(i => connect()) + // now try one more (should fail) + val conn = connect() + conn.setSoTimeout(3000) + assertEquals(-1, conn.getInputStream().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) - ) + @Test + def testMaxConnectionsPerIPOverrides(): Unit = { + val overrideNum = 6 + val overrides: Map[String, Int] = Map("localhost" -> overrideNum) + val overrideServer: SocketServer = new SocketServer(0, + Map(SecurityProtocol.PLAINTEXT -> EndPoint(null, 0, SecurityProtocol.PLAINTEXT)), + numProcessorThreads = 1, + maxQueuedRequests = 50, + sendBufferSize = 300000, + recvBufferSize = 300000, + maxRequestSize = 50, + maxConnectionsPerIp = 5, + connectionsMaxIdleMs = 60*1000, + maxConnectionsPerIpOverrides = overrides, + new SystemTime(), + new Metrics()) + overrideServer.startup() + // make the maximum allowable number of connections and then leak them + val conns = ((0 until overrideNum).map(i => connect(overrideServer))) + // now try one more (should fail) + val conn = connect(overrideServer) + conn.setSoTimeout(3000) + assertEquals(-1, conn.getInputStream.read()) + overrideServer.shutdown() } } diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala old mode 100644 new mode 100755 index 18e355501808c..be4bb878dc49f --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -23,7 +23,7 @@ import junit.framework.Assert._ import org.easymock.EasyMock import org.junit.Test import kafka.api._ -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndPoint, Broker} import kafka.common._ import kafka.message._ import kafka.producer.async._ @@ -36,8 +36,10 @@ import scala.collection.mutable.ArrayBuffer import kafka.utils._ class AsyncProducerTest extends JUnit3Suite { - val props = createBrokerConfigs(1) - val configs = props.map(p => new KafkaConfig(p)) + // One of the few cases we can just set a fixed port because the producer is mocked out here since this uses mocks + val props = Seq(createBrokerConfig(1, "127.0.0.1:1", port=65534)) + val configs = props.map(KafkaConfig.fromProps) + val brokerList = configs.map(c => org.apache.kafka.common.utils.Utils.formatAddress(c.hostName, c.port)).mkString(",") override def setUp() { super.setUp() @@ -61,7 +63,7 @@ class AsyncProducerTest extends JUnit3Suite { val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", brokerList) props.put("producer.type", "async") props.put("queue.buffering.max.messages", "10") props.put("batch.num.messages", "1") @@ -84,15 +86,11 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testProduceAfterClosed() { - val props = new Properties() - props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) - props.put("producer.type", "async") - props.put("batch.num.messages", "1") - - val config = new ProducerConfig(props) val produceData = getProduceData(10) - val producer = new Producer[String, String](config) + val producer = createProducer[String, String]( + brokerList, + encoder = classOf[StringEncoder].getName) + producer.close try { @@ -166,10 +164,10 @@ class AsyncProducerTest extends JUnit3Suite { 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) - broker1 + props.put("metadata.broker.list", brokerList) + val broker1 = new BrokerEndPoint(0, "localhost", 9092) + val broker2 = new BrokerEndPoint(1, "localhost", 9093) + // form expected partitions metadata val partition1Metadata = new PartitionMetadata(0, Some(broker1), List(broker1, broker2)) val partition2Metadata = new PartitionMetadata(1, Some(broker2), List(broker1, broker2)) @@ -216,7 +214,7 @@ class AsyncProducerTest extends JUnit3Suite { def testSerializeEvents() { val produceData = TestUtils.getMsgStrings(5).map(m => new KeyedMessage[String,String]("topic1",m)) val props = new Properties() - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", brokerList) val config = new ProducerConfig(props) // form expected partitions metadata val topic1Metadata = getTopicMetadata("topic1", 0, 0, "localhost", 9092) @@ -233,8 +231,14 @@ class AsyncProducerTest extends JUnit3Suite { topicPartitionInfos = topicPartitionInfos) val serializedData = handler.serialize(produceData) - val deserializedData = serializedData.map(d => new KeyedMessage[String,String](d.topic, Utils.readString(d.message.payload))) + val deserializedData = serializedData.map(d => new KeyedMessage[String,String](d.topic, TestUtils.readString(d.message.payload))) + + // Test that the serialize handles seq from a Stream + val streamedSerializedData = handler.serialize(Stream(produceData:_*)) + val deserializedStreamData = streamedSerializedData.map(d => new KeyedMessage[String,String](d.topic, TestUtils.readString(d.message.payload))) + TestUtils.checkEquals(produceData.iterator, deserializedData.iterator) + TestUtils.checkEquals(produceData.iterator, deserializedStreamData.iterator) } @Test @@ -242,7 +246,7 @@ class AsyncProducerTest extends JUnit3Suite { val producerDataList = new ArrayBuffer[KeyedMessage[String,Message]] producerDataList.append(new KeyedMessage[String,Message]("topic1", "key1", new Message("msg1".getBytes))) val props = new Properties() - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", brokerList) val config = new ProducerConfig(props) // form expected partitions metadata @@ -272,7 +276,7 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testNoBroker() { val props = new Properties() - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", brokerList) val config = new ProducerConfig(props) // create topic metadata with 0 partitions @@ -303,10 +307,14 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testIncompatibleEncoder() { val props = new Properties() - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) - val config = new ProducerConfig(props) + // no need to retry since the send will always fail + props.put("message.send.max.retries", "0") + val producer= createProducer[String, String]( + brokerList = brokerList, + encoder = classOf[DefaultEncoder].getName, + keyEncoder = classOf[DefaultEncoder].getName, + producerProps = props) - val producer=new Producer[String, String](config) try { producer.send(getProduceData(1): _*) fail("Should fail with ClassCastException due to incompatible Encoder") @@ -320,7 +328,7 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testRandomPartitioner() { val props = new Properties() - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", brokerList) val config = new ProducerConfig(props) // create topic metadata with 0 partitions @@ -355,46 +363,10 @@ class AsyncProducerTest extends JUnit3Suite { } } - @Test - def testBrokerListAndAsync() { - return - val props = TestUtils.getProducerConfig(TestUtils.getBrokerListStrFromConfigs(configs)) - props.put("producer.type", "async") - props.put("batch.num.messages", "5") - - val config = new ProducerConfig(props) - - val topic = "topic1" - val topic1Metadata = getTopicMetadata(topic, 0, 0, "localhost", 9092) - val topicPartitionInfos = new collection.mutable.HashMap[String, TopicMetadata] - topicPartitionInfos.put("topic1", topic1Metadata) - - val producerPool = new ProducerPool(config) - - val msgs = TestUtils.getMsgStrings(10) - - val handler = new DefaultEventHandler[String,String](config, - partitioner = null.asInstanceOf[Partitioner], - encoder = new StringEncoder, - keyEncoder = new StringEncoder, - producerPool = producerPool, - topicPartitionInfos = topicPartitionInfos) - - val producer = new Producer[String, String](config, handler) - try { - // send all 10 messages, should create 2 batches and 2 syncproducer calls - producer.send(msgs.map(m => new KeyedMessage[String,String](topic, m)): _*) - producer.close - - } catch { - case e: Exception => fail("Not expected", e) - } - } - @Test def testFailedSendRetryLogic() { val props = new Properties() - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) + props.put("metadata.broker.list", brokerList) props.put("request.required.acks", "1") props.put("serializer.class", classOf[StringEncoder].getName.toString) props.put("key.serializer.class", classOf[NullEncoder[Int]].getName.toString) @@ -497,7 +469,7 @@ class AsyncProducerTest extends JUnit3Suite { } private def getTopicMetadata(topic: String, partition: Seq[Int], brokerId: Int, brokerHost: String, brokerPort: Int): TopicMetadata = { - val broker1 = new Broker(brokerId, brokerHost, brokerPort) + val broker1 = new BrokerEndPoint(brokerId, brokerHost, brokerPort) new TopicMetadata(topic, partition.map(new PartitionMetadata(_, Some(broker1), List(broker1)))) } diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala old mode 100644 new mode 100755 index 4b2e4ade70f6f..4d2536b462c03 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -34,13 +34,11 @@ import org.junit.Assert.assertTrue import org.junit.Assert.assertFalse import org.junit.Assert.assertEquals import kafka.common.{ErrorMapping, FailedToSendMessageException} - +import kafka.serializer.StringEncoder class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ private val brokerId1 = 0 private val brokerId2 = 1 - private val ports = TestUtils.choosePorts(2) - private val (port1, port2) = (ports(0), ports(1)) private var server1: KafkaServer = null private var server2: KafkaServer = null private var consumer1: SimpleConsumer = null @@ -48,26 +46,36 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) private var servers = List.empty[KafkaServer] - private val props1 = TestUtils.createBrokerConfig(brokerId1, port1) - props1.put("num.partitions", "4") - private val config1 = new KafkaConfig(props1) - private val props2 = TestUtils.createBrokerConfig(brokerId2, port2) - props2.put("num.partitions", "4") - private val config2 = new KafkaConfig(props2) + // Creation of consumers is deferred until they are actually needed. This allows us to kill brokers that use random + // ports and then get a consumer instance that will be pointed at the correct port + def getConsumer1() = { + if (consumer1 == null) + consumer1 = new SimpleConsumer("localhost", server1.boundPort(), 1000000, 64*1024, "") + consumer1 + } + + def getConsumer2() = { + if (consumer2 == null) + consumer2 = new SimpleConsumer("localhost", server2.boundPort(), 100, 64*1024, "") + consumer2 + } override def setUp() { super.setUp() // set up 2 brokers with 4 partitions each + val props1 = TestUtils.createBrokerConfig(brokerId1, zkConnect, false) + props1.put("num.partitions", "4") + val config1 = KafkaConfig.fromProps(props1) + val props2 = TestUtils.createBrokerConfig(brokerId2, zkConnect, false) + props2.put("num.partitions", "4") + val config2 = KafkaConfig.fromProps(props2) server1 = TestUtils.createServer(config1) server2 = TestUtils.createServer(config2) servers = List(server1,server2) val props = new Properties() props.put("host", "localhost") - props.put("port", port1.toString) - - consumer1 = new SimpleConsumer("localhost", port1, 1000000, 64*1024, "") - consumer2 = new SimpleConsumer("localhost", port2, 100, 64*1024, "") + props.put("port", server1.boundPort().toString) // temporarily set request handler logger to a higher level requestHandlerLogger.setLevel(Level.FATAL) @@ -76,41 +84,48 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ override def tearDown() { // restore set request handler logger to a higher level requestHandlerLogger.setLevel(Level.ERROR) + + if (consumer1 != null) + consumer1.close() + if (consumer2 != null) + consumer2.close() + server1.shutdown server2.shutdown - Utils.rm(server1.config.logDirs) - Utils.rm(server2.config.logDirs) + CoreUtils.rm(server1.config.logDirs) + CoreUtils.rm(server2.config.logDirs) super.tearDown() } @Test def testUpdateBrokerPartitionInfo() { val topic = "new-topic" - AdminUtils.createTopic(zkClient, topic, 1, 2) - // wait until the update metadata request for new topic reaches all servers - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 500) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 2, servers = servers) + + val props = new Properties() + // no need to retry since the send will always fail + props.put("message.send.max.retries", "0") + val producer1 = TestUtils.createProducer[String, String]( + brokerList = "localhost:80,localhost:81", + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + producerProps = props) - val props1 = new util.Properties() - props1.put("metadata.broker.list", "localhost:80,localhost:81") - props1.put("serializer.class", "kafka.serializer.StringEncoder") - val producerConfig1 = new ProducerConfig(props1) - val producer1 = new Producer[String, String](producerConfig1) try{ producer1.send(new KeyedMessage[String, String](topic, "test", "test1")) fail("Test should fail because the broker list provided are not valid") } catch { - case e: FailedToSendMessageException => + case e: FailedToSendMessageException => // this is expected case oe: Throwable => fail("fails with exception", oe) } finally { producer1.close() } - val props2 = new util.Properties() - props2.put("metadata.broker.list", "localhost:80," + TestUtils.getBrokerListStrFromConfigs(Seq( config1))) - props2.put("serializer.class", "kafka.serializer.StringEncoder") - val producerConfig2= new ProducerConfig(props2) - val producer2 = new Producer[String, String](producerConfig2) + val producer2 = TestUtils.createProducer[String, String]( + brokerList = "localhost:80," + TestUtils.getBrokerListStrFromServers(Seq(server1)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName) + try{ producer2.send(new KeyedMessage[String, String](topic, "test", "test1")) } catch { @@ -119,11 +134,11 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ producer2.close() } - val props3 = new util.Properties() - props3.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) - props3.put("serializer.class", "kafka.serializer.StringEncoder") - val producerConfig3 = new ProducerConfig(props3) - val producer3 = new Producer[String, String](producerConfig3) + val producer3 = TestUtils.createProducer[String, String]( + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName) + try{ producer3.send(new KeyedMessage[String, String](topic, "test", "test1")) } catch { @@ -136,28 +151,19 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ @Test def testSendToNewTopic() { val props1 = new util.Properties() - props1.put("serializer.class", "kafka.serializer.StringEncoder") - props1.put("partitioner.class", "kafka.utils.StaticPartitioner") - props1.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) - props1.put("request.required.acks", "2") - props1.put("request.timeout.ms", "1000") - - val props2 = new util.Properties() - props2.putAll(props1) - props2.put("request.required.acks", "3") - props2.put("request.timeout.ms", "1000") - - val producerConfig1 = new ProducerConfig(props1) - val producerConfig2 = new ProducerConfig(props2) + props1.put("request.required.acks", "-1") val topic = "new-topic" // create topic with 1 partition and await leadership - AdminUtils.createTopic(zkClient, topic, 1, 2) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 2, servers = servers) + + val producer1 = TestUtils.createProducer[String, String]( + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName, + producerProps = props1) - val producer1 = new Producer[String, String](producerConfig1) - val producer2 = new Producer[String, String](producerConfig2) // Available partition ids should be 0. producer1.send(new KeyedMessage[String, String](topic, "test", "test1")) producer1.send(new KeyedMessage[String, String](topic, "test", "test2")) @@ -167,10 +173,10 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val leader = leaderOpt.get val messageSet = if(leader == server1.config.brokerId) { - val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) + val response1 = getConsumer1().fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) response1.messageSet("new-topic", 0).iterator.toBuffer }else { - val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) + val response2 = getConsumer2().fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) response2.messageSet("new-topic", 0).iterator.toBuffer } assertEquals("Should have fetched 2 messages", 2, messageSet.size) @@ -178,16 +184,25 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ assertEquals(new Message(bytes = "test2".getBytes, key = "test".getBytes), messageSet(1).message) producer1.close() + val props2 = new util.Properties() + props2.put("request.required.acks", "3") + // no need to retry since the send will always fail + props2.put("message.send.max.retries", "0") + try { - producer2.send(new KeyedMessage[String, String](topic, "test", "test2")) - fail("Should have timed out for 3 acks.") + val producer2 = TestUtils.createProducer[String, String]( + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName, + producerProps = props2) + producer2.close + fail("we don't support request.required.acks greater than 1") } catch { - case se: FailedToSendMessageException => true + case iae: IllegalArgumentException => // this is expected case e: Throwable => fail("Not expected", e) - } - finally { - producer2.close() + } } @@ -195,24 +210,23 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ @Test def testSendWithDeadBroker() { val props = new Properties() - props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("partitioner.class", "kafka.utils.StaticPartitioner") - props.put("request.timeout.ms", "2000") props.put("request.required.acks", "1") - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + // No need to retry since the topic will be created beforehand and normal send will succeed on the first try. + // Reducing the retries will save the time on the subsequent failure test. + props.put("message.send.max.retries", "0") val topic = "new-topic" // create topic - 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) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1, 500) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 2, 500) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 3, 500) - - val config = new ProducerConfig(props) - val producer = new Producer[String, String](config) + TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0)), + servers = servers) + + val producer = TestUtils.createProducer[String, String]( + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName, + producerProps = props) + try { // Available partition ids should be 0, 1, 2 and 3, all lead and hosted only // on broker 0 @@ -236,11 +250,12 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ // restart server 1 server1.startup() - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) + TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) try { // cross check if broker 1 got the messages - val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) + val response1 = getConsumer1().fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) val messageSet1 = response1.messageSet(topic, 0).iterator assertTrue("Message set should have 1 message", messageSet1.hasNext) assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes), messageSet1.next.message) @@ -255,27 +270,27 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ def testAsyncSendCanCorrectlyFailWithTimeout() { val timeoutMs = 500 val props = new Properties() - props.put("serializer.class", "kafka.serializer.StringEncoder") - props.put("partitioner.class", "kafka.utils.StaticPartitioner") props.put("request.timeout.ms", String.valueOf(timeoutMs)) - props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) props.put("request.required.acks", "1") + props.put("message.send.max.retries", "0") props.put("client.id","ProducerTest-testAsyncSendCanCorrectlyFailWithTimeout") - val config = new ProducerConfig(props) - val producer = new Producer[String, String](config) + val producer = TestUtils.createProducer[String, String]( + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName, + producerProps = props) val topic = "new-topic" // create topics in ZK - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) - TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0, 1000) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) // do a simple test to make sure plumbing is okay try { // this message should be assigned to partition 0 whose leader is on broker 0 producer.send(new KeyedMessage[String, String](topic, "test", "test")) // cross check if brokers got the messages - val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) + val response1 = getConsumer1().fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) val messageSet1 = response1.messageSet("new-topic", 0).iterator assertTrue("Message set should have 1 message", messageSet1.hasNext) assertEquals(new Message("test".getBytes), messageSet1.next.message) @@ -300,28 +315,28 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ } val t2 = SystemTime.milliseconds - // make sure we don't wait fewer than numRetries*timeoutMs milliseconds - // we do this because the DefaultEventHandler retries a number of times - assertTrue((t2-t1) >= timeoutMs*config.messageSendMaxRetries) + // make sure we don't wait fewer than timeoutMs + assertTrue((t2-t1) >= timeoutMs) } - + @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) + val producer = TestUtils.createProducer[String, String]( + brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName) + 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) - + TestUtils.waitUntilTrue(() => + AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode, + "Topic new-topic not created after timeout", + waitTime = zookeeper.tickTime) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0) + 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 8d63e312d3eda..8c3fb7a393db0 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -18,26 +18,31 @@ package kafka.producer import java.net.SocketTimeoutException +import java.util.Properties + import junit.framework.Assert import kafka.admin.AdminUtils +import kafka.api.ProducerResponseStatus +import kafka.common.{ErrorMapping, TopicAndPartition} import kafka.integration.KafkaServerTestHarness import kafka.message._ import kafka.server.KafkaConfig import kafka.utils._ +import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.Test import org.scalatest.junit.JUnit3Suite -import kafka.api.ProducerResponseStatus -import kafka.common.{TopicAndPartition, ErrorMapping} class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { - private var messageBytes = new Array[Byte](2); - val configs = List(new KafkaConfig(TestUtils.createBrokerConfigs(1).head)) - val zookeeperConnect = TestZKUtils.zookeeperConnect + private val messageBytes = new Array[Byte](2) + // turning off controlled shutdown since testProducerCanTimeout() explicitly shuts down request handler pool. + def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfigs(1, zkConnect, false).head)) @Test def testReachableServer() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + + val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort()) + val producer = new SyncProducer(new SyncProducerConfig(props)) val firstStart = SystemTime.milliseconds @@ -72,7 +77,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testEmptyProduceRequest() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort()) + val correlationId = 0 val clientId = SyncProducerConfig.DefaultClientId @@ -89,11 +95,10 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLarge() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort()) val producer = new SyncProducer(new SyncProducerConfig(props)) - AdminUtils.createTopic(zkClient, "test", 1, 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "test", 0, 500) + TestUtils.createTopic(zkClient, "test", numPartitions = 1, replicationFactor = 1, servers = servers) val message1 = new Message(new Array[Byte](configs(0).messageMaxBytes + 1)) val messageSet1 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message1) @@ -113,16 +118,17 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { Assert.assertEquals(0, response2.status(TopicAndPartition("test", 0)).offset) } + @Test def testMessageSizeTooLargeWithAckZero() { val server = servers.head + val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort()) - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) props.put("request.required.acks", "0") val producer = new SyncProducer(new SyncProducerConfig(props)) AdminUtils.createTopic(zkClient, "test", 1, 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "test", 0, 500) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "test", 0) // This message will be dropped silently since message size too large. producer.send(TestUtils.produceRequest("test", 0, @@ -143,7 +149,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceCorrectlyReceivesResponse() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort()) val producer = new SyncProducer(new SyncProducerConfig(props)) val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) @@ -163,9 +169,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { // #2 - test that we get correct offsets when partition is owned by broker AdminUtils.createTopic(zkClient, "topic1", 1, 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "topic1", 0, 500) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "topic1", 0) AdminUtils.createTopic(zkClient, "topic3", 1, 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "topic3", 0, 500) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "topic3", 0) val response2 = producer.send(request) Assert.assertNotNull(response2) @@ -189,7 +195,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { val timeoutMs = 500 val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort()) val producer = new SyncProducer(new SyncProducerConfig(props)) val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) @@ -215,7 +221,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceRequestWithNoResponse() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + + val port = server.socketServer.boundPort(SecurityProtocol.PLAINTEXT) + val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 val clientId = SyncProducerConfig.DefaultClientId val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs @@ -225,4 +233,24 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { val response = producer.send(emptyRequest) Assert.assertTrue(response == null) } + + @Test + def testNotEnoughReplicas() { + val topicName = "minisrtest" + val server = servers.head + val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort()) + + props.put("request.required.acks", "-1") + + val producer = new SyncProducer(new SyncProducerConfig(props)) + val topicProps = new Properties() + topicProps.put("min.insync.replicas","2") + AdminUtils.createTopic(zkClient, topicName, 1, 1,topicProps) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicName, 0) + + val response = producer.send(TestUtils.produceRequest(topicName, 0, + new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)),-1)) + + Assert.assertEquals(ErrorMapping.NotEnoughReplicasCode, response.status(TopicAndPartition(topicName, 0)).error) + } } diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala old mode 100644 new mode 100755 index f0c4a56b61b4f..e899b023b3153 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -17,10 +17,11 @@ package kafka.server -import org.scalatest.junit.JUnit3Suite -import kafka.zk.ZooKeeperTestHarness import junit.framework.Assert._ -import kafka.utils.{ZkUtils, Utils, TestUtils} +import kafka.utils.{TestUtils, CoreUtils, ZkUtils} +import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.protocol.SecurityProtocol +import org.scalatest.junit.JUnit3Suite class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { var server : KafkaServer = null @@ -30,23 +31,25 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() - val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) + + val props = TestUtils.createBrokerConfig(brokerId, zkConnect) props.put("advertised.host.name", advertisedHostName) props.put("advertised.port", advertisedPort.toString) - - server = TestUtils.createServer(new KafkaConfig(props)) + + server = TestUtils.createServer(KafkaConfig.fromProps(props)) } override def tearDown() { server.shutdown() - Utils.rm(server.config.logDirs) + CoreUtils.rm(server.config.logDirs) super.tearDown() } def testBrokerAdvertiseToZK { val brokerInfo = ZkUtils.getBrokerInfo(zkClient, brokerId) - assertEquals(advertisedHostName, brokerInfo.get.host) - assertEquals(advertisedPort, brokerInfo.get.port) + val endpoint = brokerInfo.get.endPoints.get(SecurityProtocol.PLAINTEXT).get + assertEquals(advertisedHostName, endpoint.host) + assertEquals(advertisedPort, endpoint.port) } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala new file mode 100644 index 0000000000000..f3ab3f4ff8eb1 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala @@ -0,0 +1,129 @@ +/** + * 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 org.scalatest.junit.JUnit3Suite +import junit.framework.Assert._ + +class DelayedOperationTest extends JUnit3Suite { + + var purgatory: DelayedOperationPurgatory[MockDelayedOperation] = null + + override def setUp() { + super.setUp() + purgatory = new DelayedOperationPurgatory[MockDelayedOperation](purgatoryName = "mock") + } + + override def tearDown() { + purgatory.shutdown() + super.tearDown() + } + + @Test + def testRequestSatisfaction() { + val r1 = new MockDelayedOperation(100000L) + val r2 = new MockDelayedOperation(100000L) + assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.checkAndComplete("test1")) + assertFalse("r1 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r1, Array("test1"))) + assertEquals("Still nothing satisfied", 0, purgatory.checkAndComplete("test1")) + assertFalse("r2 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r2, Array("test2"))) + assertEquals("Still nothing satisfied", 0, purgatory.checkAndComplete("test2")) + r1.completable = true + assertEquals("r1 satisfied", 1, purgatory.checkAndComplete("test1")) + assertEquals("Nothing satisfied", 0, purgatory.checkAndComplete("test1")) + r2.completable = true + assertEquals("r2 satisfied", 1, purgatory.checkAndComplete("test2")) + assertEquals("Nothing satisfied", 0, purgatory.checkAndComplete("test2")) + } + + @Test + def testRequestExpiry() { + val expiration = 20L + val start = System.currentTimeMillis + val r1 = new MockDelayedOperation(expiration) + val r2 = new MockDelayedOperation(200000L) + assertFalse("r1 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r1, Array("test1"))) + assertFalse("r2 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r2, Array("test2"))) + r1.awaitExpiration() + val elapsed = System.currentTimeMillis - start + assertTrue("r1 completed due to expiration", r1.isCompleted()) + assertFalse("r2 hasn't completed", r2.isCompleted()) + assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) + } + + @Test + def testRequestPurge() { + val r1 = new MockDelayedOperation(100000L) + val r2 = new MockDelayedOperation(100000L) + val r3 = new MockDelayedOperation(100000L) + purgatory.tryCompleteElseWatch(r1, Array("test1")) + purgatory.tryCompleteElseWatch(r2, Array("test1", "test2")) + purgatory.tryCompleteElseWatch(r3, Array("test1", "test2", "test3")) + + assertEquals("Purgatory should have 3 total delayed operations", 3, purgatory.delayed()) + assertEquals("Purgatory should have 6 watched elements", 6, purgatory.watched()) + + // complete the operations, it should immediately be purged from the delayed operation + r2.completable = true + r2.tryComplete() + assertEquals("Purgatory should have 2 total delayed operations instead of " + purgatory.delayed(), 2, purgatory.delayed()) + + r3.completable = true + r3.tryComplete() + assertEquals("Purgatory should have 1 total delayed operations instead of " + purgatory.delayed(), 1, purgatory.delayed()) + + // checking a watch should purge the watch list + purgatory.checkAndComplete("test1") + assertEquals("Purgatory should have 4 watched elements instead of " + purgatory.watched(), 4, purgatory.watched()) + + purgatory.checkAndComplete("test2") + assertEquals("Purgatory should have 2 watched elements instead of " + purgatory.watched(), 2, purgatory.watched()) + + purgatory.checkAndComplete("test3") + assertEquals("Purgatory should have 1 watched elements instead of " + purgatory.watched(), 1, purgatory.watched()) + } + + class MockDelayedOperation(delayMs: Long) extends DelayedOperation(delayMs) { + var completable = false + + def awaitExpiration() { + synchronized { + wait() + } + } + + override def tryComplete() = { + if (completable) + forceComplete() + else + false + } + + override def onExpiration() { + + } + + override def onComplete() { + synchronized { + notify() + } + } + } + +} diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala new file mode 100644 index 0000000000000..8a871cfaf6a53 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -0,0 +1,66 @@ +/** + * 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 junit.framework.Assert._ +import org.junit.Test +import kafka.integration.KafkaServerTestHarness +import kafka.utils._ +import kafka.common._ +import kafka.log.LogConfig +import kafka.admin.{AdminOperationException, AdminUtils} +import org.scalatest.junit.JUnit3Suite + +class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { + def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) + + @Test + def testConfigChange() { + val oldVal: java.lang.Long = 100000 + val newVal: java.lang.Long = 200000 + val tp = TopicAndPartition("test", 0) + val logProps = new Properties() + logProps.put(LogConfig.FlushMessagesProp, oldVal.toString) + AdminUtils.createTopic(zkClient, tp.topic, 1, 1, logProps) + TestUtils.retry(10000) { + val logOpt = this.servers(0).logManager.getLog(tp) + assertTrue(logOpt.isDefined) + assertEquals(oldVal, logOpt.get.config.flushInterval) + } + logProps.put(LogConfig.FlushMessagesProp, newVal.toString) + AdminUtils.changeTopicConfig(zkClient, tp.topic, logProps) + TestUtils.retry(10000) { + assertEquals(newVal, this.servers(0).logManager.getLog(tp).get.config.flushInterval) + } + } + + @Test + def testConfigChangeOnNonExistingTopic() { + val topic = TestUtils.tempTopic + try { + val logProps = new Properties() + logProps.put(LogConfig.FlushMessagesProp, 10000: java.lang.Integer) + AdminUtils.changeTopicConfig(zkClient, topic, logProps) + fail("Should fail with AdminOperationException for topic doesn't exist") + } catch { + case e: AdminOperationException => // expected + } + } + +} \ 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 old mode 100644 new mode 100755 index 02c188a412995..60cd8249e6ec0 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -25,27 +25,23 @@ import org.junit._ import org.junit.Assert._ import kafka.common._ import kafka.cluster.Replica -import kafka.utils.{SystemTime, KafkaScheduler, TestUtils, MockTime, Utils} +import kafka.utils.{SystemTime, KafkaScheduler, TestUtils, MockTime, CoreUtils} import java.util.concurrent.atomic.AtomicBoolean class HighwatermarkPersistenceTest extends JUnit3Suite { - val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_)) + val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps) val topic = "foo" - 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)) + val logManagers = configs map { config => + TestUtils.createLogManager( + logDirs = config.logDirs.map(new File(_)).toArray, + cleanerConfig = CleanerConfig()) + } @After def teardown() { for(manager <- logManagers; dir <- manager.logDirs) - Utils.rm(dir) + CoreUtils.rm(dir) } def testHighWatermarkPersistenceSinglePartition() { @@ -62,7 +58,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { replicaManager.checkpointHighWatermarks() var fooPartition0Hw = hwmFor(replicaManager, topic, 0) assertEquals(0L, fooPartition0Hw) - val partition0 = replicaManager.getOrCreatePartition(topic, 0, 1) + val partition0 = replicaManager.getOrCreatePartition(topic, 0) // create leader and follower replicas val log0 = logManagers(0).createLog(TopicAndPartition(topic, 0), LogConfig()) val leaderReplicaPartition0 = new Replica(configs.head.brokerId, partition0, SystemTime, 0, Some(log0)) @@ -71,19 +67,16 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { partition0.addReplicaIfNotExists(followerReplicaPartition0) replicaManager.checkpointHighWatermarks() fooPartition0Hw = hwmFor(replicaManager, topic, 0) - assertEquals(leaderReplicaPartition0.highWatermark, fooPartition0Hw) - try { - followerReplicaPartition0.highWatermark - fail("Should fail with KafkaException") - }catch { - case e: KafkaException => // this is ok - } - // set the highwatermark for local replica - partition0.getReplica().get.highWatermark = 5L + assertEquals(leaderReplicaPartition0.highWatermark.messageOffset, fooPartition0Hw) + // set the high watermark for local replica + partition0.getReplica().get.highWatermark = new LogOffsetMetadata(5L) replicaManager.checkpointHighWatermarks() fooPartition0Hw = hwmFor(replicaManager, topic, 0) - assertEquals(leaderReplicaPartition0.highWatermark, fooPartition0Hw) + assertEquals(leaderReplicaPartition0.highWatermark.messageOffset, fooPartition0Hw) EasyMock.verify(zkClient) + + // shutdown the replica manager upon test completion + replicaManager.shutdown(false) } def testHighWatermarkPersistenceMultiplePartitions() { @@ -101,7 +94,7 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { replicaManager.checkpointHighWatermarks() var topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) assertEquals(0L, topic1Partition0Hw) - val topic1Partition0 = replicaManager.getOrCreatePartition(topic1, 0, 1) + val topic1Partition0 = replicaManager.getOrCreatePartition(topic1, 0) // create leader log val topic1Log0 = logManagers(0).createLog(TopicAndPartition(topic1, 0), LogConfig()) // create a local replica for topic1 @@ -109,15 +102,15 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { topic1Partition0.addReplicaIfNotExists(leaderReplicaTopic1Partition0) replicaManager.checkpointHighWatermarks() topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) - assertEquals(leaderReplicaTopic1Partition0.highWatermark, topic1Partition0Hw) - // set the highwatermark for local replica - topic1Partition0.getReplica().get.highWatermark = 5L + assertEquals(leaderReplicaTopic1Partition0.highWatermark.messageOffset, topic1Partition0Hw) + // set the high watermark for local replica + topic1Partition0.getReplica().get.highWatermark = new LogOffsetMetadata(5L) replicaManager.checkpointHighWatermarks() topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) - assertEquals(5L, leaderReplicaTopic1Partition0.highWatermark) + assertEquals(5L, leaderReplicaTopic1Partition0.highWatermark.messageOffset) assertEquals(5L, topic1Partition0Hw) // add another partition and set highwatermark - val topic2Partition0 = replicaManager.getOrCreatePartition(topic2, 0, 1) + val topic2Partition0 = replicaManager.getOrCreatePartition(topic2, 0) // create leader log val topic2Log0 = logManagers(0).createLog(TopicAndPartition(topic2, 0), LogConfig()) // create a local replica for topic2 @@ -125,13 +118,13 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { topic2Partition0.addReplicaIfNotExists(leaderReplicaTopic2Partition0) replicaManager.checkpointHighWatermarks() var topic2Partition0Hw = hwmFor(replicaManager, topic2, 0) - assertEquals(leaderReplicaTopic2Partition0.highWatermark, topic2Partition0Hw) + assertEquals(leaderReplicaTopic2Partition0.highWatermark.messageOffset, topic2Partition0Hw) // set the highwatermark for local replica - topic2Partition0.getReplica().get.highWatermark = 15L - assertEquals(15L, leaderReplicaTopic2Partition0.highWatermark) + topic2Partition0.getReplica().get.highWatermark = new LogOffsetMetadata(15L) + assertEquals(15L, leaderReplicaTopic2Partition0.highWatermark.messageOffset) // change the highwatermark for topic1 - topic1Partition0.getReplica().get.highWatermark = 10L - assertEquals(10L, leaderReplicaTopic1Partition0.highWatermark) + topic1Partition0.getReplica().get.highWatermark = new LogOffsetMetadata(10L) + assertEquals(10L, leaderReplicaTopic1Partition0.highWatermark.messageOffset) replicaManager.checkpointHighWatermarks() // verify checkpointed hw for topic 2 topic2Partition0Hw = hwmFor(replicaManager, topic2, 0) @@ -140,10 +133,14 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) assertEquals(10L, topic1Partition0Hw) EasyMock.verify(zkClient) + + // shutdown the replica manager upon test completion + replicaManager.shutdown(false) + } def hwmFor(replicaManager: ReplicaManager, topic: String, partition: Int): Long = { - replicaManager.highWatermarkCheckpoints(replicaManager.config.logDirs(0)).read.getOrElse(TopicAndPartition(topic, partition), 0L) + replicaManager.highWatermarkCheckpoints(new File(replicaManager.config.logDirs(0)).getAbsolutePath).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 2cd3a3faf7be2..90529faf11dca 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -16,6 +16,8 @@ */ package kafka.server +import java.util.Properties + import org.scalatest.junit.JUnit3Suite import collection.mutable.HashMap import collection.mutable.Map @@ -25,19 +27,39 @@ import kafka.log.Log import org.junit.Assert._ import kafka.utils._ import java.util.concurrent.atomic.AtomicBoolean +import kafka.message.MessageSet + 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 replicaLagTimeMaxMs = 100L + val replicaFetchWaitMaxMs = 100 + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) + val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, overridingProps)) val topic = "foo" + val time = new MockTime + + var replicaManager: ReplicaManager = null + + override def setUp() { + super.setUp() + replicaManager = new ReplicaManager(configs.head, time, null, null, null, new AtomicBoolean(false)) + } + + override def tearDown() { + replicaManager.shutdown(false) + super.tearDown() + } + + /* + * Test the case where a follower is caught up but stops making requests to the leader. Once beyond the configured time limit, it should fall out of ISR + */ def testIsrExpirationForStuckFollowers() { - val time = new MockTime val log = getLogWithLogEndOffset(15L, 2) // set logEndOffset for leader to 15L // create one partition and all replicas @@ -45,45 +67,91 @@ class IsrExpirationTest extends JUnit3Suite { assertEquals("All replicas should be in ISR", configs.map(_.brokerId).toSet, partition0.inSyncReplicas.map(_.brokerId)) val leaderReplica = partition0.getReplica(configs.head.brokerId).get - // let the follower catch up to 10 - (partition0.assignedReplicas() - leaderReplica).foreach(r => r.logEndOffset = 10) - var partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs, configs.head.replicaLagMaxMessages) + // let the follower catch up to the Leader logEndOffset (15) + (partition0.assignedReplicas() - leaderReplica).foreach( + r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), + MessageSet.Empty), + -1L, + -1, + true))) + var partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs) assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId)) // let some time pass time.sleep(150) - // now follower (broker id 1) has caught up to only 10, while the leader is at 15 AND the follower hasn't - // pulled any data for > replicaMaxLagTimeMs ms. So it is stuck - partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs, configs.head.replicaLagMaxMessages) + // now follower hasn't pulled any data for > replicaMaxLagTimeMs ms. So it is stuck + partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs) + assertEquals("Replica 1 should be out of sync", Set(configs.last.brokerId), partition0OSR.map(_.brokerId)) + EasyMock.verify(log) + } + + /* + * Test the case where a follower never makes a fetch request. It should fall out of ISR because it will be declared stuck + */ + def testIsrExpirationIfNoFetchRequestMade() { + val log = getLogWithLogEndOffset(15L, 1) // set logEndOffset for leader to 15L + + // create one partition and all replicas + val partition0 = getPartitionWithAllReplicasInIsr(topic, 0, time, configs.head, log) + assertEquals("All replicas should be in ISR", configs.map(_.brokerId).toSet, partition0.inSyncReplicas.map(_.brokerId)) + val leaderReplica = partition0.getReplica(configs.head.brokerId).get + + // Let enough time pass for the replica to be considered stuck + time.sleep(150) + + val partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs) assertEquals("Replica 1 should be out of sync", Set(configs.last.brokerId), partition0OSR.map(_.brokerId)) EasyMock.verify(log) } + /* + * Test the case where a follower continually makes fetch requests but is unable to catch up. It should fall out of the ISR + * However, any time it makes a request to the LogEndOffset it should be back in the ISR + */ def testIsrExpirationForSlowFollowers() { - val time = new MockTime // create leader replica - val log = getLogWithLogEndOffset(15L, 1) + val log = getLogWithLogEndOffset(15L, 4) // add one partition val partition0 = getPartitionWithAllReplicasInIsr(topic, 0, time, configs.head, log) assertEquals("All replicas should be in ISR", configs.map(_.brokerId).toSet, partition0.inSyncReplicas.map(_.brokerId)) val leaderReplica = partition0.getReplica(configs.head.brokerId).get - // set remote replicas leo to something low, like 4 - (partition0.assignedReplicas() - leaderReplica).foreach(r => r.logEndOffset = 4L) - // now follower (broker id 1) has caught up to only 4, while the leader is at 15. Since the gap it larger than - // replicaMaxLagBytes, the follower is out of sync. - val partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs, configs.head.replicaLagMaxMessages) + // Make the remote replica not read to the end of log. It should be not be out of sync for at least 100 ms + for(replica <- (partition0.assignedReplicas() - leaderReplica)) + replica.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(10L), MessageSet.Empty), -1L, -1, false)) + + // Simulate 2 fetch requests spanning more than 100 ms which do not read to the end of the log. + // The replicas will no longer be in ISR. We do 2 fetches because we want to simulate the case where the replica is lagging but is not stuck + var partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs) + assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId)) + + time.sleep(75) + + (partition0.assignedReplicas() - leaderReplica).foreach( + r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(11L), MessageSet.Empty), -1L, -1, false))) + partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs) + assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId)) + + time.sleep(75) + + // The replicas will no longer be in ISR + partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs) assertEquals("Replica 1 should be out of sync", Set(configs.last.brokerId), partition0OSR.map(_.brokerId)) + // Now actually make a fetch to the end of the log. The replicas should be back in ISR + (partition0.assignedReplicas() - leaderReplica).foreach( + r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MessageSet.Empty), -1L, -1, true))) + partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs) + assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId)) + EasyMock.verify(log) } private def getPartitionWithAllReplicasInIsr(topic: String, partitionId: Int, time: Time, config: KafkaConfig, localLog: Log): Partition = { val leaderId=config.brokerId - val replicaManager = new ReplicaManager(config, time, null, null, null, new AtomicBoolean(false)) - val partition = replicaManager.getOrCreatePartition(topic, partitionId, 1) + val partition = replicaManager.getOrCreatePartition(topic, partitionId) val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog)) val allReplicas = getFollowerReplicas(partition, leaderId, time) :+ leaderReplica @@ -97,7 +165,7 @@ class IsrExpirationTest extends JUnit3Suite { private def getLogWithLogEndOffset(logEndOffset: Long, expectedCalls: Int): Log = { val log1 = EasyMock.createMock(classOf[kafka.log.Log]) - EasyMock.expect(log1.logEndOffset).andReturn(logEndOffset).times(expectedCalls) + EasyMock.expect(log1.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(logEndOffset)).times(expectedCalls) EasyMock.replay(log1) log1 diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala new file mode 100644 index 0000000000000..04a02e08a5413 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -0,0 +1,403 @@ +/** + * 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 kafka.api.ApiVersion +import kafka.message._ +import org.apache.kafka.common.protocol.SecurityProtocol +import org.junit.{Assert, Test} +import org.scalatest.junit.JUnit3Suite + +import scala.collection.Map +import scala.util.Random._ + +class KafkaConfigConfigDefTest extends JUnit3Suite { + + @Test + def testFromPropsEmpty() { + // only required + val p = new Properties() + p.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + val actualConfig = KafkaConfig.fromProps(p) + + val expectedConfig = new KafkaConfig(p) + + Assert.assertEquals(expectedConfig.zkConnect, actualConfig.zkConnect) + Assert.assertEquals(expectedConfig.zkSessionTimeoutMs, actualConfig.zkSessionTimeoutMs) + Assert.assertEquals(expectedConfig.zkConnectionTimeoutMs, actualConfig.zkConnectionTimeoutMs) + Assert.assertEquals(expectedConfig.zkSyncTimeMs, actualConfig.zkSyncTimeMs) + Assert.assertEquals(expectedConfig.maxReservedBrokerId, actualConfig.maxReservedBrokerId) + Assert.assertEquals(expectedConfig.brokerId, actualConfig.brokerId) + Assert.assertEquals(expectedConfig.messageMaxBytes, actualConfig.messageMaxBytes) + Assert.assertEquals(expectedConfig.numNetworkThreads, actualConfig.numNetworkThreads) + Assert.assertEquals(expectedConfig.numIoThreads, actualConfig.numIoThreads) + Assert.assertEquals(expectedConfig.backgroundThreads, actualConfig.backgroundThreads) + Assert.assertEquals(expectedConfig.queuedMaxRequests, actualConfig.queuedMaxRequests) + + Assert.assertEquals(expectedConfig.port, actualConfig.port) + Assert.assertEquals(expectedConfig.hostName, actualConfig.hostName) + Assert.assertEquals(expectedConfig.advertisedHostName, actualConfig.advertisedHostName) + Assert.assertEquals(expectedConfig.advertisedPort, actualConfig.advertisedPort) + Assert.assertEquals(expectedConfig.socketSendBufferBytes, actualConfig.socketSendBufferBytes) + Assert.assertEquals(expectedConfig.socketReceiveBufferBytes, actualConfig.socketReceiveBufferBytes) + Assert.assertEquals(expectedConfig.socketRequestMaxBytes, actualConfig.socketRequestMaxBytes) + Assert.assertEquals(expectedConfig.maxConnectionsPerIp, actualConfig.maxConnectionsPerIp) + Assert.assertEquals(expectedConfig.maxConnectionsPerIpOverrides, actualConfig.maxConnectionsPerIpOverrides) + Assert.assertEquals(expectedConfig.connectionsMaxIdleMs, actualConfig.connectionsMaxIdleMs) + + Assert.assertEquals(expectedConfig.numPartitions, actualConfig.numPartitions) + Assert.assertEquals(expectedConfig.logDirs, actualConfig.logDirs) + + Assert.assertEquals(expectedConfig.logSegmentBytes, actualConfig.logSegmentBytes) + + Assert.assertEquals(expectedConfig.logRollTimeMillis, actualConfig.logRollTimeMillis) + Assert.assertEquals(expectedConfig.logRollTimeJitterMillis, actualConfig.logRollTimeJitterMillis) + Assert.assertEquals(expectedConfig.logRetentionTimeMillis, actualConfig.logRetentionTimeMillis) + + Assert.assertEquals(expectedConfig.logRetentionBytes, actualConfig.logRetentionBytes) + Assert.assertEquals(expectedConfig.logCleanupIntervalMs, actualConfig.logCleanupIntervalMs) + Assert.assertEquals(expectedConfig.logCleanupPolicy, actualConfig.logCleanupPolicy) + Assert.assertEquals(expectedConfig.logCleanerThreads, actualConfig.logCleanerThreads) + Assert.assertEquals(expectedConfig.logCleanerIoMaxBytesPerSecond, actualConfig.logCleanerIoMaxBytesPerSecond, 0.0) + Assert.assertEquals(expectedConfig.logCleanerDedupeBufferSize, actualConfig.logCleanerDedupeBufferSize) + Assert.assertEquals(expectedConfig.logCleanerIoBufferSize, actualConfig.logCleanerIoBufferSize) + Assert.assertEquals(expectedConfig.logCleanerDedupeBufferLoadFactor, actualConfig.logCleanerDedupeBufferLoadFactor, 0.0) + Assert.assertEquals(expectedConfig.logCleanerBackoffMs, actualConfig.logCleanerBackoffMs) + Assert.assertEquals(expectedConfig.logCleanerMinCleanRatio, actualConfig.logCleanerMinCleanRatio, 0.0) + Assert.assertEquals(expectedConfig.logCleanerEnable, actualConfig.logCleanerEnable) + Assert.assertEquals(expectedConfig.logCleanerDeleteRetentionMs, actualConfig.logCleanerDeleteRetentionMs) + Assert.assertEquals(expectedConfig.logIndexSizeMaxBytes, actualConfig.logIndexSizeMaxBytes) + Assert.assertEquals(expectedConfig.logIndexIntervalBytes, actualConfig.logIndexIntervalBytes) + Assert.assertEquals(expectedConfig.logFlushIntervalMessages, actualConfig.logFlushIntervalMessages) + Assert.assertEquals(expectedConfig.logDeleteDelayMs, actualConfig.logDeleteDelayMs) + Assert.assertEquals(expectedConfig.logFlushSchedulerIntervalMs, actualConfig.logFlushSchedulerIntervalMs) + Assert.assertEquals(expectedConfig.logFlushIntervalMs, actualConfig.logFlushIntervalMs) + Assert.assertEquals(expectedConfig.logFlushOffsetCheckpointIntervalMs, actualConfig.logFlushOffsetCheckpointIntervalMs) + Assert.assertEquals(expectedConfig.numRecoveryThreadsPerDataDir, actualConfig.numRecoveryThreadsPerDataDir) + Assert.assertEquals(expectedConfig.autoCreateTopicsEnable, actualConfig.autoCreateTopicsEnable) + + Assert.assertEquals(expectedConfig.minInSyncReplicas, actualConfig.minInSyncReplicas) + + Assert.assertEquals(expectedConfig.controllerSocketTimeoutMs, actualConfig.controllerSocketTimeoutMs) + Assert.assertEquals(expectedConfig.defaultReplicationFactor, actualConfig.defaultReplicationFactor) + Assert.assertEquals(expectedConfig.replicaLagTimeMaxMs, actualConfig.replicaLagTimeMaxMs) + Assert.assertEquals(expectedConfig.replicaSocketTimeoutMs, actualConfig.replicaSocketTimeoutMs) + Assert.assertEquals(expectedConfig.replicaSocketReceiveBufferBytes, actualConfig.replicaSocketReceiveBufferBytes) + Assert.assertEquals(expectedConfig.replicaFetchMaxBytes, actualConfig.replicaFetchMaxBytes) + Assert.assertEquals(expectedConfig.replicaFetchWaitMaxMs, actualConfig.replicaFetchWaitMaxMs) + Assert.assertEquals(expectedConfig.replicaFetchMinBytes, actualConfig.replicaFetchMinBytes) + Assert.assertEquals(expectedConfig.replicaFetchBackoffMs, actualConfig.replicaFetchBackoffMs) + Assert.assertEquals(expectedConfig.numReplicaFetchers, actualConfig.numReplicaFetchers) + Assert.assertEquals(expectedConfig.replicaHighWatermarkCheckpointIntervalMs, actualConfig.replicaHighWatermarkCheckpointIntervalMs) + Assert.assertEquals(expectedConfig.fetchPurgatoryPurgeIntervalRequests, actualConfig.fetchPurgatoryPurgeIntervalRequests) + Assert.assertEquals(expectedConfig.producerPurgatoryPurgeIntervalRequests, actualConfig.producerPurgatoryPurgeIntervalRequests) + Assert.assertEquals(expectedConfig.autoLeaderRebalanceEnable, actualConfig.autoLeaderRebalanceEnable) + Assert.assertEquals(expectedConfig.leaderImbalancePerBrokerPercentage, actualConfig.leaderImbalancePerBrokerPercentage) + Assert.assertEquals(expectedConfig.leaderImbalanceCheckIntervalSeconds, actualConfig.leaderImbalanceCheckIntervalSeconds) + Assert.assertEquals(expectedConfig.uncleanLeaderElectionEnable, actualConfig.uncleanLeaderElectionEnable) + + Assert.assertEquals(expectedConfig.controlledShutdownMaxRetries, actualConfig.controlledShutdownMaxRetries) + Assert.assertEquals(expectedConfig.controlledShutdownRetryBackoffMs, actualConfig.controlledShutdownRetryBackoffMs) + Assert.assertEquals(expectedConfig.controlledShutdownEnable, actualConfig.controlledShutdownEnable) + + Assert.assertEquals(expectedConfig.consumerMinSessionTimeoutMs, actualConfig.consumerMinSessionTimeoutMs) + Assert.assertEquals(expectedConfig.consumerMaxSessionTimeoutMs, actualConfig.consumerMaxSessionTimeoutMs) + + Assert.assertEquals(expectedConfig.offsetMetadataMaxSize, actualConfig.offsetMetadataMaxSize) + Assert.assertEquals(expectedConfig.offsetsLoadBufferSize, actualConfig.offsetsLoadBufferSize) + Assert.assertEquals(expectedConfig.offsetsTopicReplicationFactor, actualConfig.offsetsTopicReplicationFactor) + Assert.assertEquals(expectedConfig.offsetsTopicPartitions, actualConfig.offsetsTopicPartitions) + Assert.assertEquals(expectedConfig.offsetsTopicSegmentBytes, actualConfig.offsetsTopicSegmentBytes) + Assert.assertEquals(expectedConfig.offsetsTopicCompressionCodec, actualConfig.offsetsTopicCompressionCodec) + Assert.assertEquals(expectedConfig.offsetsRetentionMinutes, actualConfig.offsetsRetentionMinutes) + Assert.assertEquals(expectedConfig.offsetsRetentionCheckIntervalMs, actualConfig.offsetsRetentionCheckIntervalMs) + Assert.assertEquals(expectedConfig.offsetCommitTimeoutMs, actualConfig.offsetCommitTimeoutMs) + Assert.assertEquals(expectedConfig.offsetCommitRequiredAcks, actualConfig.offsetCommitRequiredAcks) + + Assert.assertEquals(expectedConfig.deleteTopicEnable, actualConfig.deleteTopicEnable) + Assert.assertEquals(expectedConfig.compressionType, actualConfig.compressionType) + } + + private def atLeastXIntProp(x: Int): String = (nextInt(Int.MaxValue - x) + x).toString + + private def atLeastOneIntProp: String = atLeastXIntProp(1) + + private def inRangeIntProp(fromInc: Int, toInc: Int): String = (nextInt(toInc + 1 - fromInc) + fromInc).toString + + @Test + def testFromPropsToProps() { + import scala.util.Random._ + val expected = new Properties() + KafkaConfig.configNames().foreach(name => { + name match { + case KafkaConfig.ZkConnectProp => expected.setProperty(name, "127.0.0.1:2181") + case KafkaConfig.ZkSessionTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.ZkConnectionTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.ZkSyncTimeMsProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.NumNetworkThreadsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.NumIoThreadsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.BackgroundThreadsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.QueuedMaxRequestsProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.PortProp => expected.setProperty(name, "1234") + case KafkaConfig.HostNameProp => expected.setProperty(name, "hostname") + case KafkaConfig.ListenersProp => expected.setProperty(name, "PLAINTEXT://:9092") + case KafkaConfig.AdvertisedHostNameProp => expected.setProperty(name, "advertised.hostname") + case KafkaConfig.AdvertisedPortProp => expected.setProperty(name, "4321") + case KafkaConfig.AdvertisedListenersProp => expected.setProperty(name, "PLAINTEXT://:2909") + case KafkaConfig.SocketRequestMaxBytesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.MaxConnectionsPerIpProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.MaxConnectionsPerIpOverridesProp => expected.setProperty(name, "127.0.0.1:2, 127.0.0.2:3") + + case KafkaConfig.NumPartitionsProp => expected.setProperty(name, "2") + case KafkaConfig.LogDirsProp => expected.setProperty(name, "/tmp/logs,/tmp/logs2") + case KafkaConfig.LogDirProp => expected.setProperty(name, "/tmp/log") + case KafkaConfig.LogSegmentBytesProp => expected.setProperty(name, atLeastXIntProp(Message.MinHeaderSize)) + + case KafkaConfig.LogRollTimeMillisProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.LogRollTimeHoursProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.LogRetentionTimeMillisProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.LogRetentionTimeMinutesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.LogRetentionTimeHoursProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.LogCleanupIntervalMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.LogCleanupPolicyProp => expected.setProperty(name, randFrom(Defaults.Compact, Defaults.Delete)) + case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) + case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) + case KafkaConfig.LogCleanerMinCleanRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) + case KafkaConfig.LogCleanerEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.LogIndexSizeMaxBytesProp => expected.setProperty(name, atLeastXIntProp(4)) + case KafkaConfig.LogFlushIntervalMessagesProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.NumRecoveryThreadsPerDataDirProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.AutoCreateTopicsEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.MinInSyncReplicasProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.AutoLeaderRebalanceEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.LogPreAllocateProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.InterBrokerSecurityProtocolProp => expected.setProperty(name, SecurityProtocol.PLAINTEXT.toString) + case KafkaConfig.InterBrokerProtocolVersionProp => expected.setProperty(name, ApiVersion.latestVersion.toString) + + case KafkaConfig.ControlledShutdownEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.OffsetsLoadBufferSizeProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetsTopicPartitionsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetsTopicSegmentBytesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetsTopicCompressionCodecProp => expected.setProperty(name, randFrom(GZIPCompressionCodec.codec.toString, + SnappyCompressionCodec.codec.toString, LZ4CompressionCodec.codec.toString, NoCompressionCodec.codec.toString)) + case KafkaConfig.OffsetsRetentionMinutesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetCommitTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.DeleteTopicEnableProp => expected.setProperty(name, randFrom("true", "false")) + + // explicit, non trivial validations or with transient dependencies + + // require(brokerId >= -1 && brokerId <= maxReservedBrokerId) + case KafkaConfig.MaxReservedBrokerIdProp => expected.setProperty(name, "100") + case KafkaConfig.BrokerIdProp => expected.setProperty(name, inRangeIntProp(0, 100)) + // require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024) + case KafkaConfig.LogCleanerThreadsProp => expected.setProperty(name, "2") + case KafkaConfig.LogCleanerDedupeBufferSizeProp => expected.setProperty(name, (1024 * 1024 * 3 + 1).toString) + // require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs) + case KafkaConfig.ReplicaFetchWaitMaxMsProp => expected.setProperty(name, "321") + case KafkaConfig.ReplicaSocketTimeoutMsProp => expected.setProperty(name, atLeastXIntProp(321)) + // require(replicaFetchMaxBytes >= messageMaxBytes) + case KafkaConfig.MessageMaxBytesProp => expected.setProperty(name, "1234") + case KafkaConfig.ReplicaFetchMaxBytesProp => expected.setProperty(name, atLeastXIntProp(1234)) + // require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs) + case KafkaConfig.ReplicaLagTimeMaxMsProp => expected.setProperty(name, atLeastXIntProp(321)) + //require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor) + case KafkaConfig.OffsetCommitRequiredAcksProp => expected.setProperty(name, "-1") + case KafkaConfig.OffsetsTopicReplicationFactorProp => expected.setProperty(name, inRangeIntProp(1, Short.MaxValue)) + //BrokerCompressionCodec.isValid(compressionType) + case KafkaConfig.CompressionTypeProp => expected.setProperty(name, randFrom(BrokerCompressionCodec.brokerCompressionOptions)) + + case KafkaConfig.MetricNumSamplesProp => expected.setProperty(name, "2") + case KafkaConfig.MetricSampleWindowMsProp => expected.setProperty(name, "1000") + case KafkaConfig.MetricReporterClassesProp => expected.setProperty(name, "") + + case nonNegativeIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString) + } + }) + + val actual = KafkaConfig.fromProps(expected).originals + Assert.assertEquals(expected, actual) + } + + @Test + def testFromPropsInvalid() { + def getBaseProperties(): Properties = { + val validRequiredProperties = new Properties() + validRequiredProperties.put(KafkaConfig.ZkConnectProp, "127.0.0.1") + validRequiredProperties + } + // to ensure a basis is valid - bootstraps all needed validation + KafkaConfig.fromProps(getBaseProperties()) + + KafkaConfig.configNames().foreach(name => { + name match { + case KafkaConfig.ZkConnectProp => // ignore string + case KafkaConfig.ZkSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ZkConnectionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ZkSyncTimeMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + + case KafkaConfig.BrokerIdProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.NumNetworkThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.HostNameProp => // ignore string + case KafkaConfig.AdvertisedHostNameProp => //ignore string + case KafkaConfig.AdvertisedPortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.SocketSendBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.SocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.MaxConnectionsPerIpOverridesProp => + assertPropertyInvalid(getBaseProperties(), name, "127.0.0.1:not_a_number") + case KafkaConfig.ConnectionsMaxIdleMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + + case KafkaConfig.NumPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogDirsProp => // ignore string + case KafkaConfig.LogDirProp => // ignore string + case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", Message.MinHeaderSize - 1) + + case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + case KafkaConfig.LogRetentionTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogRetentionTimeMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogRetentionTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(getBaseProperties(), name, "unknown_policy", "0") + case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanerDedupeBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "1024") + case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanerEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean") + case KafkaConfig.LogCleanerDeleteRetentionMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanerMinCleanRatioProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "3") + case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.MinInSyncReplicasProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ControllerSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.DefaultReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaLagTimeMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") + case KafkaConfig.ReplicaSocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchWaitMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchMinBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.NumReplicaFetchersProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ProducerPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.AutoLeaderRebalanceEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.LeaderImbalancePerBrokerPercentageProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.UncleanLeaderElectionEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.ConsumerMinSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ConsumerMaxSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.OffsetMetadataMaxSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.OffsetsLoadBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicCompressionCodecProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") + case KafkaConfig.OffsetsRetentionMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetCommitTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetCommitRequiredAcksProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") + + case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + + case KafkaConfig.MetricNumSamplesProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") + case KafkaConfig.MetricSampleWindowMsProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") + case KafkaConfig.MetricReporterClassesProp => // ignore string + + case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") + } + }) + } + + @Test + def testSpecificProperties(): Unit = { + val defaults = new Properties() + defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + // For ZkConnectionTimeoutMs + defaults.put(KafkaConfig.ZkSessionTimeoutMsProp, "1234") + defaults.put(KafkaConfig.MaxReservedBrokerIdProp, "1") + defaults.put(KafkaConfig.BrokerIdProp, "1") + defaults.put(KafkaConfig.HostNameProp, "127.0.0.1") + defaults.put(KafkaConfig.PortProp, "1122") + defaults.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3") + defaults.put(KafkaConfig.LogDirProp, "/tmp1,/tmp2") + defaults.put(KafkaConfig.LogRollTimeHoursProp, "12") + defaults.put(KafkaConfig.LogRollTimeJitterHoursProp, "11") + defaults.put(KafkaConfig.LogRetentionTimeHoursProp, "10") + //For LogFlushIntervalMsProp + defaults.put(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123") + defaults.put(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString) + + val config = KafkaConfig.fromProps(defaults) + Assert.assertEquals("127.0.0.1:2181", config.zkConnect) + Assert.assertEquals(1234, config.zkConnectionTimeoutMs) + Assert.assertEquals(1, config.maxReservedBrokerId) + Assert.assertEquals(1, config.brokerId) + Assert.assertEquals("127.0.0.1", config.hostName) + Assert.assertEquals(1122, config.advertisedPort) + Assert.assertEquals("127.0.0.1", config.advertisedHostName) + Assert.assertEquals(Map("127.0.0.1" -> 2, "127.0.0.2" -> 3), config.maxConnectionsPerIpOverrides) + Assert.assertEquals(List("/tmp1", "/tmp2"), config.logDirs) + Assert.assertEquals(12 * 60L * 1000L * 60, config.logRollTimeMillis) + Assert.assertEquals(11 * 60L * 1000L * 60, config.logRollTimeJitterMillis) + Assert.assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis) + Assert.assertEquals(123L, config.logFlushIntervalMs) + Assert.assertEquals(SnappyCompressionCodec, config.offsetsTopicCompressionCodec) + } + + private def assertPropertyInvalid(validRequiredProps: => Properties, name: String, values: Any*) { + values.foreach((value) => { + val props = validRequiredProps + props.setProperty(name, value.toString) + intercept[Exception] { + KafkaConfig.fromProps(props) + } + }) + } + + private def randFrom[T](choices: T*): T = { + import scala.util.Random + choices(Random.nextInt(choices.size)) + } + + private def randFrom[T](choices: List[T]): T = { + import scala.util.Random + choices(Random.nextInt(choices.size)) + } +} diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala old mode 100644 new mode 100755 index 89c207a3f56c7..d3544526b66ce --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -17,81 +17,367 @@ package kafka.server -import org.junit.Test +import java.util.Properties + import junit.framework.Assert._ +import kafka.api.{ApiVersion, KAFKA_082} +import kafka.utils.{TestUtils, CoreUtils} +import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.protocol.SecurityProtocol +import org.junit.Test 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 props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRetentionTimeHoursProp, "1") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals(60L * 60L * 1000L, cfg.logRetentionTimeMillis) - } @Test def testLogRetentionTimeMinutesProvided() { - val props = TestUtils.createBrokerConfig(0, 8181) - props.put("log.retention.minutes", "30") + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRetentionTimeMinutesProp, "30") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) + } + + @Test + def testLogRetentionTimeMsProvided() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRetentionTimeMillisProp, "1800000") + val cfg = KafkaConfig.fromProps(props) + assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) } @Test def testLogRetentionTimeNoConfigProvided() { - val props = TestUtils.createBrokerConfig(0, 8181) + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(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 props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRetentionTimeMinutesProp, "30") + props.put(KafkaConfig.LogRetentionTimeHoursProp, "1") + + val cfg = KafkaConfig.fromProps(props) + assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) + } + + @Test + def testLogRetentionTimeBothMinutesAndMsProvided() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRetentionTimeMillisProp, "1800000") + props.put(KafkaConfig.LogRetentionTimeMinutesProp, "10") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) + } + @Test + def testLogRetentionUnlimited() { + val props1 = TestUtils.createBrokerConfig(0,TestUtils.MockZkConnect, port = 8181) + val props2 = TestUtils.createBrokerConfig(0,TestUtils.MockZkConnect, port = 8181) + val props3 = TestUtils.createBrokerConfig(0,TestUtils.MockZkConnect, port = 8181) + val props4 = TestUtils.createBrokerConfig(0,TestUtils.MockZkConnect, port = 8181) + val props5 = TestUtils.createBrokerConfig(0,TestUtils.MockZkConnect, port = 8181) + + props1.put("log.retention.ms", "-1") + props2.put("log.retention.minutes", "-1") + props3.put("log.retention.hours", "-1") + + val cfg1 = KafkaConfig.fromProps(props1) + val cfg2 = KafkaConfig.fromProps(props2) + val cfg3 = KafkaConfig.fromProps(props3) + assertEquals("Should be -1", -1, cfg1.logRetentionTimeMillis) + assertEquals("Should be -1", -1, cfg2.logRetentionTimeMillis) + assertEquals("Should be -1", -1, cfg3.logRetentionTimeMillis) + + props4.put("log.retention.ms", "-1") + props4.put("log.retention.minutes", "30") + + val cfg4 = KafkaConfig.fromProps(props4) + assertEquals("Should be -1", -1, cfg4.logRetentionTimeMillis) + + props5.put("log.retention.ms", "0") + + intercept[IllegalArgumentException] { + val cfg5 = KafkaConfig.fromProps(props5) + } + } + + @Test + def testLogRetentionValid { + val props1 = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val props2 = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val props3 = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + + props1.put("log.retention.ms", "0") + props2.put("log.retention.minutes", "0") + props3.put("log.retention.hours", "0") + + intercept[IllegalArgumentException] { + val cfg1 = KafkaConfig.fromProps(props1) + } + intercept[IllegalArgumentException] { + val cfg2 = KafkaConfig.fromProps(props2) + } + intercept[IllegalArgumentException] { + val cfg3 = KafkaConfig.fromProps(props3) + } } @Test def testAdvertiseDefaults() { - val port = 9999 + 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) + + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.remove(KafkaConfig.ListenersProp) + props.put(KafkaConfig.HostNameProp, hostName) + props.put(KafkaConfig.PortProp, port) + val serverConfig = KafkaConfig.fromProps(props) + val endpoints = serverConfig.advertisedListeners + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get + assertEquals(endpoint.host, hostName) + assertEquals(endpoint.port, port.toInt) } @Test def testAdvertiseConfigured() { - val port = 9999 val advertisedHostName = "routable-host" - val advertisedPort = 1234 + val advertisedPort = "1234" + + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.put(KafkaConfig.AdvertisedHostNameProp, advertisedHostName) + props.put(KafkaConfig.AdvertisedPortProp, advertisedPort) + + val serverConfig = KafkaConfig.fromProps(props) + val endpoints = serverConfig.advertisedListeners + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get - val props = TestUtils.createBrokerConfig(0, port) - props.put("advertised.host.name", advertisedHostName) - props.put("advertised.port", advertisedPort.toString) + assertEquals(endpoint.host, advertisedHostName) + assertEquals(endpoint.port, advertisedPort.toInt) + } - val serverConfig = new KafkaConfig(props) + @Test + def testAdvertisePortDefault() { + val advertisedHostName = "routable-host" + val port = "9999" + + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.put(KafkaConfig.AdvertisedHostNameProp, advertisedHostName) + props.put(KafkaConfig.PortProp, port) + + val serverConfig = KafkaConfig.fromProps(props) + val endpoints = serverConfig.advertisedListeners + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get + + assertEquals(endpoint.host, advertisedHostName) + assertEquals(endpoint.port, port.toInt) + } + + @Test + def testAdvertiseHostNameDefault() { + val hostName = "routable-host" + val advertisedPort = "9999" + + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.put(KafkaConfig.HostNameProp, hostName) + props.put(KafkaConfig.AdvertisedPortProp, advertisedPort) + + val serverConfig = KafkaConfig.fromProps(props) + val endpoints = serverConfig.advertisedListeners + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get + + assertEquals(endpoint.host, hostName) + assertEquals(endpoint.port, advertisedPort.toInt) + } - assertEquals(serverConfig.advertisedHostName, advertisedHostName) - assertEquals(serverConfig.advertisedPort, advertisedPort) + @Test + def testDuplicateListeners() { + val props = new Properties() + props.put(KafkaConfig.BrokerIdProp, "1") + props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + + // listeners with duplicate port + props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9091,TRACE://localhost:9091") + assert(!isValidKafkaConfig(props)) + + // listeners with duplicate protocol + props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9091,PLAINTEXT://localhost:9092") + assert(!isValidKafkaConfig(props)) + + // advertised listeners with duplicate port + props.put(KafkaConfig.AdvertisedListenersProp, "PLAINTEXT://localhost:9091,TRACE://localhost:9091") + assert(!isValidKafkaConfig(props)) + } + + @Test + def testBadListenerProtocol() { + val props = new Properties() + props.put(KafkaConfig.BrokerIdProp, "1") + props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.put(KafkaConfig.ListenersProp, "BAD://localhost:9091") + + assert(!isValidKafkaConfig(props)) + } + + @Test + def testListenerDefaults() { + val props = new Properties() + props.put(KafkaConfig.BrokerIdProp, "1") + props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + + // configuration with host and port, but no listeners + props.put(KafkaConfig.HostNameProp, "myhost") + props.put(KafkaConfig.PortProp, "1111") + + val conf = KafkaConfig.fromProps(props) + assertEquals(CoreUtils.listenerListToEndPoints("PLAINTEXT://myhost:1111"), conf.listeners) + + // configuration with null host + props.remove(KafkaConfig.HostNameProp) + + val conf2 = KafkaConfig.fromProps(props) + assertEquals(CoreUtils.listenerListToEndPoints("PLAINTEXT://:1111"), conf2.listeners) + assertEquals(CoreUtils.listenerListToEndPoints("PLAINTEXT://:1111"), conf2.advertisedListeners) + assertEquals(null, conf2.listeners(SecurityProtocol.PLAINTEXT).host) + + // configuration with advertised host and port, and no advertised listeners + props.put(KafkaConfig.AdvertisedHostNameProp, "otherhost") + props.put(KafkaConfig.AdvertisedPortProp, "2222") + + val conf3 = KafkaConfig.fromProps(props) + assertEquals(conf3.advertisedListeners, CoreUtils.listenerListToEndPoints("PLAINTEXT://otherhost:2222")) + } + + @Test + def testVersionConfiguration() { + val props = new Properties() + props.put(KafkaConfig.BrokerIdProp, "1") + props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + val conf = KafkaConfig.fromProps(props) + assertEquals(ApiVersion.latestVersion, conf.interBrokerProtocolVersion) + + props.put(KafkaConfig.InterBrokerProtocolVersionProp,"0.8.2.0") + val conf2 = KafkaConfig.fromProps(props) + assertEquals(KAFKA_082, conf2.interBrokerProtocolVersion) + + // check that 0.8.2.0 is the same as 0.8.2.1 + props.put(KafkaConfig.InterBrokerProtocolVersionProp,"0.8.2.1") + val conf3 = KafkaConfig.fromProps(props) + assertEquals(KAFKA_082, conf3.interBrokerProtocolVersion) + + //check that latest is newer than 0.8.2 + assert(ApiVersion.latestVersion.onOrAfter(conf3.interBrokerProtocolVersion)) + } + + private def isValidKafkaConfig(props: Properties): Boolean = { + try { + KafkaConfig.fromProps(props) + true + } catch { + case e: IllegalArgumentException => false + } + } + + @Test + def testUncleanLeaderElectionDefault() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val serverConfig = KafkaConfig.fromProps(props) + + assertEquals(serverConfig.uncleanLeaderElectionEnable, true) + } + + @Test + def testUncleanElectionDisabled() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.UncleanLeaderElectionEnableProp, String.valueOf(false)) + val serverConfig = KafkaConfig.fromProps(props) + + assertEquals(serverConfig.uncleanLeaderElectionEnable, false) + } + + @Test + def testUncleanElectionEnabled() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.UncleanLeaderElectionEnableProp, String.valueOf(true)) + val serverConfig = KafkaConfig.fromProps(props) + + assertEquals(serverConfig.uncleanLeaderElectionEnable, true) + } + + @Test + def testUncleanElectionInvalid() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.UncleanLeaderElectionEnableProp, "invalid") + + intercept[ConfigException] { + KafkaConfig.fromProps(props) + } + } + + @Test + def testLogRollTimeMsProvided() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRollTimeMillisProp, "1800000") + + val cfg = KafkaConfig.fromProps(props) + assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis) } + @Test + def testLogRollTimeBothMsAndHoursProvided() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.LogRollTimeMillisProp, "1800000") + props.put(KafkaConfig.LogRollTimeHoursProp, "1") + + val cfg = KafkaConfig.fromProps(props) + assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis) + } + + @Test + def testLogRollTimeNoConfigProvided() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + + val cfg = KafkaConfig.fromProps(props) + assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRollTimeMillis ) + } + + @Test + def testDefaultCompressionType() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val serverConfig = KafkaConfig.fromProps(props) + + assertEquals(serverConfig.compressionType, "producer") + } + + @Test + def testValidCompressionType() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put("compression.type", "gzip") + val serverConfig = KafkaConfig.fromProps(props) + + assertEquals(serverConfig.compressionType, "gzip") + } + + @Test + def testInvalidCompressionType() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + props.put(KafkaConfig.CompressionTypeProp, "abc") + intercept[IllegalArgumentException] { + KafkaConfig.fromProps(props) + } + } } diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala old mode 100644 new mode 100755 index 38e3ae72a87e1..f1977d850a5bf --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -17,41 +17,40 @@ package kafka.server -import org.scalatest.junit.JUnit3Suite -import kafka.zk.ZooKeeperTestHarness -import kafka.admin.AdminUtils -import kafka.utils.TestUtils._ import junit.framework.Assert._ -import kafka.utils.{ZkUtils, Utils, TestUtils} -import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} +import kafka.api._ +import kafka.utils.{TestUtils, ZkUtils, CoreUtils} import kafka.cluster.Broker import kafka.common.ErrorMapping -import kafka.api._ +import kafka.controller.{ControllerChannelManager, ControllerContext, LeaderIsrAndControllerEpoch} +import kafka.utils.TestUtils._ +import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.protocol.SecurityProtocol +import org.scalatest.junit.JUnit3Suite class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val brokerId1 = 0 val brokerId2 = 1 - val port1 = TestUtils.choosePort() - val port2 = TestUtils.choosePort() - - val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) - val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] var staleControllerEpochDetected = false override def setUp() { super.setUp() + + val configProps1 = TestUtils.createBrokerConfig(brokerId1, zkConnect, enableControlledShutdown = false) + val configProps2 = TestUtils.createBrokerConfig(brokerId2, zkConnect, enableControlledShutdown = false) + // start both servers - val server1 = TestUtils.createServer(new KafkaConfig(configProps1)) - val server2 = TestUtils.createServer(new KafkaConfig(configProps2)) + val server1 = TestUtils.createServer(KafkaConfig.fromProps(configProps1)) + val server2 = TestUtils.createServer(KafkaConfig.fromProps(configProps2)) servers ++= List(server1, server2) } override def tearDown() { - servers.map(server => server.shutdown()) - servers.map(server => Utils.rm(server.config.logDirs)) + servers.foreach(_.shutdown()) + servers.foreach(server => CoreUtils.rm(server.config.logDirs)) super.tearDown() } @@ -61,10 +60,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val partitionId = 0 // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(0, 1))) + val leader1 = createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(0, 1)), servers = servers)(0) - // wait until leader is elected - val leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) val leaderEpoch1 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId) debug("leader Epoc: " + leaderEpoch1) debug("Leader is elected to be: %s".format(leader1.getOrElse(-1))) @@ -76,8 +73,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // kill the server hosting the preferred replica servers.last.shutdown() // check if leader moves to the other server - val leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 1500, - if(leader1.get == 0) None else leader1) + val leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, + oldLeaderOpt = if(leader1.get == 0) None else leader1) val leaderEpoch2 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId) debug("Leader is elected to be: %s".format(leader1.getOrElse(-1))) debug("leader Epoc: " + leaderEpoch2) @@ -90,8 +87,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { servers.last.startup() servers.head.shutdown() Thread.sleep(zookeeper.tickTime) - val leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 1500, - if(leader2.get == 1) None else leader2) + val leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, + oldLeaderOpt = if(leader2.get == 1) None else leader2) val leaderEpoch3 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId) debug("leader Epoc: " + leaderEpoch3) debug("Leader is elected to be: %s".format(leader3.getOrElse(-1))) @@ -108,10 +105,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val partitionId = 0 // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0 -> Seq(0, 1))) + val leader1 = createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(0, 1)), servers = servers)(0) - // wait until leader is elected - val leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) val leaderEpoch1 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId) debug("leader Epoc: " + leaderEpoch1) debug("Leader is elected to be: %s".format(leader1.getOrElse(-1))) @@ -122,8 +117,11 @@ 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 controllerConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(controllerId, zkConnect)) + val brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.boundPort())) + val brokerEndPoints = brokers.map(b => b.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) + val controllerContext = new ControllerContext(zkClient, 6000) controllerContext.liveBrokers = brokers.toSet val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) @@ -133,11 +131,12 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { leaderAndIsr.put((topic, partitionId), new LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId2, List(brokerId1, brokerId2)), 2)) val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, Set(0,1))).toMap - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, controllerId, + val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokerEndPoints.toSet, controllerId, staleControllerEpoch, 0, "") controllerChannelManager.sendRequest(brokerId2, leaderAndIsrRequest, staleControllerEpochCallback) - TestUtils.waitUntilTrue(() => staleControllerEpochDetected == true, 1000) + TestUtils.waitUntilTrue(() => staleControllerEpochDetected == true, + "Controller epoch should be stale") assertTrue("Stale controller epoch not detected by the broker", staleControllerEpochDetected) controllerChannelManager.shutdown() @@ -150,4 +149,4 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { case _ => false } } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala old mode 100644 new mode 100755 index e10228555550d..e57c1dec2dee4 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -22,7 +22,6 @@ import kafka.utils._ import junit.framework.Assert._ import java.util.{Random, 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 @@ -30,8 +29,6 @@ import kafka.admin.AdminUtils import kafka.api.{PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest} import kafka.utils.TestUtils._ import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.utils.nonthreadsafe -import kafka.utils.threadsafe import org.junit.After import org.junit.Before import org.junit.Test @@ -42,26 +39,25 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { 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 config: Properties = createBrokerConfig(1) 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, "") + server = TestUtils.createServer(KafkaConfig.fromProps(config), time) + simpleConsumer = new SimpleConsumer("localhost", server.boundPort(), 1000000, 64*1024, "") } @After override def tearDown() { simpleConsumer.close server.shutdown - Utils.rm(logDir) + CoreUtils.rm(logDir) super.tearDown() } @@ -85,8 +81,8 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { AdminUtils.createTopic(zkClient, topic, 1, 1) val logManager = server.getLogManager - assertTrue("Log for partition [topic,0] should be created", - waitUntilTrue(() => logManager.getLog(TopicAndPartition(topic, part)).isDefined, 1000)) + waitUntilTrue(() => logManager.getLog(TopicAndPartition(topic, part)).isDefined, + "Log for partition [topic,0] should be created") val log = logManager.getLog(TopicAndPartition(topic, part)).get val message = new Message(Integer.toString(42).getBytes()) @@ -95,16 +91,16 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { log.flush() val offsets = server.apis.fetchOffsets(logManager, TopicAndPartition(topic, part), OffsetRequest.LatestTime, 10) - assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), offsets) + assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0), offsets) - waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), 1000) + waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected") val topicAndPartition = TopicAndPartition(topic, part) val offsetRequest = OffsetRequest( Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 10)), replicaId = 0) val consumerOffsets = simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets - assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), consumerOffsets) + assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0), consumerOffsets) // try to fetch using latest offset val fetchResponse = simpleConsumer.fetch( @@ -122,8 +118,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { val topic = topicPartition.split("-").head // setup brokers in zookeeper as owners of partitions for this test - AdminUtils.createTopic(zkClient, topic, 1, 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server)) var offsetChanged = false for(i <- 1 to 14) { @@ -159,14 +154,14 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs val offsets = server.apis.fetchOffsets(logManager, TopicAndPartition(topic, part), now, 10) - assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), offsets) + assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0L), offsets) - waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), 1000) + waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected") val topicAndPartition = TopicAndPartition(topic, part) val offsetRequest = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(now, 10)), replicaId = 0) val consumerOffsets = simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets - assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), consumerOffsets) + assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0L), consumerOffsets) } @Test @@ -189,7 +184,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(Seq(0L), offsets) - waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), 1000) + waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected") val topicAndPartition = TopicAndPartition(topic, part) val offsetRequest = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.EarliestTime, 10))) @@ -198,10 +193,10 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(Seq(0L), consumerOffsets) } - private def createBrokerConfig(nodeId: Int, port: Int): Properties = { + private def createBrokerConfig(nodeId: Int): Properties = { val props = new Properties props.put("broker.id", nodeId.toString) - props.put("port", port.toString) + props.put("port", TestUtils.RandomPort.toString()) props.put("log.dir", getLogDir.getAbsolutePath) props.put("log.flush.interval.messages", "1") props.put("enable.zookeeper", "false") diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala old mode 100644 new mode 100755 index 17a99f182f64c..7688f26fe42c4 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -16,80 +16,95 @@ */ package kafka.server -import org.scalatest.junit.JUnit3Suite -import org.junit.Assert._ -import java.io.File -import kafka.admin.AdminUtils +import java.util.Properties + import kafka.utils.TestUtils._ -import kafka.utils.IntEncoder -import kafka.utils.{Utils, TestUtils} +import kafka.utils.{IntEncoder, CoreUtils, TestUtils} import kafka.zk.ZooKeeperTestHarness import kafka.common._ -import kafka.producer.{ProducerConfig, KeyedMessage, Producer} +import kafka.producer.{KeyedMessage, Producer} +import kafka.serializer.StringEncoder + +import java.io.File + +import org.scalatest.junit.JUnit3Suite +import org.junit.Assert._ 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 replicaLagTimeMaxMs = 5000L + val replicaLagMaxMessages = 10L + val replicaFetchWaitMaxMs = 1000 + val replicaFetchMinBytes = 20 + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaFetchMinBytesProp, replicaFetchMinBytes.toString) + + var configs: Seq[KafkaConfig] = null val topic = "new-topic" val partitionId = 0 var server1: KafkaServer = null var server2: KafkaServer = null - val configProps1 = configs.head - val configProps2 = configs.last + def configProps1 = configs.head + def configProps2 = configs.last val message = "hello" var producer: Producer[Int, String] = null - var hwFile1: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps1.logDirs(0), ReplicaManager.HighWatermarkFilename)) - var hwFile2: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps2.logDirs(0), ReplicaManager.HighWatermarkFilename)) + def hwFile1: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps1.logDirs(0), ReplicaManager.HighWatermarkFilename)) + def 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)) - } + + // Some tests restart the brokers then produce more data. But since test brokers use random ports, we need + // to use a new producer that knows the new ports + def updateProducer() = { + if (producer != null) + producer.close() + producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[IntEncoder].getName) } - def testHWCheckpointNoFailuresSingleLogSegment { + override def setUp() { + super.setUp() + + configs = TestUtils.createBrokerConfigs(2, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps)) + // start both servers server1 = TestUtils.createServer(configProps1) server2 = TestUtils.createServer(configProps2) - servers ++= List(server1, server2) - - producer = new Producer[Int, String](new ProducerConfig(producerProps)) + servers = List(server1, server2) // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) + createTopic(zkClient, topic, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) - // wait until leader is elected - var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) - assertTrue("Leader should get elected", leader.isDefined) - // 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)) + // create the producer + updateProducer() + } + override def tearDown() { + producer.close() + for(server <- servers) { + server.shutdown() + CoreUtils.rm(server.config.logDirs(0)) + } + super.tearDown() + } + + def testHWCheckpointNoFailuresSingleLogSegment { val numMessages = 2L sendMessages(numMessages.toInt) // give some time for the follower 1 to record leader HW - assertTrue("Failed to update highwatermark for follower after 1000 ms", - TestUtils.waitUntilTrue(() => - server2.replicaManager.getReplica(topic, 0).get.highWatermark == numMessages, 10000)) + TestUtils.waitUntilTrue(() => + server2.replicaManager.getReplica(topic, 0).get.highWatermark.messageOffset == numMessages, + "Failed to update high watermark for follower after timeout") servers.foreach(server => server.replicaManager.checkpointHighWatermarks()) - producer.close() val leaderHW = hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L) assertEquals(numMessages, leaderHW) val followerHW = hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L) @@ -97,22 +112,8 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { } def testHWCheckpointWithFailuresSingleLogSegment { - // start both servers - server1 = TestUtils.createServer(configProps1) - server2 = TestUtils.createServer(configProps2) - servers ++= List(server1, server2) + var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) - producer = new Producer[Int, String](new ProducerConfig(producerProps)) - - // create topic with 1 partition, 2 replicas, one on each broker - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) - - // wait until leader is elected - var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) - assertTrue("Leader should get elected", leader.isDefined) - // 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.getOrElse(TopicAndPartition(topic, 0), 0L)) sendMessages(1) @@ -124,13 +125,15 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) // check if leader moves to the other server - leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader) + leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader) assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1)) // bring the preferred replica back server1.startup() + // Update producer with new server settings + updateProducer() - leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) + leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) 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)) @@ -140,50 +143,33 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) server2.startup() - leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader) + updateProducer() + leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader) assertTrue("Leader must remain on broker 0, in case of zookeeper session expiration it can move to broker 1", leader.isDefined && (leader.get == 0 || leader.get == 1)) sendMessages(1) hw += 1 - + // give some time for follower 1 to record leader HW of 60 - assertTrue("Failed to update highwatermark for follower after 1000 ms", TestUtils.waitUntilTrue(() => - server2.replicaManager.getReplica(topic, 0).get.highWatermark == hw, 2000)) + TestUtils.waitUntilTrue(() => + server2.replicaManager.getReplica(topic, 0).get.highWatermark.messageOffset == hw, + "Failed to update high watermark for follower after timeout") // shutdown the servers to allow the hw to be checkpointed servers.foreach(server => server.shutdown()) - producer.close() assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) } def testHWCheckpointNoFailuresMultipleLogSegments { - // start both servers - server1 = TestUtils.createServer(configs.head) - server2 = TestUtils.createServer(configs.last) - servers ++= List(server1, server2) - - 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 - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(0->Seq(0,1))) - - // wait until leader is elected - var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500) - assertTrue("Leader should get elected", leader.isDefined) - // 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)) sendMessages(20) - var hw = 20L + val hw = 20L // give some time for follower 1 to record leader HW of 600 - assertTrue("Failed to update highwatermark for follower after 1000 ms", TestUtils.waitUntilTrue(() => - server2.replicaManager.getReplica(topic, 0).get.highWatermark == hw, 1000)) + TestUtils.waitUntilTrue(() => + server2.replicaManager.getReplica(topic, 0).get.highWatermark.messageOffset == hw, + "Failed to update high watermark for follower after timeout") // shutdown the servers to allow the hw to be checkpointed servers.foreach(server => server.shutdown()) - producer.close() val leaderHW = hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L) assertEquals(hw, leaderHW) val followerHW = hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L) @@ -191,31 +177,15 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { } def testHWCheckpointWithFailuresMultipleLogSegments { - // start both servers - server1 = TestUtils.createServer(configs.head) - server2 = TestUtils.createServer(configs.last) - servers ++= List(server1, server2) - - 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 - 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) - assertTrue("Leader should get elected", leader.isDefined) - // 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)) + var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId) sendMessages(2) var hw = 2L - + // allow some time for the follower to get the leader HW - assertTrue("Failed to update highwatermark for follower after 1000 ms", TestUtils.waitUntilTrue(() => - server2.replicaManager.getReplica(topic, 0).get.highWatermark == hw, 1000)) + TestUtils.waitUntilTrue(() => + server2.replicaManager.getReplica(topic, 0).get.highWatermark.messageOffset == hw, + "Failed to update high watermark for follower after timeout") // kill the server hosting the preferred replica server1.shutdown() server2.shutdown() @@ -223,27 +193,29 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) server2.startup() + updateProducer() // check if leader moves to the other server - leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader) + leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader) assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1)) assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) // bring the preferred replica back server1.startup() + updateProducer() assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) sendMessages(2) hw += 2 - + // allow some time for the follower to get the leader HW - assertTrue("Failed to update highwatermark for follower after 1000 ms", TestUtils.waitUntilTrue(() => - server1.replicaManager.getReplica(topic, 0).get.highWatermark == hw, 1000)) + TestUtils.waitUntilTrue(() => + server1.replicaManager.getReplica(topic, 0).get.highWatermark.messageOffset == hw, + "Failed to update high watermark for follower after timeout") // shutdown the servers to allow the hw to be checkpointed servers.foreach(server => server.shutdown()) - producer.close() assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L)) assertEquals(hw, hwFile2.read.getOrElse(TopicAndPartition(topic, 0), 0L)) } diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala old mode 100644 new mode 100755 index 6a96d80daac57..39a68526c8b85 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.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 @@ -17,47 +17,61 @@ package kafka.server -import java.io.File -import kafka.utils._ -import junit.framework.Assert._ -import java.util.Properties +import kafka.api.{ConsumerMetadataRequest, OffsetCommitRequest, OffsetFetchRequest} import kafka.consumer.SimpleConsumer -import org.junit.{After, Before, Test} -import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} +import kafka.common.{OffsetMetadata, OffsetMetadataAndError, OffsetAndMetadata, ErrorMapping, TopicAndPartition} +import kafka.utils._ +import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness + +import org.junit.{After, Before, Test} import org.scalatest.junit.JUnit3Suite -import kafka.api.{OffsetCommitRequest, OffsetFetchRequest} -import kafka.utils.TestUtils._ -import kafka.common.{ErrorMapping, TopicAndPartition, OffsetMetadataAndError} + +import java.util.Properties +import java.io.File + import scala.util.Random -import kafka.admin.AdminUtils +import scala.collection._ + +import junit.framework.Assert._ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val random: Random = new Random() + val group = "test-group" + val retentionCheckInterval: Long = 100L 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 config: Properties = createBrokerConfig(1, zkConnect) + config.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") + config.setProperty(KafkaConfig.OffsetsRetentionCheckIntervalMsProp, retentionCheckInterval.toString) 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") + server = TestUtils.createServer(KafkaConfig.fromProps(config), time) + simpleConsumer = new SimpleConsumer("localhost", server.boundPort(), 1000000, 64*1024, "test-client") + val consumerMetadataRequest = ConsumerMetadataRequest(group) + Stream.continually { + val consumerMetadataResponse = simpleConsumer.send(consumerMetadataRequest) + consumerMetadataResponse.coordinatorOpt.isDefined + }.dropWhile(success => { + if (!success) Thread.sleep(1000) + !success + }) } @After override def tearDown() { simpleConsumer.close server.shutdown - Utils.rm(logDir) + CoreUtils.rm(logDir) super.tearDown() } @@ -69,39 +83,45 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val topicAndPartition = TopicAndPartition(topic, 0) val expectedReplicaAssignment = Map(0 -> List(1)) // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) - val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError(offset=42L))) + createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = Seq(server)) + + val commitRequest = OffsetCommitRequest(group, immutable.Map(topicAndPartition -> OffsetAndMetadata(offset = 42L))) val commitResponse = simpleConsumer.commitOffsets(commitRequest) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(topicAndPartition).get) // Fetch it and verify - val fetchRequest = OffsetFetchRequest("test-group", Seq(topicAndPartition)) + val fetchRequest = OffsetFetchRequest(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(OffsetMetadata.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( + val commitRequest1 = OffsetCommitRequest(group, immutable.Map(topicAndPartition -> OffsetAndMetadata( offset=100L, metadata="some metadata" ))) val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) - assertEquals(ErrorMapping.NoError, commitResponse1.requestInfo.get(topicAndPartition).get) + assertEquals(ErrorMapping.NoError, commitResponse1.commitStatus.get(topicAndPartition).get) // Fetch it and verify - val fetchRequest1 = OffsetFetchRequest("test-group", Seq(topicAndPartition)) + val fetchRequest1 = OffsetFetchRequest(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("some metadata", fetchResponse1.requestInfo.get(topicAndPartition).get.metadata) assertEquals(100L, fetchResponse1.requestInfo.get(topicAndPartition).get.offset) + // Fetch an unknown topic and verify + val unknownTopicAndPartition = TopicAndPartition("unknownTopic", 0) + val fetchRequest2 = OffsetFetchRequest(group, Seq(unknownTopicAndPartition)) + val fetchResponse2 = simpleConsumer.fetchOffsets(fetchRequest2) + + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse2.requestInfo.get(unknownTopicAndPartition).get) + assertEquals(1, fetchResponse2.requestInfo.size) } @Test @@ -109,105 +129,182 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val topic1 = "topic-1" val topic2 = "topic-2" val topic3 = "topic-3" - val topic4 = "topic-4" + val topic4 = "topic-4" // Topic that group never consumes + val topic5 = "topic-5" // Non-existent topic - val expectedReplicaAssignment = Map(0 -> List(1)) - // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic1, expectedReplicaAssignment) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic2, expectedReplicaAssignment) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic3, expectedReplicaAssignment) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic4, expectedReplicaAssignment) - var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic4, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - - 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) + createTopic(zkClient, topic1, servers = Seq(server), numPartitions = 1) + createTopic(zkClient, topic2, servers = Seq(server), numPartitions = 2) + createTopic(zkClient, topic3, servers = Seq(server), numPartitions = 1) + createTopic(zkClient, topic4, servers = Seq(server), numPartitions = 1) + + val commitRequest = OffsetCommitRequest("test-group", immutable.Map( + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="metadata one"), + TopicAndPartition(topic2, 0) -> OffsetAndMetadata(offset=43L, metadata="metadata two"), + TopicAndPartition(topic3, 0) -> OffsetAndMetadata(offset=44L, metadata="metadata three"), + TopicAndPartition(topic2, 1) -> OffsetAndMetadata(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) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic1, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic2, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic3, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic2, 1)).get) - val fetchRequest = OffsetFetchRequest("test-group", Seq( + val fetchRequest = OffsetFetchRequest(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 + TopicAndPartition(topic4, 0), // An unused topic + TopicAndPartition(topic5, 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(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) + + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) + + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get) + + 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(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) + assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) + assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) + assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 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) + + assertEquals(OffsetMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) + assertEquals(OffsetMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) + assertEquals(OffsetMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.offset) } @Test def testLargeMetadataPayload() { val topicAndPartition = TopicAndPartition("large-metadata", 0) val expectedReplicaAssignment = Map(0 -> List(1)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topicAndPartition.topic, expectedReplicaAssignment) - var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicAndPartition.topic, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) + createTopic(zkClient, topicAndPartition.topic, partitionReplicaAssignment = expectedReplicaAssignment, + servers = Seq(server)) - val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( + val commitRequest = OffsetCommitRequest("test-group", immutable.Map(topicAndPartition -> OffsetAndMetadata( offset=42L, metadata=random.nextString(server.config.offsetMetadataMaxSize) ))) val commitResponse = simpleConsumer.commitOffsets(commitRequest) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(topicAndPartition).get) - val commitRequest1 = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( + val commitRequest1 = OffsetCommitRequest(group, immutable.Map(topicAndPartition -> OffsetAndMetadata( offset=42L, metadata=random.nextString(server.config.offsetMetadataMaxSize + 1) ))) val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) - assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.requestInfo.get(topicAndPartition).get) + assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.commitStatus.get(topicAndPartition).get) + } + + @Test + def testOffsetExpiration() { + // set up topic partition + val topic = "topic" + val topicPartition = TopicAndPartition(topic, 0) + createTopic(zkClient, topic, servers = Seq(server), numPartitions = 1) + + val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition(topic, 0))) + + // v0 version commit request + // committed offset should not exist with fetch version 1 since it was stored in ZK + val commitRequest0 = OffsetCommitRequest( + groupId = group, + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(1L, "metadata")), + versionId = 0 + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest0).commitStatus.get(topicPartition).get) + assertEquals(-1L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) + + // committed offset should exist with fetch version 0 + assertEquals(1L, simpleConsumer.fetchOffsets(OffsetFetchRequest(group, Seq(TopicAndPartition(topic, 0)), versionId = 0)).requestInfo.get(topicPartition).get.offset) + + + // v1 version commit request with commit timestamp set to -1 + // committed offset should not expire + val commitRequest1 = OffsetCommitRequest( + groupId = group, + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(2L, "metadata", -1L)), + versionId = 1 + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest1).commitStatus.get(topicPartition).get) + Thread.sleep(retentionCheckInterval * 2) + assertEquals(2L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) + + // v1 version commit request with commit timestamp set to now - two days + // committed offset should expire + val commitRequest2 = OffsetCommitRequest( + groupId = group, + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(3L, "metadata", SystemTime.milliseconds - 2*24*60*60*1000L)), + versionId = 1 + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest2).commitStatus.get(topicPartition).get) + Thread.sleep(retentionCheckInterval * 2) + assertEquals(-1L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) + + // v2 version commit request with retention time set to 1 hour + // committed offset should not expire + val commitRequest3 = OffsetCommitRequest( + groupId = group, + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(4L, "metadata", -1L)), + versionId = 2, + retentionMs = 1000 * 60 * 60L + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest3).commitStatus.get(topicPartition).get) + Thread.sleep(retentionCheckInterval * 2) + assertEquals(4L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) + + // v2 version commit request with retention time set to 0 second + // committed offset should expire + val commitRequest4 = OffsetCommitRequest( + groupId = "test-group", + requestInfo = immutable.Map(TopicAndPartition(topic, 0) -> OffsetAndMetadata(5L, "metadata", -1L)), + versionId = 2, + retentionMs = 0L + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest4).commitStatus.get(topicPartition).get) + Thread.sleep(retentionCheckInterval * 2) + assertEquals(-1L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) } @Test - def testNullMetadata() { - val topicAndPartition = TopicAndPartition("null-metadata", 0) + def testNonExistingTopicOffsetCommit() { + val topic1 = "topicDoesNotExists" + val topic2 = "topic-2" + + createTopic(zkClient, topic2, servers = Seq(server), numPartitions = 1) + + // Commit an offset val expectedReplicaAssignment = Map(0 -> List(1)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topicAndPartition.topic, expectedReplicaAssignment) - var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicAndPartition.topic, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( - offset=42L, - metadata=null - ))) + val commitRequest = OffsetCommitRequest(group, immutable.Map( + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L), + TopicAndPartition(topic2, 0) -> OffsetAndMetadata(offset=42L) + )) val commitResponse = simpleConsumer.commitOffsets(commitRequest) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get) + + assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, commitResponse.commitStatus.get(TopicAndPartition(topic1, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic2, 0)).get) } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala index dd85c71816715..a3a03db88c4c3 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala @@ -22,21 +22,19 @@ import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import kafka.producer.KeyedMessage import kafka.serializer.StringEncoder -import kafka.admin.AdminUtils -import kafka.utils.TestUtils -import junit.framework.Assert._ +import kafka.utils.{TestUtils} import kafka.common._ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { - val props = createBrokerConfigs(2) - val configs = props.map(p => new KafkaConfig(p)) var brokers: Seq[KafkaServer] = null val topic1 = "foo" val topic2 = "bar" override def setUp() { super.setUp() - brokers = configs.map(config => TestUtils.createServer(config)) + brokers = createBrokerConfigs(2, zkConnect, false) + .map(KafkaConfig.fromProps) + .map(config => TestUtils.createServer(config)) } override def tearDown() { @@ -51,14 +49,13 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { // create a topic and partition and await leadership for (topic <- List(topic1,topic2)) { - AdminUtils.createTopic(zkClient, topic, 1, 2) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 2, servers = brokers) } // send test messages to leader - val producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromConfigs(configs), - new StringEncoder(), - new StringEncoder()) + val producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromServers(brokers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName) val messages = testMessageList1.map(m => new KeyedMessage(topic1, m, m)) ++ testMessageList2.map(m => new KeyedMessage(topic2, m, m)) producer.send(messages:_*) producer.close() @@ -68,11 +65,12 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { for (topic <- List(topic1, topic2)) { 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(topicAndPart).get.logEndOffset) } + result = result && expectedOffset > 0 && brokers.forall { item => + (expectedOffset == item.getLogManager().getLog(topicAndPart).get.logEndOffset) + } } result } - assertTrue("Broker logs should be identical", waitUntilTrue(logsMatch, 6000)) + waitUntilTrue(logsMatch, "Broker logs should be identical") } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala new file mode 100644 index 0000000000000..00d59337a99ac --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -0,0 +1,92 @@ +/** + * 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.api.{ProducerResponseStatus, SerializationTestUtils, ProducerRequest} +import kafka.common.TopicAndPartition +import kafka.utils.{MockScheduler, MockTime, TestUtils} + +import java.util.concurrent.atomic.AtomicBoolean +import java.io.File + +import org.apache.kafka.common.protocol.Errors +import org.easymock.EasyMock +import org.I0Itec.zkclient.ZkClient +import org.scalatest.junit.JUnit3Suite +import org.junit.Test + +import scala.collection.Map + +class ReplicaManagerTest extends JUnit3Suite { + + val topic = "test-topic" + + @Test + def testHighWaterMarkDirectoryMapping() { + val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) + val config = KafkaConfig.fromProps(props) + val zkClient = EasyMock.createMock(classOf[ZkClient]) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) + val time: MockTime = new MockTime() + val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) + val partition = rm.getOrCreatePartition(topic, 1) + partition.getOrCreateReplica(1) + rm.checkpointHighWatermarks() + + // shutdown the replica manager upon test completion + rm.shutdown(false) + } + + @Test + def testHighwaterMarkRelativeDirectoryMapping() { + val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) + props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) + val config = KafkaConfig.fromProps(props) + val zkClient = EasyMock.createMock(classOf[ZkClient]) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) + val time: MockTime = new MockTime() + val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) + val partition = rm.getOrCreatePartition(topic, 1) + partition.getOrCreateReplica(1) + rm.checkpointHighWatermarks() + + // shutdown the replica manager upon test completion + rm.shutdown(false) + } + + @Test + def testIllegalRequiredAcks() { + val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) + val config = KafkaConfig.fromProps(props) + val zkClient = EasyMock.createMock(classOf[ZkClient]) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) + val time: MockTime = new MockTime() + val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) + val produceRequest = new ProducerRequest(1, "client 1", 3, 1000, SerializationTestUtils.topicDataProducerRequest) + def callback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) = { + assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS.code) + } + + rm.appendMessages(timeout = 0, requiredAcks = 3, internalTopicsAllowed = false, messagesPerPartition = produceRequest.data, responseCallback = callback) + + rm.shutdown(false); + + TestUtils.verifyNonDaemonThreadsStatus + + } +} diff --git a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala deleted file mode 100644 index 4f61f8469df99..0000000000000 --- a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala +++ /dev/null @@ -1,94 +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 scala.collection._ -import org.junit.Test -import junit.framework.Assert._ -import kafka.message._ -import kafka.api._ -import kafka.utils.TestUtils -import org.scalatest.junit.JUnit3Suite - - -class RequestPurgatoryTest extends JUnit3Suite { - - val producerRequest1 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello1".getBytes))) - val producerRequest2 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello2".getBytes))) - var purgatory: MockRequestPurgatory = null - - override def setUp() { - super.setUp() - purgatory = new MockRequestPurgatory() - } - - override def tearDown() { - purgatory.shutdown() - super.tearDown() - } - - @Test - def testRequestSatisfaction() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r2 = new DelayedRequest(Array("test2"), null, 100000L) - assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.update("test1", producerRequest1).size) - purgatory.watch(r1) - assertEquals("Still nothing satisfied", 0, purgatory.update("test1", producerRequest1).size) - purgatory.watch(r2) - assertEquals("Still nothing satisfied", 0, purgatory.update("test2", producerRequest2).size) - purgatory.satisfied += r1 - assertEquals("r1 satisfied", mutable.ArrayBuffer(r1), purgatory.update("test1", producerRequest1)) - assertEquals("Nothing satisfied", 0, purgatory.update("test1", producerRequest2).size) - purgatory.satisfied += r2 - assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2", producerRequest2)) - assertEquals("Nothing satisfied", 0, purgatory.update("test2", producerRequest2).size) - } - - @Test - def testRequestExpiry() { - val expiration = 20L - val r1 = new DelayedRequest(Array("test1"), null, expiration) - val r2 = new DelayedRequest(Array("test1"), null, 200000L) - val start = System.currentTimeMillis - purgatory.watch(r1) - purgatory.watch(r2) - purgatory.awaitExpiration(r1) - val elapsed = System.currentTimeMillis - start - assertTrue("r1 expired", purgatory.expired.contains(r1)) - assertTrue("r2 hasn't expired", !purgatory.expired.contains(r2)) - assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) - } - - class MockRequestPurgatory extends RequestPurgatory[DelayedRequest, ProducerRequest] { - val satisfied = mutable.Set[DelayedRequest]() - val expired = mutable.Set[DelayedRequest]() - def awaitExpiration(delayed: DelayedRequest) = { - delayed synchronized { - delayed.wait() - } - } - def checkSatisfied(request: ProducerRequest, delayed: DelayedRequest): Boolean = satisfied.contains(delayed) - def expire(delayed: DelayedRequest) { - expired += delayed - delayed synchronized { - delayed.notify() - } - } - } - -} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala new file mode 100755 index 0000000000000..12269cde06aab --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala @@ -0,0 +1,136 @@ +/** + * 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 kafka.zk.ZooKeeperTestHarness +import kafka.utils.{TestUtils, CoreUtils} +import org.junit.Test +import org.scalatest.junit.JUnit3Suite +import junit.framework.Assert._ +import java.io.File + +class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { + var props1: Properties = null + var config1: KafkaConfig = null + var props2: Properties = null + var config2: KafkaConfig = null + val brokerMetaPropsFile = "meta.properties" + + override def setUp() { + super.setUp() + props1 = TestUtils.createBrokerConfig(-1, zkConnect) + config1 = KafkaConfig.fromProps(props1) + props2 = TestUtils.createBrokerConfig(0, zkConnect) + config2 = KafkaConfig.fromProps(props2) + } + + @Test + def testAutoGenerateBrokerId() { + var server1 = new KafkaServer(config1) + server1.startup() + server1.shutdown() + assertTrue(verifyBrokerMetadata(config1.logDirs, 1001)) + // restart the server check to see if it uses the brokerId generated previously + server1 = new KafkaServer(config1) + server1.startup() + assertEquals(server1.config.brokerId, 1001) + server1.shutdown() + CoreUtils.rm(server1.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus + } + + @Test + def testUserConfigAndGeneratedBrokerId() { + // start the server with broker.id as part of config + val server1 = new KafkaServer(config1) + val server2 = new KafkaServer(config2) + val props3 = TestUtils.createBrokerConfig(-1, zkConnect) + val config3 = KafkaConfig.fromProps(props3) + val server3 = new KafkaServer(config3) + server1.startup() + assertEquals(server1.config.brokerId,1001) + server2.startup() + assertEquals(server2.config.brokerId,0) + server3.startup() + assertEquals(server3.config.brokerId,1002) + server1.shutdown() + server2.shutdown() + server3.shutdown() + assertTrue(verifyBrokerMetadata(server1.config.logDirs,1001)) + assertTrue(verifyBrokerMetadata(server2.config.logDirs,0)) + assertTrue(verifyBrokerMetadata(server3.config.logDirs,1002)) + CoreUtils.rm(server1.config.logDirs) + CoreUtils.rm(server2.config.logDirs) + CoreUtils.rm(server3.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus + } + + @Test + def testMultipleLogDirsMetaProps() { + // add multiple logDirs and check if the generate brokerId is stored in all of them + val logDirs = props1.getProperty("log.dir")+ "," + TestUtils.tempDir().getAbsolutePath + + "," + TestUtils.tempDir().getAbsolutePath + props1.setProperty("log.dir",logDirs) + config1 = KafkaConfig.fromProps(props1) + var server1 = new KafkaServer(config1) + server1.startup() + server1.shutdown() + assertTrue(verifyBrokerMetadata(config1.logDirs, 1001)) + // addition to log.dirs after generation of a broker.id from zk should be copied over + val newLogDirs = props1.getProperty("log.dir") + "," + TestUtils.tempDir().getAbsolutePath + props1.setProperty("log.dir",newLogDirs) + config1 = KafkaConfig.fromProps(props1) + server1 = new KafkaServer(config1) + server1.startup() + server1.shutdown() + assertTrue(verifyBrokerMetadata(config1.logDirs, 1001)) + CoreUtils.rm(server1.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus + } + + @Test + def testConsistentBrokerIdFromUserConfigAndMetaProps() { + // check if configured brokerId and stored brokerId are equal or throw InconsistentBrokerException + var server1 = new KafkaServer(config1) //auto generate broker Id + server1.startup() + server1.shutdown() + server1 = new KafkaServer(config2) // user specified broker id + try { + server1.startup() + } catch { + case e: kafka.common.InconsistentBrokerIdException => //success + } + server1.shutdown() + CoreUtils.rm(server1.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus + } + + def verifyBrokerMetadata(logDirs: Seq[String], brokerId: Int): Boolean = { + for(logDir <- logDirs) { + val brokerMetadataOpt = (new BrokerMetadataCheckpoint( + new File(logDir + File.separator + brokerMetaPropsFile))).read() + brokerMetadataOpt match { + case Some(brokerMetadata: BrokerMetadata) => + if (brokerMetadata.brokerId != brokerId) return false + case _ => return false + } + } + true + } +} diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala old mode 100644 new mode 100755 index 20fe93e623319..95534e36c2902 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -16,41 +16,44 @@ */ package kafka.server -import java.io.File -import kafka.consumer.SimpleConsumer -import org.junit.Test -import junit.framework.Assert._ -import kafka.message.ByteBufferMessageSet -import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness +import kafka.consumer.SimpleConsumer import kafka.producer._ -import kafka.utils.IntEncoder +import kafka.utils.{IntEncoder, TestUtils, CoreUtils} import kafka.utils.TestUtils._ -import kafka.admin.AdminUtils import kafka.api.FetchRequestBuilder -import kafka.utils.{TestUtils, Utils} +import kafka.message.ByteBufferMessageSet +import kafka.serializer.StringEncoder -class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { - val port = TestUtils.choosePort - val props = TestUtils.createBrokerConfig(0, port) - val config = new KafkaConfig(props) +import java.io.File +import org.junit.Test +import org.scalatest.junit.JUnit3Suite +import junit.framework.Assert._ + +class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { + var config: KafkaConfig = null val host = "localhost" val topic = "test" val sent1 = List("hello", "there") val sent2 = List("more", "messages") + override def setUp(): Unit = { + super.setUp() + val props = TestUtils.createBrokerConfig(0, zkConnect) + config = KafkaConfig.fromProps(props) + } + @Test def testCleanShutdown() { var server = new KafkaServer(config) server.startup() - val producerConfig = getProducerConfig(TestUtils.getBrokerListStrFromConfigs(Seq(config))) - producerConfig.put("key.serializer.class", classOf[IntEncoder].getName.toString) - var producer = new Producer[Int, String](new ProducerConfig(producerConfig)) + var producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(Seq(server)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[IntEncoder].getName) // create topic - AdminUtils.createTopic(zkClient, topic, 1, 1) - TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) + createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server)) // send some messages producer.send(sent1.map(m => new KeyedMessage[Int, String](topic, 0, m)):_*) @@ -69,17 +72,19 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { server.startup() // wait for the broker to receive the update metadata request after startup - TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) + TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0) - producer = new Producer[Int, String](new ProducerConfig(producerConfig)) - val consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "") + producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(Seq(server)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[IntEncoder].getName) + val consumer = new SimpleConsumer(host, server.boundPort(), 1000000, 64*1024, "") var fetchedMessage: ByteBufferMessageSet = null while(fetchedMessage == null || fetchedMessage.validBytes == 0) { val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).maxWait(0).build()) fetchedMessage = fetched.messageSet(topic, 0) } - assertEquals(sent1, fetchedMessage.map(m => Utils.readString(m.message.payload))) + assertEquals(sent1, fetchedMessage.map(m => TestUtils.readString(m.message.payload))) val newOffset = fetchedMessage.last.nextOffset // send some more messages @@ -90,11 +95,81 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, newOffset, 10000).build()) fetchedMessage = fetched.messageSet(topic, 0) } - assertEquals(sent2, fetchedMessage.map(m => Utils.readString(m.message.payload))) + assertEquals(sent2, fetchedMessage.map(m => TestUtils.readString(m.message.payload))) consumer.close() producer.close() server.shutdown() - Utils.rm(server.config.logDirs) + CoreUtils.rm(server.config.logDirs) + verifyNonDaemonThreadsStatus + } + + @Test + def testCleanShutdownWithDeleteTopicEnabled() { + val newProps = TestUtils.createBrokerConfig(0, zkConnect) + newProps.setProperty("delete.topic.enable", "true") + val newConfig = KafkaConfig.fromProps(newProps) + val server = new KafkaServer(newConfig) + server.startup() + server.shutdown() + server.awaitShutdown() + CoreUtils.rm(server.config.logDirs) + verifyNonDaemonThreadsStatus + } + + @Test + def testCleanShutdownAfterFailedStartup() { + val newProps = TestUtils.createBrokerConfig(0, zkConnect) + newProps.setProperty("zookeeper.connect", "fakehostthatwontresolve:65535") + val newConfig = KafkaConfig.fromProps(newProps) + val server = new KafkaServer(newConfig) + try { + server.startup() + fail("Expected KafkaServer setup to fail, throw exception") + } + catch { + // Try to clean up carefully without hanging even if the test fails. This means trying to accurately + // identify the correct exception, making sure the server was shutdown, and cleaning up if anything + // goes wrong so that awaitShutdown doesn't hang + case e: org.I0Itec.zkclient.exception.ZkException => + assertEquals(NotRunning.state, server.brokerState.currentState) + case e: Throwable => + fail("Expected ZkException during Kafka server starting up but caught a different exception %s".format(e.toString)) + } + finally { + if (server.brokerState.currentState != NotRunning.state) + server.shutdown() + server.awaitShutdown() + } + CoreUtils.rm(server.config.logDirs) + verifyNonDaemonThreadsStatus + } + + private[this] def isNonDaemonKafkaThread(t: Thread): Boolean = { + val threadName = Option(t.getClass.getCanonicalName) + .getOrElse(t.getClass.getName()) + .toLowerCase + + !t.isDaemon && t.isAlive && threadName.startsWith("kafka") + } + + def verifyNonDaemonThreadsStatus() { + assertEquals(0, Thread.getAllStackTraces.keySet().toArray + .map{ _.asInstanceOf[Thread] } + .count(isNonDaemonKafkaThread)) + } + + def testConsecutiveShutdown(){ + val server = new KafkaServer(config) + try { + server.startup() + server.shutdown() + server.awaitShutdown() + server.shutdown() + assertTrue(true) + } + catch{ + case ex: Throwable => fail() + } } } diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala new file mode 100755 index 0000000000000..60e10b3d5adda --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -0,0 +1,69 @@ +/** + * 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.utils.ZkUtils +import kafka.utils.CoreUtils +import kafka.utils.TestUtils + +import kafka.zk.ZooKeeperTestHarness +import junit.framework.Assert._ + +class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { + + def testBrokerCreatesZKChroot { + val brokerId = 0 + val zookeeperChroot = "/kafka-chroot-for-unittest" + val props = TestUtils.createBrokerConfig(brokerId, zkConnect) + val zooKeeperConnect = props.get("zookeeper.connect") + props.put("zookeeper.connect", zooKeeperConnect + zookeeperChroot) + val server = TestUtils.createServer(KafkaConfig.fromProps(props)) + + val pathExists = ZkUtils.pathExists(zkClient, zookeeperChroot) + assertTrue(pathExists) + + server.shutdown() + CoreUtils.rm(server.config.logDirs) + } + + def testConflictBrokerRegistration { + // Try starting a broker with the a conflicting broker id. + // This shouldn't affect the existing broker registration. + + val brokerId = 0 + val props1 = TestUtils.createBrokerConfig(brokerId, zkConnect) + val server1 = TestUtils.createServer(KafkaConfig.fromProps(props1)) + val brokerRegistration = ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 + + val props2 = TestUtils.createBrokerConfig(brokerId, zkConnect) + try { + TestUtils.createServer(KafkaConfig.fromProps(props2)) + fail("Registering a broker with a conflicting id should fail") + } catch { + case e : RuntimeException => + // this is expected + } + + // broker registration shouldn't change + assertEquals(brokerRegistration, ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1) + + server1.shutdown() + CoreUtils.rm(server1.config.logDirs) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 1317b4c3c60b8..09a0961f73969 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -16,197 +16,138 @@ */ package kafka.server -import kafka.cluster.{Partition, Replica} +import kafka.api._ +import kafka.utils._ +import kafka.cluster.Replica +import kafka.common.TopicAndPartition import kafka.log.Log -import kafka.message.{ByteBufferMessageSet, Message} -import kafka.network.RequestChannel -import kafka.utils.{ZkUtils, Time, TestUtils, MockTime} +import kafka.message.{MessageSet, ByteBufferMessageSet, Message} + +import scala.Some +import java.util.{Properties, Collections} +import java.util.concurrent.atomic.AtomicBoolean +import collection.JavaConversions._ + import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient import org.scalatest.junit.JUnit3Suite -import kafka.api._ -import scala.Some -import kafka.common.TopicAndPartition +import junit.framework.Assert._ 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 + val replicaLagTimeMaxMs = 100L + val replicaFetchWaitMaxMs = 100 + val replicaLagMaxMessages = 10L - /** - * 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 - * but is still in ISR (hasn't yet expired from ISR). - * - * When a normal consumer fetches data, it only should only see data upto the HW of the leader, - * in this case up an offset of "5". - */ - def testNonReplicaSeesHwWhenFetching() { - /* setup */ - val time = new MockTime - val leo = 20 - val hw = 5 - val fetchSize = 100 - val messages = new Message("test-message".getBytes()) + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) - // create nice mock since we don't particularly care about zkclient calls - val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) - EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) - EasyMock.replay(zkClient) + val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, overridingProps)) - 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) + // set the replica manager with the partition + val time = new MockTime + val leaderLEO = 20L + val followerLEO = 15L + val partitionHW = 5 - val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(TopicAndPartition(topic, partitionId))).andReturn(Some(log)).anyTimes() - EasyMock.replay(logManager) + val fetchSize = 100 + val messagesToHW = new Message("messageToHW".getBytes()) + val messagesToLEO = new Message("messageToLEO".getBytes()) - val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(configs.head) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) - EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) - EasyMock.replay(replicaManager) - - val partition = getPartitionWithAllReplicasInISR(topic, partitionId, time, configs.head.brokerId, log, hw, replicaManager) - partition.getReplica(configs(1).brokerId).get.logEndOffset = leo - 5L - - EasyMock.reset(replicaManager) - EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() - EasyMock.replay(replicaManager) - - val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) - - // start a request channel with 2 processors and a queue size of 5 (this is more or less arbitrary) - // don't provide replica or leader callbacks since they will not be tested here - val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) - val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) - apis.metadataCache.put(TopicAndPartition(topic, partitionId), partitionStateInfo) - EasyMock.replay(partitionStateInfo) - // 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() - .replicaId(Request.OrdinaryConsumerId) - .addFetch(topic, partitionId, 0, fetchSize) - .build() - val goodFetchBB = TestUtils.createRequestByteBuffer(goodFetch) - - // send the request - apis.handleFetchRequest(new RequestChannel.Request(processor=1, requestKey=5, buffer=goodFetchBB, startTimeMs=1)) - - // make sure the log only reads bytes between 0->HW (5) - EasyMock.verify(log) - } + val topic = "test-topic" + val partitionId = 0 + val topicAndPartition = TopicAndPartition(topic, partitionId) - /** - * The scenario for this test is that there is one topic, "test-topic", on 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 - * but is still in ISR (hasn't yet expired from ISR). - * - * When the follower from broker "1" fetches data, it should see data upto the log end offset ("20") - */ - def testReplicaSeesLeoWhenFetching() { - /* setup */ - val time = new MockTime - val leo = 20 - val hw = 5 + val fetchInfo = Collections.singletonMap(topicAndPartition, PartitionFetchInfo(0, fetchSize)).toMap - val messages = new Message("test-message".getBytes()) + var replicaManager: ReplicaManager = null - val followerReplicaId = configs(1).brokerId - val followerLEO = 15 + override def setUp() { + super.setUp() + // create nice mock since we don't particularly care about zkclient calls val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) - EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) EasyMock.replay(zkClient) - val log = EasyMock.createMock(classOf[kafka.log.Log]) - EasyMock.expect(log.logEndOffset).andReturn(leo).anyTimes() - EasyMock.expect(log.read(followerLEO, Integer.MAX_VALUE, None)).andReturn(new ByteBufferMessageSet(messages)) + // create nice mock since we don't particularly care about scheduler calls + val scheduler = EasyMock.createNiceMock(classOf[KafkaScheduler]) + EasyMock.replay(scheduler) + + // create the log which takes read with either HW max offset or none max offset + val log = EasyMock.createMock(classOf[Log]) + EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes() + EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes() + EasyMock.expect(log.read(0, fetchSize, Some(partitionHW))).andReturn( + new FetchDataInfo( + new LogOffsetMetadata(0L, 0L, 0), + new ByteBufferMessageSet(messagesToHW) + )).anyTimes() + EasyMock.expect(log.read(0, fetchSize, None)).andReturn( + new FetchDataInfo( + new LogOffsetMetadata(0L, 0L, 0), + new ByteBufferMessageSet(messagesToLEO) + )).anyTimes() EasyMock.replay(log) + // create the log manager that is aware of this mock log val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(TopicAndPartition(topic, 0))).andReturn(Some(log)).anyTimes() + EasyMock.expect(logManager.getLog(topicAndPartition)).andReturn(Some(log)).anyTimes() EasyMock.replay(logManager) - val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(configs.head) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) - EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) - EasyMock.replay(replicaManager) - - val partition = getPartitionWithAllReplicasInISR(topic, partitionId, time, configs.head.brokerId, log, hw, replicaManager) - partition.getReplica(followerReplicaId).get.logEndOffset = followerLEO.asInstanceOf[Long] - - EasyMock.reset(replicaManager) - EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.expect(replicaManager.recordFollowerPosition(topic, partitionId, followerReplicaId, followerLEO)) - EasyMock.expect(replicaManager.getReplica(topic, partitionId, followerReplicaId)).andReturn(partition.inSyncReplicas.find(_.brokerId == configs(1).brokerId)) - EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() - EasyMock.replay(replicaManager) - - val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) - - val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) - val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) - apis.metadataCache.put(TopicAndPartition(topic, partitionId), partitionStateInfo) - EasyMock.replay(partitionStateInfo) - - /** - * This fetch, coming from a replica, requests all data at offset "15". Because the request is coming - * from a follower, the leader should oblige and read beyond the HW. - */ - val bigFetch = new FetchRequestBuilder() - .replicaId(followerReplicaId) - .addFetch(topic, partitionId, followerLEO, Integer.MAX_VALUE) - .build() - - val fetchRequestBB = TestUtils.createRequestByteBuffer(bigFetch) - - // send the request - apis.handleFetchRequest(new RequestChannel.Request(processor=0, requestKey=5, buffer=fetchRequestBB, startTimeMs=1)) - - /** - * Make sure the log satisfies the fetch from a follower by reading data beyond the HW, mainly all bytes after - * an offset of 15 - */ - EasyMock.verify(log) - } + // create the replica manager + replicaManager = new ReplicaManager(configs.head, time, zkClient, scheduler, logManager, new AtomicBoolean(false)) + + // add the partition with two replicas, both in ISR + val partition = replicaManager.getOrCreatePartition(topic, partitionId) + + // create the leader replica with the local log + val leaderReplica = new Replica(configs(0).brokerId, partition, time, 0, Some(log)) + leaderReplica.highWatermark = new LogOffsetMetadata(partitionHW) + partition.leaderReplicaIdOpt = Some(leaderReplica.brokerId) - private def getPartitionWithAllReplicasInISR(topic: String, partitionId: Int, time: Time, leaderId: Int, - localLog: Log, leaderHW: Long, replicaManager: ReplicaManager): Partition = { - val partition = new Partition(topic, partitionId, 2, time, replicaManager) - val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog)) + // create the follower replica with defined log end offset + val followerReplica= new Replica(configs(1).brokerId, partition, time) + val leo = new LogOffsetMetadata(followerLEO, 0L, followerLEO.toInt) + followerReplica.updateLogReadResult(new LogReadResult(FetchDataInfo(leo, MessageSet.Empty), -1L, -1, true)) - val allReplicas = getFollowerReplicas(partition, leaderId, time) :+ leaderReplica + // add both of them to ISR + val allReplicas = List(leaderReplica, followerReplica) allReplicas.foreach(partition.addReplicaIfNotExists(_)) - // set in sync replicas for this partition to all the assigned replicas partition.inSyncReplicas = allReplicas.toSet - // set the leader and its hw and the hw update time - partition.leaderReplicaIdOpt = Some(leaderId) - leaderReplica.highWatermark = leaderHW - partition } - private def getFollowerReplicas(partition: Partition, leaderId: Int, time: Time): Seq[Replica] = { - configs.filter(_.brokerId != leaderId).map { config => - new Replica(config.brokerId, partition, time) - } + override def tearDown() { + replicaManager.shutdown(false) + super.tearDown() } + /** + * The scenario for this test is that there is one topic that has one partition + * with one leader replica on broker "0" and one follower replica on broker "1" + * inside the replica manager's metadata. + * + * 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 but is still in ISR (hasn't yet expired from ISR). + * + * When a fetch operation with read committed data turned on is received, the replica manager + * should only return data up to the HW of the partition; when a fetch operation with read + * committed data turned off is received, the replica manager could return data up to the LEO + * of the local leader replica's log. + * + * This test also verifies counts of fetch requests recorded by the ReplicaManager + */ + def testReadFromLog() { + val initialTopicCount = BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count(); + val initialAllTopicsCount = BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count(); + + assertEquals("Reading committed data should return messages only up to high watermark", messagesToHW, + replicaManager.readFromLocalLog(true, true, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) + assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO, + replicaManager.readFromLocalLog(true, false, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) + + assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count()); + assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count()); + } } diff --git a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala new file mode 100644 index 0000000000000..fd8cf7bf04c5f --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala @@ -0,0 +1,98 @@ +/** + * 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.concurrent.TimeUnit + +import junit.framework.Assert._ +import org.junit.Test + +class ByteBoundedBlockingQueueTest { + val sizeFunction = (a: String) => a.length + val queue = new ByteBoundedBlockingQueue[String](5, 15, Some(sizeFunction)) + + @Test + def testByteBoundedBlockingQueue() { + assertEquals(5, queue.remainingSize) + assertEquals(15, queue.remainingByteSize) + + //offer a message whose size is smaller than remaining capacity + val m0 = new String("0123456789") + assertEquals(true, queue.offer(m0)) + assertEquals(1, queue.size()) + assertEquals(10, queue.byteSize()) + assertEquals(4, queue.remainingSize) + assertEquals(5, queue.remainingByteSize) + + // offer a message where remaining capacity < message size < capacity limit + val m1 = new String("1234567890") + assertEquals(true, queue.offer(m1)) + assertEquals(2, queue.size()) + assertEquals(20, queue.byteSize()) + assertEquals(3, queue.remainingSize) + assertEquals(0, queue.remainingByteSize) + + // offer a message using timeout, should fail because no space is left + val m2 = new String("2345678901") + assertEquals(false, queue.offer(m2, 10, TimeUnit.MILLISECONDS)) + assertEquals(2, queue.size()) + assertEquals(20, queue.byteSize()) + assertEquals(3, queue.remainingSize) + assertEquals(0, queue.remainingByteSize) + + // take an element out of the queue + assertEquals("0123456789", queue.take()) + assertEquals(1, queue.size()) + assertEquals(10, queue.byteSize()) + assertEquals(4, queue.remainingSize) + assertEquals(5, queue.remainingByteSize) + + // add 5 small elements into the queue, first 4 should succeed, the 5th one should fail + // test put() + assertEquals(true, queue.put("a")) + assertEquals(true, queue.offer("b")) + assertEquals(true, queue.offer("c")) + assertEquals(4, queue.size()) + assertEquals(13, queue.byteSize()) + assertEquals(1, queue.remainingSize) + assertEquals(2, queue.remainingByteSize) + + assertEquals(true, queue.offer("d")) + assertEquals(5, queue.size()) + assertEquals(14, queue.byteSize()) + assertEquals(0, queue.remainingSize) + assertEquals(1, queue.remainingByteSize) + + assertEquals(false, queue.offer("e")) + assertEquals(5, queue.size()) + assertEquals(14, queue.byteSize()) + assertEquals(0, queue.remainingSize) + assertEquals(1, queue.remainingByteSize) + + // try take 6 elements out of the queue, the last poll() should fail as there is no element anymore + // test take() + assertEquals("1234567890", queue.poll(10, TimeUnit.MILLISECONDS)) + // test poll + assertEquals("a", queue.poll()) + assertEquals("b", queue.poll()) + assertEquals("c", queue.poll()) + assertEquals("d", queue.poll()) + assertEquals(null, queue.poll(10, TimeUnit.MILLISECONDS)) + } + +} diff --git a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala new file mode 100644 index 0000000000000..6380b6e623e7a --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala @@ -0,0 +1,48 @@ +/** + * 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 org.junit.Test + +class CommandLineUtilsTest { + + + @Test + def testParseEmptyArg() { + val argArray = Array("my.empty.property=") + val props = CommandLineUtils.parseKeyValueArgs(argArray) + assertEquals("Empty value should be equal to empty string",props.getProperty("my.empty.property"),"") + } + + @Test + def testParseSingleArg() { + val argArray = Array("my.property=value") + val props = CommandLineUtils.parseKeyValueArgs(argArray) + assertEquals("Value of a single property should be 'value' ",props.getProperty("my.property"),"value") + } + + @Test + def testParseArgs() { + val argArray = Array("first.property=first","second.property=second") + val props = CommandLineUtils.parseKeyValueArgs(argArray) + assertEquals("Value of first property should be 'first'",props.getProperty("first.property"),"first") + assertEquals("Value of second property should be 'second'",props.getProperty("second.property"),"second") + } + +} diff --git a/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala index 46a4e899ef293..fbd245cad0afa 100644 --- a/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala +++ b/core/src/test/scala/unit/kafka/utils/IteratorTemplateTest.scala @@ -22,7 +22,7 @@ import org.junit.{Test, After, Before} class IteratorTemplateTest extends Assertions { - val lst = (0 until 10).toSeq + val lst = (0 until 10) val iterator = new IteratorTemplate[Int]() { var i = 0 override def makeNext() = { @@ -54,4 +54,4 @@ class IteratorTemplateTest extends Assertions { } } -} \ 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 index d5896ed4d3b73..eeafeda042230 100644 --- a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala +++ b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala @@ -36,6 +36,8 @@ class MockScheduler(val time: Time) extends Scheduler { /* a priority queue of tasks ordered by next execution time */ var tasks = new PriorityQueue[MockTask]() + + def isStarted = true def startup() {} @@ -79,10 +81,10 @@ case class MockTask(val name: String, val fun: () => Unit, var nextExecution: Lo def periodic = period >= 0 def compare(t: MockTask): Int = { if(t.nextExecution == nextExecution) - return 0 + 0 else if (t.nextExecution < nextExecution) - return -1 + -1 else - return 1 + 1 } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala new file mode 100644 index 0000000000000..b9de8d677a46e --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.utils + +import kafka.controller.LeaderIsrAndControllerEpoch +import kafka.server.{ReplicaFetcherManager, KafkaConfig} +import kafka.api.LeaderAndIsr +import kafka.zk.ZooKeeperTestHarness +import kafka.common.TopicAndPartition +import org.scalatest.junit.JUnit3Suite +import org.junit.Assert._ +import org.junit.Test +import org.easymock.EasyMock + + +class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { + val topic = "my-topic-test" + val partitionId = 0 + val brokerId = 1 + val leaderEpoch = 1 + val controllerEpoch = 1 + val zkVersion = 1 + val topicPath = "/brokers/topics/my-topic-test/partitions/0/state" + val topicData = Json.encode(Map("controller_epoch" -> 1, "leader" -> 1, + "versions" -> 1, "leader_epoch" -> 1,"isr" -> List(1,2))) + val topicDataVersionMismatch = Json.encode(Map("controller_epoch" -> 1, "leader" -> 1, + "versions" -> 2, "leader_epoch" -> 1,"isr" -> List(1,2))) + val topicDataMismatch = Json.encode(Map("controller_epoch" -> 1, "leader" -> 1, + "versions" -> 2, "leader_epoch" -> 2,"isr" -> List(1,2))) + + val topicDataLeaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(1,leaderEpoch,List(1,2),0), controllerEpoch) + + + override def setUp() { + super.setUp() + ZkUtils.createPersistentPath(zkClient,topicPath,topicData) + } + + @Test + def testUpdateLeaderAndIsr() { + val configs = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps) + val log = EasyMock.createMock(classOf[kafka.log.Log]) + EasyMock.expect(log.logEndOffset).andReturn(20).anyTimes() + EasyMock.expect(log) + EasyMock.replay(log) + + val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) + EasyMock.expect(logManager.getLog(TopicAndPartition(topic, partitionId))).andReturn(Some(log)).anyTimes() + EasyMock.replay(logManager) + + val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) + EasyMock.expect(replicaManager.config).andReturn(configs.head) + EasyMock.expect(replicaManager.logManager).andReturn(logManager) + EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) + EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) + EasyMock.replay(replicaManager) + + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath) + + val replicas = List(0,1) + + // regular update + val newLeaderAndIsr1 = new LeaderAndIsr(brokerId, leaderEpoch, replicas, 0) + val (updateSucceeded1,newZkVersion1) = ReplicationUtils.updateLeaderAndIsr(zkClient, + "my-topic-test", partitionId, newLeaderAndIsr1, controllerEpoch, 0) + assertTrue(updateSucceeded1) + assertEquals(newZkVersion1, 1) + + // mismatched zkVersion with the same data + val newLeaderAndIsr2 = new LeaderAndIsr(brokerId, leaderEpoch, replicas, zkVersion + 1) + val (updateSucceeded2,newZkVersion2) = ReplicationUtils.updateLeaderAndIsr(zkClient, + "my-topic-test", partitionId, newLeaderAndIsr2, controllerEpoch, zkVersion + 1) + assertTrue(updateSucceeded2) + // returns true with existing zkVersion + assertEquals(newZkVersion2,1) + + // mismatched zkVersion and leaderEpoch + val newLeaderAndIsr3 = new LeaderAndIsr(brokerId, leaderEpoch + 1, replicas, zkVersion + 1) + val (updateSucceeded3,newZkVersion3) = ReplicationUtils.updateLeaderAndIsr(zkClient, + "my-topic-test", partitionId, newLeaderAndIsr3, controllerEpoch, zkVersion + 1) + assertFalse(updateSucceeded3) + assertEquals(newZkVersion3,-1) + } + + @Test + def testGetLeaderIsrAndEpochForPartition() { + val leaderIsrAndControllerEpoch = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partitionId) + assertEquals(topicDataLeaderIsrAndControllerEpoch, leaderIsrAndControllerEpoch.get) + assertEquals(None, ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partitionId + 1)) + } + +} diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index b364ac2d6d623..cfea63b88e259 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -90,4 +90,21 @@ class SchedulerTest { assertTrue("Should count to 20", counter1.get >= 20) } } + + @Test + def testRestart() { + // schedule a task to increment a counter + mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1) + mockTime.sleep(1) + assertEquals(1, counter1.get()) + + // restart the scheduler + mockTime.scheduler.shutdown() + mockTime.scheduler.startup() + + // schedule another task to increment the counter + mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1) + mockTime.sleep(1) + assertEquals(2, counter1.get()) + } } \ 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 old mode 100644 new mode 100755 index 500eeca2f95d9..eb169d8b33c27 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -18,34 +18,43 @@ package kafka.utils import java.io._ -import java.net._ import java.nio._ import java.nio.channels._ import java.util.Random import java.util.Properties -import junit.framework.AssertionFailedError -import junit.framework.Assert._ +import charset.Charset + +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.utils.Utils._ + +import collection.mutable.ListBuffer + +import org.I0Itec.zkclient.ZkClient + import kafka.server._ import kafka.producer._ import kafka.message._ -import org.I0Itec.zkclient.ZkClient -import kafka.cluster.Broker -import collection.mutable.ListBuffer -import kafka.consumer.ConsumerConfig -import java.util.concurrent.locks.ReentrantLock -import java.util.concurrent.TimeUnit import kafka.api._ -import collection.mutable.Map +import kafka.cluster.Broker +import kafka.consumer.{ConsumerTimeoutException, KafkaStream, ConsumerConfig} import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} import kafka.common.TopicAndPartition -import junit.framework.Assert +import kafka.admin.AdminUtils +import kafka.producer.ProducerConfig +import kafka.log._ + +import junit.framework.AssertionFailedError +import junit.framework.Assert._ +import org.apache.kafka.clients.producer.KafkaProducer +import scala.collection.Map +import org.apache.kafka.clients.consumer.KafkaConsumer /** * Utility functions to help with testing */ object TestUtils extends Logging { - + val IoTmpDir = System.getProperty("java.io.tmpdir") val Letters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz" @@ -56,23 +65,13 @@ object TestUtils extends Logging { val seededRandom = new Random(192348092834L) val random = new Random() - /** - * Choose a number of random available ports - */ - def choosePorts(count: Int): List[Int] = { - val sockets = - for(i <- 0 until count) - yield new ServerSocket(0) - val socketList = sockets.toList - val ports = socketList.map(_.getLocalPort) - socketList.map(_.close) - ports - } + /* 0 gives a random port; you can then retrieve the assigned port from the Socket object. */ + val RandomPort = 0 - /** - * Choose an available port - */ - def choosePort(): Int = choosePorts(1).head + /** Port to use for unit tests that mock/don't require a real ZK server. */ + val MockZkPort = 1 + /** Zookeeper connection string to use for unit tests that mock/don't require a real ZK server. */ + val MockZkConnect = "127.0.0.1:" + MockZkPort /** * Create a temporary directory @@ -81,6 +80,25 @@ object TestUtils extends Logging { val f = new File(IoTmpDir, "kafka-" + random.nextInt(1000000)) f.mkdirs() f.deleteOnExit() + + Runtime.getRuntime().addShutdownHook(new Thread() { + override def run() = { + CoreUtils.rm(f) + } + }) + + f + } + + def tempTopic(): String = "testTopic" + random.nextInt(1000000) + + /** + * Create a temporary relative directory + */ + def tempRelativeDir(parent: String): File = { + val f = new File(parent, "kafka-" + random.nextInt(1000000)) + f.mkdirs() + f.deleteOnExit() f } @@ -112,29 +130,74 @@ object TestUtils extends Logging { /** * Create a test config for the given node id */ - def createBrokerConfigs(numConfigs: Int): List[Properties] = { - for((port, node) <- choosePorts(numConfigs).zipWithIndex) - yield createBrokerConfig(node, port) + def createBrokerConfigs(numConfigs: Int, + zkConnect: String, + enableControlledShutdown: Boolean = true, + enableDeleteTopic: Boolean = false): Seq[Properties] = { + (0 until numConfigs).map(node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic)) } - def getBrokerListStrFromConfigs(configs: Seq[KafkaConfig]): String = { - configs.map(c => c.hostName + ":" + c.port).mkString(",") + def getBrokerListStrFromServers(servers: Seq[KafkaServer]): String = { + servers.map(s => formatAddress(s.config.hostName, s.boundPort())).mkString(",") } /** * Create a test config for the given node id */ - def createBrokerConfig(nodeId: Int, port: Int = choosePort()): Properties = { + def createBrokerConfig(nodeId: Int, zkConnect: String, + enableControlledShutdown: Boolean = true, + enableDeleteTopic: Boolean = false, + port: Int = RandomPort): Properties = { val props = new Properties - props.put("broker.id", nodeId.toString) - props.put("host.name", "localhost") - props.put("port", port.toString) + if (nodeId >= 0) props.put("broker.id", nodeId.toString) + props.put("listeners", "PLAINTEXT://localhost:"+port.toString) props.put("log.dir", TestUtils.tempDir().getAbsolutePath) - props.put("zookeeper.connect", TestZKUtils.zookeeperConnect) + props.put("zookeeper.connect", zkConnect) props.put("replica.socket.timeout.ms", "1500") + props.put("controller.socket.timeout.ms", "1500") + props.put("controlled.shutdown.enable", enableControlledShutdown.toString) + props.put("delete.topic.enable", enableDeleteTopic.toString) + props.put("controlled.shutdown.retry.backoff.ms", "100") + props.put("port", port.toString) props } + /** + * Create a topic in zookeeper. + * Wait until the leader is elected and the metadata is propagated to all brokers. + * Return the leader for each partition. + */ + def createTopic(zkClient: ZkClient, + topic: String, + numPartitions: Int = 1, + replicationFactor: Int = 1, + servers: Seq[KafkaServer], + topicConfig: Properties = new Properties) : scala.collection.immutable.Map[Int, Option[Int]] = { + // create topic + AdminUtils.createTopic(zkClient, topic, numPartitions, replicationFactor, topicConfig) + // wait until the update metadata request for new topic reaches all servers + (0 until numPartitions).map { case i => + TestUtils.waitUntilMetadataIsPropagated(servers, topic, i) + i -> TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, i) + }.toMap + } + + /** + * Create a topic in zookeeper using a customized replica assignment. + * Wait until the leader is elected and the metadata is propagated to all brokers. + * Return the leader for each partition. + */ + def createTopic(zkClient: ZkClient, topic: String, partitionReplicaAssignment: collection.Map[Int, Seq[Int]], + servers: Seq[KafkaServer]) : scala.collection.immutable.Map[Int, Option[Int]] = { + // create topic + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaAssignment) + // wait until the update metadata request for new topic reaches all servers + partitionReplicaAssignment.keySet.map { case i => + TestUtils.waitUntilMetadataIsPropagated(servers, topic, i) + i -> TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, i) + }.toMap + } + /** * Create a test config for a consumer */ @@ -145,7 +208,7 @@ object TestUtils extends Logging { props.put("group.id", groupId) props.put("consumer.id", consumerId) props.put("consumer.timeout.ms", consumerTimeout.toString) - props.put("zookeeper.session.timeout.ms", "400") + props.put("zookeeper.session.timeout.ms", "6000") props.put("zookeeper.sync.time.ms", "200") props.put("auto.commit.interval.ms", "1000") props.put("rebalance.max.retries", "4") @@ -290,30 +353,87 @@ object TestUtils extends Logging { } /** - * Create a producer for the given host and port + * Create a producer with a few pre-configured properties. + * If certain properties need to be overridden, they can be provided in producerProps. */ - def createProducer[K, V](brokerList: String, - encoder: Encoder[V] = new DefaultEncoder(), - keyEncoder: Encoder[K] = new DefaultEncoder()): Producer[K, V] = { - val props = new Properties() - props.put("metadata.broker.list", brokerList) - props.put("send.buffer.bytes", "65536") - props.put("connect.timeout.ms", "100000") - props.put("reconnect.interval", "10000") - props.put("serializer.class", encoder.getClass.getCanonicalName) - props.put("key.serializer.class", keyEncoder.getClass.getCanonicalName) + def createProducer[K, V](brokerList: String, + encoder: String = classOf[DefaultEncoder].getName, + keyEncoder: String = classOf[DefaultEncoder].getName, + partitioner: String = classOf[DefaultPartitioner].getName, + producerProps: Properties = null): Producer[K, V] = { + val props: Properties = getProducerConfig(brokerList) + + //override any explicitly specified properties + if (producerProps != null) + props.putAll(producerProps) + + props.put("serializer.class", encoder) + props.put("key.serializer.class", keyEncoder) + props.put("partitioner.class", partitioner) new Producer[K, V](new ProducerConfig(props)) } - def getProducerConfig(brokerList: String, partitioner: String = "kafka.producer.DefaultPartitioner"): Properties = { + /** + * Create a (new) producer with a few pre-configured properties. + */ + def createNewProducer(brokerList: String, + acks: Int = -1, + metadataFetchTimeout: Long = 3000L, + blockOnBufferFull: Boolean = true, + bufferSize: Long = 1024L * 1024L, + retries: Int = 0, + lingerMs: Long = 0) : KafkaProducer[Array[Byte],Array[Byte]] = { + import org.apache.kafka.clients.producer.ProducerConfig + + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.ACKS_CONFIG, acks.toString) + producerProps.put(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG, metadataFetchTimeout.toString) + producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, blockOnBufferFull.toString) + producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, bufferSize.toString) + producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString) + producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "100") + producerProps.put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200") + producerProps.put(ProducerConfig.LINGER_MS_CONFIG, lingerMs.toString) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + } + + /** + * Create a new consumer with a few pre-configured properties. + */ + def createNewConsumer(brokerList: String, + groupId: String, + autoOffsetReset: String = "earliest", + partitionFetchSize: Long = 4096L) : KafkaConsumer[Array[Byte],Array[Byte]] = { + import org.apache.kafka.clients.consumer.ConsumerConfig + + val consumerProps= new Properties() + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetReset) + consumerProps.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, partitionFetchSize.toString) + consumerProps.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, "100") + consumerProps.put(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200") + consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer") + consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer") + new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps) + } + + /** + * Create a default producer config properties map with the given metadata broker list + */ + def getProducerConfig(brokerList: String): Properties = { val props = new Properties() props.put("metadata.broker.list", brokerList) - props.put("partitioner.class", partitioner) - props.put("message.send.max.retries", "3") + props.put("message.send.max.retries", "5") props.put("retry.backoff.ms", "1000") - props.put("request.timeout.ms", "500") + props.put("request.timeout.ms", "2000") props.put("request.required.acks", "-1") - props.put("serializer.class", classOf[StringEncoder].getName.toString) + props.put("send.buffer.bytes", "65536") + props.put("connect.timeout.ms", "100000") + props.put("reconnect.interval", "10000") props } @@ -324,12 +444,12 @@ object TestUtils extends Logging { props.put("port", port.toString) props.put("request.timeout.ms", "500") props.put("request.required.acks", "1") - props.put("serializer.class", classOf[StringEncoder].getName.toString) + props.put("serializer.class", classOf[StringEncoder].getName) props } def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = { - val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) + val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs) ZkUtils.updatePersistentPath(zkClient, path, offset.toString) } @@ -338,21 +458,21 @@ object TestUtils extends Logging { new IteratorTemplate[Message] { override def makeNext(): Message = { if (iter.hasNext) - return iter.next.message + iter.next.message else - return allDone() + allDone() } } } 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, SecurityProtocol.PLAINTEXT)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, "localhost", 6667, b.endPoints, 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, SecurityProtocol.PLAINTEXT)) brokers.foreach(b => ZkUtils.deletePath(zkClient, ZkUtils.BrokerIdsPath + "/" + b)) brokers } @@ -367,9 +487,9 @@ object TestUtils extends Logging { /** * Create a wired format request based on simple basic information */ - def produceRequest(topic: String, - partition: Int, - message: ByteBufferMessageSet, + def produceRequest(topic: String, + partition: Int, + message: ByteBufferMessageSet, acks: Int = SyncProducerConfig.DefaultRequiredAcks, timeout: Int = SyncProducerConfig.DefaultAckTimeoutMs, correlationId: Int = 0, @@ -377,17 +497,17 @@ object TestUtils extends Logging { produceRequestWithAcks(Seq(topic), Seq(partition), message, acks, timeout, correlationId, clientId) } - def produceRequestWithAcks(topics: Seq[String], - partitions: Seq[Int], - message: ByteBufferMessageSet, - acks: Int = SyncProducerConfig.DefaultRequiredAcks, + def produceRequestWithAcks(topics: Seq[String], + partitions: Seq[Int], + message: ByteBufferMessageSet, + acks: Int = SyncProducerConfig.DefaultRequiredAcks, timeout: Int = SyncProducerConfig.DefaultAckTimeoutMs, correlationId: Int = 0, clientId: String = SyncProducerConfig.DefaultClientId): ProducerRequest = { val data = topics.flatMap(topic => partitions.map(partition => (TopicAndPartition(topic, partition), message)) ) - new ProducerRequest(correlationId, clientId, acks.toShort, timeout, Map(data:_*)) + new ProducerRequest(correlationId, clientId, acks.toShort, timeout, collection.mutable.Map(data:_*)) } def makeLeaderForPartition(zkClient: ZkClient, topic: String, @@ -418,36 +538,51 @@ object TestUtils extends Logging { } } - def waitUntilLeaderIsElectedOrChanged(zkClient: ZkClient, topic: String, partition: Int, timeoutMs: Long, oldLeaderOpt: Option[Int] = None): Option[Int] = { - val leaderLock = new ReentrantLock() - val leaderExistsOrChanged = leaderLock.newCondition() + /** + * If neither oldLeaderOpt nor newLeaderOpt is defined, wait until the leader of a partition is elected. + * If oldLeaderOpt is defined, it waits until the new leader is different from the old leader. + * If newLeaderOpt is defined, it waits until the new leader becomes the expected new leader. + * @return The new leader or assertion failure if timeout is reached. + */ + def waitUntilLeaderIsElectedOrChanged(zkClient: ZkClient, topic: String, partition: Int, timeoutMs: Long = 5000L, + oldLeaderOpt: Option[Int] = None, newLeaderOpt: Option[Int] = None): Option[Int] = { + require(!(oldLeaderOpt.isDefined && newLeaderOpt.isDefined), "Can't define both the old and the new leader") + val startTime = System.currentTimeMillis() + var isLeaderElectedOrChanged = false - if(oldLeaderOpt == None) - info("Waiting for leader to be elected for partition [%s,%d]".format(topic, partition)) - else - info("Waiting for leader for partition [%s,%d] to be changed from old leader %d".format(topic, partition, oldLeaderOpt.get)) + trace("Waiting for leader to be elected or changed for partition [%s,%d], older leader is %s, new leader is %s" + .format(topic, partition, oldLeaderOpt, newLeaderOpt)) - leaderLock.lock() - try { - zkClient.subscribeDataChanges(ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition), new LeaderExistsOrChangedListener(topic, partition, leaderLock, leaderExistsOrChanged, oldLeaderOpt, zkClient)) - leaderExistsOrChanged.await(timeoutMs, TimeUnit.MILLISECONDS) + var leader: Option[Int] = None + while (!isLeaderElectedOrChanged && System.currentTimeMillis() < startTime + timeoutMs) { // check if leader is elected - val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition) + leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition) leader match { case Some(l) => - if(oldLeaderOpt == None) - info("Leader %d is elected for partition [%s,%d]".format(l, topic, partition)) - else - info("Leader for partition [%s,%d] is changed from %d to %d".format(topic, partition, oldLeaderOpt.get, l)) - case None => error("Timing out after %d ms since leader is not elected for partition [%s,%d]" - .format(timeoutMs, topic, partition)) + if (newLeaderOpt.isDefined && newLeaderOpt.get == l) { + trace("Expected new leader %d is elected for partition [%s,%d]".format(l, topic, partition)) + isLeaderElectedOrChanged = true + } else if (oldLeaderOpt.isDefined && oldLeaderOpt.get != l) { + trace("Leader for partition [%s,%d] is changed from %d to %d".format(topic, partition, oldLeaderOpt.get, l)) + isLeaderElectedOrChanged = true + } else if (!oldLeaderOpt.isDefined) { + trace("Leader %d is elected for partition [%s,%d]".format(l, topic, partition)) + isLeaderElectedOrChanged = true + } else { + trace("Current leader for partition [%s,%d] is %d".format(topic, partition, l)) + } + case None => + trace("Leader for partition [%s,%d] is not elected yet".format(topic, partition)) } - leader - } finally { - leaderLock.unlock() + Thread.sleep(timeoutMs.min(100L)) } + if (!isLeaderElectedOrChanged) + fail("Timing out after %d ms since leader is not elected or changed for partition [%s,%d]" + .format(timeoutMs, topic, partition)) + + leader } - + /** * Execute the given block. If it throws an assert error, retry. Repeat * until no error is thrown or the time limit ellapses @@ -461,7 +596,7 @@ object TestUtils extends Logging { return } catch { case e: AssertionFailedError => - val ellapsed = System.currentTimeMillis - startTime + val ellapsed = System.currentTimeMillis - startTime if(ellapsed > maxWaitMs) { throw e } else { @@ -474,15 +609,15 @@ object TestUtils extends Logging { } /** - * Wait until the given condition is true or the given wait time ellapses + * Wait until the given condition is true or throw an exception if the given wait time elapses. */ - def waitUntilTrue(condition: () => Boolean, waitTime: Long): Boolean = { + def waitUntilTrue(condition: () => Boolean, msg: String, waitTime: Long = 5000L): Boolean = { val startTime = System.currentTimeMillis() while (true) { if (condition()) return true if (System.currentTimeMillis() > startTime + waitTime) - return false + fail(msg) Thread.sleep(waitTime.min(100L)) } // should never hit here @@ -510,12 +645,35 @@ object TestUtils extends Logging { byteBuffer } - def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int, timeout: Long) = { - Assert.assertTrue("Partition [%s,%d] metadata not propagated after timeout".format(topic, partition), - TestUtils.waitUntilTrue(() => - servers.foldLeft(true)(_ && _.apis.metadataCache.keySet.contains(TopicAndPartition(topic, partition))), timeout)) + + /** + * Wait until a valid leader is propagated to the metadata cache in each broker. + * It assumes that the leader propagated to each broker is the same. + * @param servers The list of servers that the metadata should reach to + * @param topic The topic name + * @param partition The partition Id + * @param timeout The amount of time waiting on this condition before assert to fail + * @return The leader of the partition. + */ + def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int, timeout: Long = 5000L): Int = { + var leader: Int = -1 + TestUtils.waitUntilTrue(() => + servers.foldLeft(true) { + (result, server) => + val partitionStateOpt = server.apis.metadataCache.getPartitionInfo(topic, partition) + partitionStateOpt match { + case None => false + case Some(partitionState) => + leader = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr.leader + result && Request.isValidBrokerId(leader) + } + }, + "Partition [%s,%d] metadata not propagated after %d ms".format(topic, partition, timeout), + waitTime = timeout) + + leader } - + def writeNonsenseToFile(fileName: File, position: Long, size: Int) { val file = new RandomAccessFile(fileName, "rw") file.seek(position) @@ -523,7 +681,7 @@ object TestUtils extends Logging { file.writeByte(random.nextInt(255)) file.close() } - + def appendNonsenseToFile(fileName: File, size: Int) { val file = new FileOutputStream(fileName, true) for(i <- 0 until size) @@ -541,23 +699,181 @@ object TestUtils extends Logging { def ensureNoUnderReplicatedPartitions(zkClient: ZkClient, 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) + TestUtils.waitUntilTrue(() => { + val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionToBeReassigned) + inSyncReplicas.size == assignedReplicas.size + }, + "Reassigned partition [%s,%d] is under replicated".format(topic, partitionToBeReassigned)) + var leader: Option[Int] = None + TestUtils.waitUntilTrue(() => { + leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitionToBeReassigned) + leader.isDefined + }, + "Reassigned partition [%s,%d] is unavailable".format(topic, partitionToBeReassigned)) + TestUtils.waitUntilTrue(() => { + val leaderBroker = servers.filter(s => s.config.brokerId == leader.get).head + leaderBroker.replicaManager.underReplicatedPartitionCount() == 0 + }, + "Reassigned partition [%s,%d] is under-replicated as reported by the leader %d".format(topic, partitionToBeReassigned, leader.get)) } def checkIfReassignPartitionPathExists(zkClient: ZkClient): Boolean = { ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath) } -} -object TestZKUtils { - val zookeeperConnect = "127.0.0.1:" + TestUtils.choosePort() + def verifyNonDaemonThreadsStatus() { + assertEquals(0, Thread.getAllStackTraces.keySet().toArray + .map(_.asInstanceOf[Thread]) + .count(t => !t.isDaemon && t.isAlive && t.getClass.getCanonicalName.toLowerCase.startsWith("kafka"))) + } + + /** + * Create new LogManager instance with default configuration for testing + */ + def createLogManager(logDirs: Array[File] = Array.empty[File], + defaultConfig: LogConfig = LogConfig(), + cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), + time: MockTime = new MockTime()): LogManager = { + new LogManager(logDirs = logDirs, + topicConfigs = Map(), + defaultConfig = defaultConfig, + cleanerConfig = cleanerConfig, + ioThreads = 4, + flushCheckMs = 1000L, + flushCheckpointMs = 10000L, + retentionCheckMs = 1000L, + scheduler = time.scheduler, + time = time, + brokerState = new BrokerState()) + } + def sendMessages(servers: Seq[KafkaServer], + topic: String, + numMessages: Int, + partition: Int = -1, + compression: CompressionCodec = NoCompressionCodec): List[String] = { + val header = "test-%d".format(partition) + val props = new Properties() + props.put("compression.codec", compression.codec.toString) + val ms = 0.until(numMessages).map(x => header + "-" + x) + + // Specific Partition + if (partition >= 0) { + val producer: Producer[Int, String] = + createProducer(TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[IntEncoder].getName, + partitioner = classOf[FixedValuePartitioner].getName, + producerProps = props) + + producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) + debug("Sent %d messages for partition [%s,%d]".format(ms.size, topic, partition)) + producer.close() + ms.toList + } else { + // Use topic as the key to determine partition + val producer: Producer[String, String] = createProducer( + TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[DefaultPartitioner].getName, + producerProps = props) + producer.send(ms.map(m => new KeyedMessage[String, String](topic, topic, m)):_*) + producer.close() + debug("Sent %d messages for topic [%s]".format(ms.size, topic)) + ms.toList + } + + } + + def sendMessage(servers: Seq[KafkaServer], + topic: String, + message: String) = { + + val producer: Producer[String, String] = + createProducer(TestUtils.getBrokerListStrFromServers(servers), + encoder = classOf[StringEncoder].getName(), + keyEncoder = classOf[StringEncoder].getName()) + + producer.send(new KeyedMessage[String, String](topic, topic, message)) + producer.close() + } + + /** + * Consume all messages (or a specific number of messages) + * @param topicMessageStreams the Topic Message Streams + * @param nMessagesPerThread an optional field to specify the exact number of messages to be returned. + * ConsumerTimeoutException will be thrown if there are no messages to be consumed. + * If not specified, then all available messages will be consumed, and no exception is thrown. + * + * + * @return the list of messages consumed. + */ + def getMessages(topicMessageStreams: Map[String, List[KafkaStream[String, String]]], + nMessagesPerThread: Int = -1): List[String] = { + + var messages: List[String] = Nil + val shouldGetAllMessages = nMessagesPerThread < 0 + for ((topic, messageStreams) <- topicMessageStreams) { + for (messageStream <- messageStreams) { + val iterator = messageStream.iterator() + try { + var i = 0 + while ((shouldGetAllMessages && iterator.hasNext()) || (i < nMessagesPerThread)) { + assertTrue(iterator.hasNext) + val message = iterator.next.message // will throw a timeout exception if the message isn't there + messages ::= message + debug("received message: " + message) + i += 1 + } + } catch { + case e: ConsumerTimeoutException => + if (shouldGetAllMessages) { + // swallow the exception + debug("consumer timed out after receiving " + messages.length + " message(s).") + } else { + throw e + } + } + } + } + + messages.reverse + } + + def verifyTopicDeletion(zkClient: ZkClient, topic: String, numPartitions: Int, servers: Seq[KafkaServer]) { + val topicAndPartitions = (0 until numPartitions).map(TopicAndPartition(topic, _)) + // wait until admin path for delete topic is deleted, signaling completion of topic deletion + TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), + "Admin path /admin/delete_topic/%s path not deleted even after a replica is restarted".format(topic)) + TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)), + "Topic path /brokers/topics/%s not deleted after /admin/delete_topic/%s path is deleted".format(topic, topic)) + // ensure that the topic-partition has been deleted from all brokers' replica managers + TestUtils.waitUntilTrue(() => + servers.forall(server => topicAndPartitions.forall(tp => server.replicaManager.getPartition(tp.topic, tp.partition) == None)), + "Replica manager's should have deleted all of this topic's partitions") + // ensure that logs from all replicas are deleted if delete topic is marked successful in zookeeper + assertTrue("Replica logs not deleted after delete topic is complete", + servers.forall(server => topicAndPartitions.forall(tp => server.getLogManager().getLog(tp).isEmpty))) + // ensure that topic is removed from all cleaner offsets + TestUtils.waitUntilTrue(() => servers.forall(server => topicAndPartitions.forall { tp => + val checkpoints = server.getLogManager().logDirs.map { logDir => + new OffsetCheckpoint(new File(logDir, "cleaner-offset-checkpoint")).read() + } + checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp)) + }), "Cleaner offset for deleted partition should have been removed") + } + + /** + * Translate the given buffer into a string + * @param buffer The buffer to translate + * @param encoding The encoding to use in translating bytes to characters + */ + def readString(buffer: ByteBuffer, encoding: String = Charset.defaultCharset.toString): String = { + val bytes = new Array[Byte](buffer.remaining) + buffer.get(bytes) + new String(bytes, encoding) + } + } class IntEncoder(props: VerifiableProperties = null) extends Encoder[Int] { diff --git a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala old mode 100644 new mode 100755 index 920f318c85a74..9e8869c44a115 --- a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala @@ -20,13 +20,13 @@ 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 kafka.utils.CoreUtils.inLock import org.junit.Test +import org.apache.kafka.common.utils.Utils class UtilsTest extends JUnitSuite { @@ -35,13 +35,13 @@ class UtilsTest extends JUnitSuite { @Test def testSwallow() { - Utils.swallow(logger.info, throw new KafkaException("test")) + CoreUtils.swallow(logger.info, throw new KafkaException("test")) } @Test def testCircularIterator() { val l = List(1, 2) - val itl = Utils.circularIterator(l) + val itl = CoreUtils.circularIterator(l) assertEquals(1, itl.next()) assertEquals(2, itl.next()) assertEquals(1, itl.next()) @@ -49,7 +49,7 @@ class UtilsTest extends JUnitSuite { assertFalse(itl.hasDefiniteSize) val s = Set(1, 2) - val its = Utils.circularIterator(s) + val its = CoreUtils.circularIterator(s) assertEquals(1, its.next()) assertEquals(2, its.next()) assertEquals(1, its.next()) @@ -64,39 +64,83 @@ class UtilsTest extends JUnitSuite { assertTrue(Arrays.equals(bytes, Utils.readBytes(ByteBuffer.wrap(bytes)))) } } - + + @Test + def testAbs() { + assertEquals(0, Utils.abs(Integer.MIN_VALUE)) + assertEquals(1, Utils.abs(-1)) + assertEquals(0, Utils.abs(0)) + assertEquals(1, Utils.abs(1)) + assertEquals(Integer.MAX_VALUE, Utils.abs(Integer.MAX_VALUE)) + } + @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")) + assertEquals("blah.foo.text", CoreUtils.replaceSuffix("blah.foo.txt", ".txt", ".text")) + assertEquals("blah.foo", CoreUtils.replaceSuffix("blah.foo.txt", ".txt", "")) + assertEquals("txt.txt", CoreUtils.replaceSuffix("txt.txt.txt", ".txt", "")) + assertEquals("foo.txt", CoreUtils.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)) + assertEquals("Written value should match read value.", values(i), CoreUtils.readInt(buffer.array, i*4)) } - } @Test def testCsvList() { val emptyString:String = "" val nullString:String = null - val emptyList = Utils.parseCsvList(emptyString) - val emptyListFromNullString = Utils.parseCsvList(nullString) + val emptyList = CoreUtils.parseCsvList(emptyString) + val emptyListFromNullString = CoreUtils.parseCsvList(nullString) val emptyStringList = Seq.empty[String] assertTrue(emptyList!=null) assertTrue(emptyListFromNullString!=null) assertTrue(emptyStringList.equals(emptyListFromNullString)) assertTrue(emptyStringList.equals(emptyList)) } - + + @Test + def testCsvMap() { + val emptyString: String = "" + val emptyMap = CoreUtils.parseCsvMap(emptyString) + val emptyStringMap = Map.empty[String, String] + assertTrue(emptyMap != null) + assertTrue(emptyStringMap.equals(emptyStringMap)) + + val kvPairsIpV6: String = "a:b:c:v,a:b:c:v" + val ipv6Map = CoreUtils.parseCsvMap(kvPairsIpV6) + for (m <- ipv6Map) { + assertTrue(m._1.equals("a:b:c")) + assertTrue(m._2.equals("v")) + } + + val singleEntry:String = "key:value" + val singleMap = CoreUtils.parseCsvMap(singleEntry) + val value = singleMap.getOrElse("key", 0) + assertTrue(value.equals("value")) + + val kvPairsIpV4: String = "192.168.2.1/30:allow, 192.168.2.1/30:allow" + val ipv4Map = CoreUtils.parseCsvMap(kvPairsIpV4) + for (m <- ipv4Map) { + assertTrue(m._1.equals("192.168.2.1/30")) + assertTrue(m._2.equals("allow")) + } + + val kvPairsSpaces: String = "key:value , key: value" + val spaceMap = CoreUtils.parseCsvMap(kvPairsSpaces) + for (m <- spaceMap) { + assertTrue(m._1.equals("key")) + assertTrue(m._2.equals("value")) + } + } + + @Test def testInLock() { val lock = new ReentrantLock() @@ -106,6 +150,55 @@ class UtilsTest extends JUnitSuite { } assertEquals(2, result) assertFalse("Should be unlocked", lock.isLocked) - } + + @Test + def testDoublyLinkedList() { + val list = new DoublyLinkedList[Int] + + // test remove from a single-entry list. + list.add(new DoublyLinkedListNode[Int](0)) + list.remove() + assert(list.size == 0) + assert(list.peek() == null) + + // test add + for (i <- 0 to 2) { + list.add(new DoublyLinkedListNode[Int](i)) + } + val toBeRemoved1 = new DoublyLinkedListNode[Int](3) + list.add(toBeRemoved1) + for (i <- 4 to 6) { + list.add(new DoublyLinkedListNode[Int](i)) + } + val toBeRemoved2 = new DoublyLinkedListNode[Int](7) + list.add(toBeRemoved2) + + // test iterator + val iter = list.iterator + for (i <- 0 to 7) { + assert(iter.hasNext) + assert(iter.next().element == i) + } + assert(!iter.hasNext) + + // remove from head + list.remove() + assert(list.peek().element == 1) + // remove from middle + list.remove(toBeRemoved1) + // remove from tail + list.remove(toBeRemoved2) + + // List = [1,2,4,5,6] + val iter2 = list.iterator + for (i <- Array[Int](1,2,4,5,6)) { + assert(iter2.hasNext) + assert(iter2.next().element == i) + } + + // test size + assert(list.size == 5) + } + } diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala new file mode 100644 index 0000000000000..052aecdc32ae1 --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala @@ -0,0 +1,95 @@ +/** + * 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.timer + +import junit.framework.Assert._ +import java.util.concurrent.atomic._ +import org.junit.{Test, After, Before} + +class TimerTaskListTest { + + private class TestTask(val expirationMs: Long) extends TimerTask { + def run(): Unit = { } + } + + private def size(list: TimerTaskList): Int = { + var count = 0 + list.foreach(_ => count += 1) + count + } + + @Test + def testAll() { + val sharedCounter = new AtomicInteger(0) + val runCounter = new AtomicInteger(0) + val execCounter = new AtomicInteger(0) + val list1 = new TimerTaskList(sharedCounter) + val list2 = new TimerTaskList(sharedCounter) + val list3 = new TimerTaskList(sharedCounter) + + val tasks = (1 to 10).map { i => + val task = new TestTask(10L) + list1.add(new TimerTaskEntry(task)) + assertEquals(i, sharedCounter.get) + task + }.toSeq + + assertEquals(tasks.size, sharedCounter.get) + + // reinserting the existing tasks shouldn't change the task count + tasks.take(4).foreach { task => + val prevCount = sharedCounter.get + // new TimerTaskEntry(task) will remove the existing entry from the list + list2.add(new TimerTaskEntry(task)) + assertEquals(prevCount, sharedCounter.get) + } + assertEquals(10 - 4, size(list1)) + assertEquals(4, size(list2)) + + assertEquals(tasks.size, sharedCounter.get) + + // reinserting the existing tasks shouldn't change the task count + tasks.drop(4).foreach { task => + val prevCount = sharedCounter.get + // new TimerTaskEntry(task) will remove the existing entry from the list + list3.add(new TimerTaskEntry(task)) + assertEquals(prevCount, sharedCounter.get) + } + assertEquals(0, size(list1)) + assertEquals(4, size(list2)) + assertEquals(6, size(list3)) + + assertEquals(tasks.size, sharedCounter.get) + + // cancel tasks in lists + list1.foreach { _.cancel() } + assertEquals(0, size(list1)) + assertEquals(4, size(list2)) + assertEquals(6, size(list3)) + + list2.foreach { _.cancel() } + assertEquals(0, size(list1)) + assertEquals(0, size(list2)) + assertEquals(6, size(list3)) + + list3.foreach { _.cancel() } + assertEquals(0, size(list1)) + assertEquals(0, size(list2)) + assertEquals(0, size(list3)) + } + +} diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala new file mode 100644 index 0000000000000..8507592d1f374 --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala @@ -0,0 +1,111 @@ +/** + * 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.timer + +import java.util.concurrent.{CountDownLatch, ExecutorService, Executors, TimeUnit} + +import junit.framework.Assert._ +import java.util.concurrent.atomic._ +import org.junit.{Test, After, Before} + +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +class TimerTest { + + private class TestTask(override val expirationMs: Long, id: Int, latch: CountDownLatch, output: ArrayBuffer[Int]) extends TimerTask { + private[this] val completed = new AtomicBoolean(false) + def run(): Unit = { + if (completed.compareAndSet(false, true)) { + output.synchronized { output += id } + latch.countDown() + } + } + } + + private[this] var executor: ExecutorService = null + + @Before + def setup() { + executor = Executors.newSingleThreadExecutor() + } + + @After + def teardown(): Unit = { + executor.shutdown() + executor = null + } + + @Test + def testAlreadyExpiredTask(): Unit = { + val startTime = System.currentTimeMillis() + val timer = new Timer(taskExecutor = executor, tickMs = 1, wheelSize = 3, startMs = startTime) + val output = new ArrayBuffer[Int]() + + + val latches = (-5 until 0).map { i => + val latch = new CountDownLatch(1) + timer.add(new TestTask(startTime + i, i, latch, output)) + latch + } + + latches.take(5).foreach { latch => + assertEquals("already expired tasks should run immediately", true, latch.await(3, TimeUnit.SECONDS)) + } + + assertEquals("output of already expired tasks", Set(-5, -4, -3, -2, -1), output.toSet) + } + + @Test + def testTaskExpiration(): Unit = { + val startTime = System.currentTimeMillis() + val timer = new Timer(taskExecutor = executor, tickMs = 1, wheelSize = 3, startMs = startTime) + val output = new ArrayBuffer[Int]() + + val tasks = new ArrayBuffer[TestTask]() + val ids = new ArrayBuffer[Int]() + + val latches = + (0 until 5).map { i => + val latch = new CountDownLatch(1) + tasks += new TestTask(startTime + i, i, latch, output) + ids += i + latch + } ++ (10 until 100).map { i => + val latch = new CountDownLatch(2) + tasks += new TestTask(startTime + i, i, latch, output) + tasks += new TestTask(startTime + i, i, latch, output) + ids += i + ids += i + latch + } ++ (100 until 500).map { i => + val latch = new CountDownLatch(1) + tasks += new TestTask(startTime + i, i, latch, output) + ids += i + latch + } + + // randomly submit requests + Random.shuffle(tasks.toSeq).foreach { task => timer.add(task) } + + while (timer.advanceClock(1000)) {} + + latches.foreach { latch => latch.await() } + + assertEquals("output should match", ids.sorted, output.toSeq) + } +} diff --git a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala old mode 100644 new mode 100755 index d883bdeee1f58..2bca2cf5a554d --- a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala +++ b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala @@ -18,25 +18,28 @@ package kafka.zk import org.apache.zookeeper.server.ZooKeeperServer -import org.apache.zookeeper.server.NIOServerCnxn +import org.apache.zookeeper.server.NIOServerCnxnFactory import kafka.utils.TestUtils import java.net.InetSocketAddress -import kafka.utils.Utils +import kafka.utils.CoreUtils +import org.apache.kafka.common.utils.Utils.getPort -class EmbeddedZookeeper(val connectString: String) { +class EmbeddedZookeeper() { val snapshotDir = TestUtils.tempDir() val logDir = TestUtils.tempDir() val tickTime = 500 val zookeeper = new ZooKeeperServer(snapshotDir, logDir, tickTime) - val port = connectString.split(":")(1).toInt - val factory = new NIOServerCnxn.Factory(new InetSocketAddress("127.0.0.1", port)) + val factory = new NIOServerCnxnFactory() + private val addr = new InetSocketAddress("127.0.0.1", TestUtils.RandomPort) + factory.configure(addr, 0) factory.startup(zookeeper) + val port = zookeeper.getClientPort() def shutdown() { - Utils.swallow(zookeeper.shutdown()) - Utils.swallow(factory.shutdown()) - Utils.rm(logDir) - Utils.rm(snapshotDir) + CoreUtils.swallow(zookeeper.shutdown()) + CoreUtils.swallow(factory.shutdown()) + CoreUtils.rm(logDir) + CoreUtils.rm(snapshotDir) } } diff --git a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala index 85eec6fa5d09f..2be1619856572 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala @@ -19,7 +19,7 @@ package kafka.zk import kafka.consumer.ConsumerConfig import org.I0Itec.zkclient.ZkClient -import kafka.utils.{ZkUtils, ZKStringSerializer} +import kafka.utils.ZkUtils import kafka.utils.TestUtils import org.junit.Assert import org.scalatest.junit.JUnit3Suite @@ -29,8 +29,7 @@ class ZKEphemeralTest extends JUnit3Suite with ZooKeeperTestHarness { def testEphemeralNodeCleanup = { val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) - var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) try { ZkUtils.createEphemeralPathExpectConflict(zkClient, "/tmp/zktest", "node created") @@ -42,8 +41,7 @@ class ZKEphemeralTest extends JUnit3Suite with ZooKeeperTestHarness { testData = ZkUtils.readData(zkClient, "/tmp/zktest")._1 Assert.assertNotNull(testData) zkClient.close - zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - ZKStringSerializer) + zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) val nodeExists = ZkUtils.pathExists(zkClient, "/tmp/zktest") Assert.assertFalse(nodeExists) } diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala new file mode 100644 index 0000000000000..d3e44c62e272b --- /dev/null +++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala @@ -0,0 +1,145 @@ +/** + * 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.zk + +import junit.framework.Assert +import kafka.consumer.ConsumerConfig +import kafka.utils.{ZkPath, TestUtils, ZkUtils} +import org.apache.kafka.common.config.ConfigException +import org.scalatest.junit.JUnit3Suite + +class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { + + val path: String = "/some_dir" + val zkSessionTimeoutMs = 1000 + def zkConnectWithInvalidRoot: String = zkConnect + "/ghost" + + def testCreatePersistentPathThrowsException { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, + "test", "1")) + var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) + try { + ZkPath.resetNamespaceCheckedState + ZkUtils.createPersistentPath(zkClient, path) + fail("Failed to throw ConfigException for missing zookeeper root node") + } catch { + case configException: ConfigException => + case exception: Throwable => fail("Should have thrown ConfigException") + } + } + + def testCreatePersistentPath { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) + try { + ZkPath.resetNamespaceCheckedState + ZkUtils.createPersistentPath(zkClient, path) + } catch { + case exception: Throwable => fail("Failed to create persistent path") + } + + Assert.assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) + } + + def testMakeSurePersistsPathExistsThrowsException { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, + "test", "1")) + var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) + try { + ZkPath.resetNamespaceCheckedState + ZkUtils.makeSurePersistentPathExists(zkClient, path) + fail("Failed to throw ConfigException for missing zookeeper root node") + } catch { + case configException: ConfigException => + case exception: Throwable => fail("Should have thrown ConfigException") + } + } + + def testMakeSurePersistsPathExists { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) + try { + ZkPath.resetNamespaceCheckedState + ZkUtils.makeSurePersistentPathExists(zkClient, path) + } catch { + case exception: Throwable => fail("Failed to create persistent path") + } + + Assert.assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)) + } + + def testCreateEphemeralPathThrowsException { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, + "test", "1")) + var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) + try { + ZkPath.resetNamespaceCheckedState + ZkUtils.createEphemeralPathExpectConflict(zkClient, path, "somedata") + fail("Failed to throw ConfigException for missing zookeeper root node") + } catch { + case configException: ConfigException => + case exception: Throwable => fail("Should have thrown ConfigException") + } + } + + def testCreateEphemeralPathExists { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) + try { + ZkPath.resetNamespaceCheckedState + ZkUtils.createEphemeralPathExpectConflict(zkClient, path, "somedata") + } catch { + case exception: Throwable => fail("Failed to create ephemeral path") + } + + Assert.assertTrue("Failed to create ephemeral path", ZkUtils.pathExists(zkClient, path)) + } + + def testCreatePersistentSequentialThrowsException { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, + "test", "1")) + var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs) + try { + ZkPath.resetNamespaceCheckedState + ZkUtils.createSequentialPersistentPath(zkClient, path) + fail("Failed to throw ConfigException for missing zookeeper root node") + } catch { + case configException: ConfigException => + case exception: Throwable => fail("Should have thrown ConfigException") + } + } + + def testCreatePersistentSequentialExists { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) + var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs) + + var actualPath: String = "" + try { + ZkPath.resetNamespaceCheckedState + actualPath = ZkUtils.createSequentialPersistentPath(zkClient, path) + } catch { + case exception: Throwable => fail("Failed to create persistent path") + } + + Assert.assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, actualPath)) + } +} diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala old mode 100644 new mode 100755 index 4e25b926d32e4..1f4d10d25ab3b --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -19,25 +19,28 @@ package kafka.zk import org.scalatest.junit.JUnit3Suite import org.I0Itec.zkclient.ZkClient -import kafka.utils.{ZKStringSerializer, TestZKUtils, Utils} +import kafka.utils.{ZkUtils, CoreUtils} trait ZooKeeperTestHarness extends JUnit3Suite { - val zkConnect: String = TestZKUtils.zookeeperConnect + var zkPort: Int = -1 var zookeeper: EmbeddedZookeeper = null var zkClient: ZkClient = null val zkConnectionTimeout = 6000 val zkSessionTimeout = 6000 + def zkConnect: String = "127.0.0.1:" + zkPort + override def setUp() { - zookeeper = new EmbeddedZookeeper(zkConnect) - zkClient = new ZkClient(zookeeper.connectString, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) super.setUp + zookeeper = new EmbeddedZookeeper() + zkPort = zookeeper.port + zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeout, zkConnectionTimeout) } override def tearDown() { + CoreUtils.swallow(zkClient.close()) + CoreUtils.swallow(zookeeper.shutdown()) super.tearDown - Utils.swallow(zkClient.close()) - Utils.swallow(zookeeper.shutdown()) } } diff --git a/dev-utils/test-patch.py b/dev-utils/test-patch.py new file mode 100644 index 0000000000000..66ec79f8f2dab --- /dev/null +++ b/dev-utils/test-patch.py @@ -0,0 +1,466 @@ +#!/usr/bin/env python +# +# 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. +# + +# +# Pre Commit Hook for running tests and updating JIRA +# +# Original version was copied from SQOOP project. +# +import sys, os, re, urllib2, base64, subprocess, tempfile, shutil +import json +import datetime +from optparse import OptionParser + +tmp_dir = None +BASE_JIRA_URL = 'https://issues.apache.org/jira' +BRANCHES = ["trunk", "0.7", "0.7.0", "0.7.1", "0.7.2", "0.8", "0.8.1", "0.8.2"] + +# Write output to file +def write_file(filename, content): + with open(filename, "w") as text_file: + text_file.write(content) + +# Guess branch for given versions +# +# Return None if detects that JIRA belongs to more than one branch +def kafka_guess_branch(versions): + if not versions: + return BRANCHES[0] + + for version in versions: + for branch in BRANCHES: + if version == branch: + return branch + + return BRANCHES[0] + +# Verify supported branch +def kafka_verify_branch(branch): + return branch in BRANCHES + +def execute(cmd, log=True): + if log: + print "INFO: Executing %s" % (cmd) + return subprocess.call(cmd, shell=True) + +def jenkins_link_for_jira(name, endpoint): + if "BUILD_URL" in os.environ: + return "[%s|%s%s]" % (name, os.environ['BUILD_URL'], endpoint) + else: + return name + +def jenkins_file_link_for_jira(name, file): + return jenkins_link_for_jira(name, "artifact/patch-process/%s" % file) + +def jira_request(result, url, username, password, data, headers): + request = urllib2.Request(url, data, headers) + print "INFO: URL = %s, Username = %s, data = %s, headers = %s" % (url, username, data, str(headers)) + if username and password: + base64string = base64.encodestring('%s:%s' % (username, password)).replace('\n', '') + request.add_header("Authorization", "Basic %s" % base64string) + return urllib2.urlopen(request) + +def jira_get_defect_html(result, defect, username, password): + url = "%s/browse/%s" % (BASE_JIRA_URL, defect) + return jira_request(result, url, username, password, None, {}).read() + +def jira_get_defect(result, defect, username, password): + url = "%s/rest/api/2/issue/%s" % (BASE_JIRA_URL, defect) + return jira_request(result, url, username, password, None, {}).read() + +def jira_generate_comment(result, branch): + body = [ "Testing file [%s|%s] against branch %s took %s." % (result.attachment.split('/')[-1] , result.attachment, branch, datetime.datetime.now() - result.start_time) ] + body += [ "" ] + if result._fatal: + result._error = [ result._fatal ] + result._error + if result._error: + count = len(result._error) + if count == 1: + body += [ "{color:red}Overall:{color} -1 due to an error" ] + else: + body += [ "{color:red}Overall:{color} -1 due to %d errors" % (count) ] + else: + body += [ "{color:green}Overall:{color} +1 all checks pass" ] + body += [ "" ] + for error in result._error: + body += [ "{color:red}ERROR:{color} %s" % (error.replace("\n", "\\n")) ] + for info in result._info: + body += [ "INFO: %s" % (info.replace("\n", "\\n")) ] + for success in result._success: + body += [ "{color:green}SUCCESS:{color} %s" % (success.replace("\n", "\\n")) ] + if "BUILD_URL" in os.environ: + body += [ "" ] + body += [ "Console output is available %s." % (jenkins_link_for_jira("here", "console")) ] + body += [ "" ] + body += [ "This message is automatically generated." ] + return "\\n".join(body) + +def jira_post_comment(result, defect, branch, username, password): + url = "%s/rest/api/2/issue/%s/comment" % (BASE_JIRA_URL, defect) + + # Generate body for the comment and save it to a file + body = jira_generate_comment(result, branch) + write_file("%s/jira-comment.txt" % output_dir, body.replace("\\n", "\n")) + + # Send the comment to the JIRA + body = "{\"body\": \"%s\"}" % body + headers = {'Content-Type' : 'application/json'} + response = jira_request(result, url, username, password, body, headers) + body = response.read() + if response.code != 201: + msg = """Request for %s failed: + URL = '%s' + Code = '%d' + Comment = '%s' + Response = '%s' + """ % (defect, url, response.code, comment, body) + print "FATAL: %s" % (msg) + sys.exit(1) + +# hack (from hadoop) but REST api doesn't list attachments? +def jira_get_attachment(result, defect, username, password): + html = jira_get_defect_html(result, defect, username, password) + escaped_colon = re.escape("%3A") + pattern = "(/secure/attachment/[0-9]+/(bug)?%s[0-9\-]*((\.|-)v?[0-9]+)?\.(patch|txt|patch\.txt))" % (re.escape(defect)) + kafka_pattern = "(/secure/attachment/[0-9]+/(bug)?%s_[0-9]+-[0-9]+-[0-9]+_[0-9]+%s[0-9]+%s[0-9]+[0-9\-]*((\.|-)v?[0-9]+)?\.(patch|txt|patch\.txt))" % (re.escape(defect), escaped_colon, escaped_colon) + matches = [] + for match in re.findall(kafka_pattern, html, re.IGNORECASE) or re.findall(pattern, html, re.IGNORECASE): + matches += [ match[0] ] + if matches: + matches.sort() + return "%s%s" % (BASE_JIRA_URL, matches.pop()) + return None + +# Get versions from JIRA JSON object +def json_get_version(json): + versions = [] + + # Load affectedVersion field + for version in json.get("fields").get("versions"): + versions = versions + [version.get("name").strip()] + + # Load fixVersion field + for version in json.get("fields").get("fixVersions"): + versions = versions + [version.get("name").strip()] + + if not versions: + print "No Affected or Fixed version found in JIRA" + + return versions + +def git_cleanup(): + rc = execute("git clean -d -f", False) + if rc != 0: + print "ERROR: git clean failed" + rc = execute("git reset --hard HEAD", False) + if rc != 0: + print "ERROR: git reset failed" + +def git_checkout(result, branch): + if not branch: + result.fatal("Branch wasn't specified nor was correctly guessed") + return + + if execute("git checkout %s" % (branch)) != 0: + result.fatal("git checkout %s failed" % branch) + if execute("git clean -d -f") != 0: + result.fatal("git clean failed") + if execute("git reset --hard HEAD") != 0: + result.fatal("git reset failed") + if execute("git fetch origin") != 0: + result.fatal("git fetch failed") + if execute("git merge --ff-only origin/%s" % (branch)): + result.fatal("git merge failed") + +def git_apply(result, cmd, patch_file, strip, output_dir): + output_file = "%s/apply.txt" % (output_dir) + rc = execute("%s -p%s < %s 1>%s 2>&1" % (cmd, strip, patch_file, output_file)) + output = "" + if os.path.exists(output_file): + with open(output_file) as fh: + output = fh.read() + if rc == 0: + if output: + result.success("Patch applied, but there has been warnings:\n{code}%s{code}\n" % (output)) + else: + result.success("Patch applied correctly") + else: + result.fatal("failed to apply patch (exit code %d):\n{code}%s{code}\n" % (rc, output)) + +def static_test(result, patch_file, output_dir): + output_file = "%s/static-test.txt" % (output_dir) + rc = execute("grep '^+++.*/test' %s 1>%s 2>&1" % (patch_file, output_file)) + if rc == 0: + result.success("Patch add/modify test case") + else: + result.error("Patch does not add/modify any test case") + +def gradle_bootstrap(result, output_dir): + rc = execute("gradle 1>%s/bootstrap.txt 2>&1" % output_dir) + if rc == 0: + result.success("Gradle bootstrap was successful") + else: + result.fatal("failed to bootstrap project (exit code %d, %s)" % (rc, jenkins_file_link_for_jira("report", "bootstrap.txt"))) + +def gradle_clean(result, output_dir): + rc = execute("./gradlew clean 1>%s/clean.txt 2>&1" % output_dir) + if rc == 0: + result.success("Clean was successful") + else: + result.fatal("failed to clean project (exit code %d, %s)" % (rc, jenkins_file_link_for_jira("report", "clean.txt"))) + +def gradle_install(result, output_dir): + rc = execute("./gradlew jarAll 1>%s/install.txt 2>&1" % output_dir) + if rc == 0: + result.success("Patch compiled") + else: + result.fatal("failed to build with patch (exit code %d, %s)" % (rc, jenkins_file_link_for_jira("report", "install.txt"))) + +def checkstyleMain(result, output_dir): + rc = execute("./gradlew checkstyleMain 1>%s/checkstyleMain.txt 2>&1" % output_dir) + if rc == 0: + result.success("Checked style for Main") + else: + result.fatal("checkstyleMain failed with patch (exit code %d, %s)" % (rc, jenkins_file_link_for_jira("report", "checkstyleMain.txt"))) + +def checkstyleTest(result, output_dir): + rc = execute("./gradlew checkstyleTest 1>%s/checkstyleTest.txt 2>&1" % output_dir) + if rc == 0: + result.success("Checked style for Test") + else: + result.fatal("checkstyleTest failed with patch (exit code %d, %s)" % (rc, jenkins_file_link_for_jira("report", "checkstyleTest.txt"))) + +def gradle_test(result, output_dir): + run_gradle_test("testAll", "unit", result, output_dir) + +def run_gradle_test(command, test_type, result, output_dir): + rc = execute("./gradlew %s 1>%s/test_%s.txt 2>&1" % (command, output_dir, test_type)) + if rc == 0: + result.success("All %s tests passed" % test_type) + else: + result.error("Some %s tests failed (%s)" % (test_type, jenkins_file_link_for_jira("report", "test_%s.txt" % test_type))) + failed_tests = [] + fd = open("%s/test_%s.txt" % (output_dir, test_type), "r") + for line in fd: + if "FAILED" in line and " > " in line: + failed_tests += [line] + fd.close() + for failed_test in set(failed_tests): + result.error("Failed %s test: {{%s}}" % (test_type, failed_test)) + +def clean_folder(folder): + for the_file in os.listdir(folder): + file_path = os.path.join(folder, the_file) + try: + if os.path.isfile(file_path): + os.unlink(file_path) + except Exception, e: + print e + +class Result(object): + def __init__(self): + self._error = [] + self._info = [] + self._success = [] + self._fatal = None + self.exit_handler = None + self.attachment = "Not Found" + self.start_time = datetime.datetime.now() + def error(self, msg): + self._error.append(msg) + def info(self, msg): + self._info.append(msg) + def success(self, msg): + self._success.append(msg) + def fatal(self, msg): + self._fatal = msg + self.exit_handler() + self.exit() + def exit(self): + git_cleanup() + global tmp_dir + global copy_output_dir + global output_dir + if copy_output_dir: + print "INFO: Moving output to %s" % (copy_output_dir) + os.renames(output_dir, copy_output_dir) + tmp_dir = None + if tmp_dir: + print "INFO: output is located %s" % (tmp_dir) + sys.exit(0) + +usage = "usage: %prog [options]" +parser = OptionParser(usage) +parser.add_option("--branch", dest="branch", + help="Local git branch to test against", metavar="trunk") +parser.add_option("--defect", dest="defect", + help="Defect name", metavar="KAFKA-1856") +parser.add_option("--file", dest="filename", + help="Test patch file", metavar="FILE") +parser.add_option("--run-tests", dest="run_tests", + help="Run Tests", action="store_true") +parser.add_option("--username", dest="username", + help="JIRA Username", metavar="USERNAME", default="kafkaqa") +parser.add_option("--output", dest="output_dir", + help="Directory to write output", metavar="DIRECTORY") +parser.add_option("--post-results", dest="post_results", + help="Post results to JIRA (only works in defect mode)", action="store_true") +parser.add_option("--password", dest="password", + help="JIRA Password", metavar="PASSWORD") +parser.add_option("--patch-command", dest="patch_cmd", default="git apply", + help="Patch command such as `git apply' or `patch'", metavar="COMMAND") +parser.add_option("-p", "--strip", dest="strip", default="1", + help="Remove leading slashes from diff paths", metavar="N") +parser.add_option("--get-latest-patch", dest="get_latest_patch", + help="Get the latest patch attached to JIRA", action="store_true") + +(options, args) = parser.parse_args() +if not (options.defect or options.filename): + print "FATAL: Either --defect or --file is required." + sys.exit(1) + +if options.defect and options.filename: + print "FATAL: Both --defect and --file cannot be specified." + sys.exit(1) + +if options.post_results and not options.password: + print "FATAL: --post-results requires --password" + sys.exit(1) + +if options.get_latest_patch and not options.defect: + print "FATAL: --get-latest-patch requires --defect" + sys.exit(1) + +branch = options.branch +if options.output_dir and not options.output_dir.startswith('/'): + print "INFO: A temporary staging dir for output will be used to avoid deletion of output files during 'git reset'" + copy_output_dir = options.output_dir + output_dir = None +else: + output_dir = options.output_dir + copy_output_dir = None +defect = options.defect +username = options.username +password = options.password +run_tests = options.run_tests +post_results = options.post_results +strip = options.strip +get_latest_patch = options.get_latest_patch +patch_cmd = options.patch_cmd +result = Result() + +if output_dir and os.path.isdir(output_dir): + clean_folder(output_dir) +if copy_output_dir and os.path.isdir(copy_output_dir): + clean_folder(copy_output_dir) + +# Default exit handler in case that we do not want to submit results to JIRA +def log_and_exit(): + # Write down comment generated for jira (won't be posted) + write_file("%s/jira-comment.txt" % output_dir, jira_generate_comment(result, branch).replace("\\n", "\n")) + + if result._fatal: + print "FATAL: %s" % (result._fatal) + for error in result._error: + print "ERROR: %s" % (error) + for info in result._info: + print "INFO: %s" % (info) + for success in result._success: + print "SUCCESS: %s" % (success) + result.exit() + +result.exit_handler = log_and_exit + +if post_results: + def post_jira_comment_and_exit(): + jira_post_comment(result, defect, branch, username, password) + result.exit() + result.exit_handler = post_jira_comment_and_exit + +if not output_dir: + tmp_dir = tempfile.mkdtemp() + output_dir = tmp_dir + +if output_dir.endswith("/"): + output_dir = output_dir[:-1] + +if output_dir and not os.path.isdir(output_dir): + os.makedirs(output_dir) + +def get_latest_patch(): + global jira_json, json, versions, branch, attachment, patch_contents, patch_file, fh + print "Defect: %s" % defect + jira_json = jira_get_defect(result, defect, username, password) + json = json.loads(jira_json) + # JIRA must be in Patch Available state + if '"Patch Available"' not in jira_json: + print "ERROR: Defect %s not in patch available state" % (defect) + sys.exit(1) + + # If branch is not specified, let's try to guess it from JIRA details + if not branch: + versions = json_get_version(json) + branch = kafka_guess_branch(versions) + if not branch: + print "ERROR: Can't guess branch name from %s" % (versions) + sys.exit(1) + else: + print "INFO: Guessed branch as %s" % (branch) + attachment = jira_get_attachment(result, defect, username, password) + if not attachment: + print "ERROR: No attachments found for %s" % (defect) + sys.exit(1) + result.attachment = attachment + patch_contents = jira_request(result, result.attachment, username, password, None, {}).read() + patch_file = "%s/%s.patch" % (output_dir, defect) + with open(patch_file, 'a') as fh: + fh.write(patch_contents) + +if defect: + # If defect parameter is specified let's download the latest attachment + get_latest_patch() + if options.get_latest_patch: + print "Saving latest attachment of %s as %s/%s.patch" % (defect, output_dir, defect) + sys.exit(0) +elif options.filename: + patch_file = options.filename +else: + raise Exception("Not reachable") + +# Verify that we are on supported branch +if not kafka_verify_branch(branch): + print "ERROR: Unsupported branch %s" % (branch) + sys.exit(1) + +gradle_bootstrap(result, output_dir) +gradle_clean(result, output_dir) +git_checkout(result, branch) +git_apply(result, patch_cmd, patch_file, strip, output_dir) +static_test(result, patch_file, output_dir) +gradle_bootstrap(result, output_dir) +gradle_install(result, output_dir) +checkstyleMain(result, output_dir) +checkstyleTest(result, output_dir) +if run_tests: + gradle_test(result, output_dir) +else: + result.info("patch applied and built but tests did not execute") + +result.exit_handler() diff --git a/doap_Kafka.rdf b/doap_Kafka.rdf new file mode 100644 index 0000000000000..5381e9f8ce4c2 --- /dev/null +++ b/doap_Kafka.rdf @@ -0,0 +1,57 @@ + + + + + + 2014-04-12 + + Apache Kafka + + + Apache Kafka is a distributed, fault tolerant, publish-subscribe messaging. + A single Kafka broker can handle hundreds of megabytes of reads and writes per second from thousands of clients. Kafka is designed to allow a single cluster to serve as the central data backbone for a large organization. It can be elastically and transparently expanded without downtime. Data streams are partitioned and spread over a cluster of machines to allow data streams larger than the capability of any single machine and to allow clusters of co-ordinated consumers. Kafka has a modern cluster-centric design that offers strong durability and fault-tolerance guarantees. Messages are persisted on disk and replicated within the cluster to prevent data loss. Each broker can handle terabytes of messages without performance impact. + + + + Scala + + + + Kafka 0.8.1 + 2014-03-12 + 0.8.1 + + + + + + + + + + + Jun Rao + + + + + diff --git a/examples/README b/examples/README index 61de2868de29e..f6e3410a2a0aa 100644 --- a/examples/README +++ b/examples/README @@ -1,19 +1,9 @@ This directory contains examples of client code that uses kafka. -The default target for ant is kafka.examples.KafkaConsumerProducerDemo which sends and receives -messages from Kafka server. +To run the demo: -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-examples - 4. execute run -> run - 5. For simple consumer demo, select option 1 - For unlimited producer-consumer run, select option 2 - -To run the demo using scripts: - - 1. Start Zookeeper and the Kafka server - 2. For simple consumer demo, run bin/java-simple-consumer-demo.sh - 3. For unlimited producer-consumer run, run bin/java-producer-consumer-demo.sh + 2. For simple consumer demo, `run bin/java-simple-consumer-demo.sh` + 3. For unlimited sync-producer-consumer run, `run bin/java-producer-consumer-demo.sh sync` + 4. For unlimited async-producer-consumer run, `run bin/java-producer-consumer-demo.sh` diff --git a/examples/bin/java-producer-consumer-demo.sh b/examples/bin/java-producer-consumer-demo.sh index 29e01c2dcf823..fd25e5955397e 100755 --- a/examples/bin/java-producer-consumer-demo.sh +++ b/examples/bin/java-producer-consumer-demo.sh @@ -16,44 +16,7 @@ base_dir=$(dirname $0)/../.. -for file in $base_dir/project/boot/scala-2.8.0/lib/*.jar; -do - if [ ${file##*/} != "sbt-launch.jar" ]; then - CLASSPATH=$CLASSPATH:$file - fi -done - -for file in $base_dir/core/lib_managed/scala_2.8.0/compile/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/core/lib/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/core/target/scala_2.8.0/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/examples/target/scala_2.8.0/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -echo $CLASSPATH - -if [ -z "$KAFKA_PERF_OPTS" ]; then - KAFKA_OPTS="-Xmx512M -server -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=3333 -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" -fi - -if [ -z "$JAVA_HOME" ]; then - JAVA="java" -else - JAVA="$JAVA_HOME/bin/java" +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" fi - -$JAVA $KAFKA_OPTS -cp $CLASSPATH kafka.examples.KafkaConsumerProducerDemo $@ - +exec $base_dir/bin/kafka-run-class.sh kafka.examples.KafkaConsumerProducerDemo $@ diff --git a/examples/bin/java-simple-consumer-demo.sh b/examples/bin/java-simple-consumer-demo.sh index 4716a098c7d40..c4f103e827e69 100755 --- a/examples/bin/java-simple-consumer-demo.sh +++ b/examples/bin/java-simple-consumer-demo.sh @@ -16,44 +16,7 @@ base_dir=$(dirname $0)/../.. -for file in $base_dir/project/boot/scala-2.8.0/lib/*.jar; -do - if [ ${file##*/} != "sbt-launch.jar" ]; then - CLASSPATH=$CLASSPATH:$file - fi -done - -for file in $base_dir/core/lib_managed/scala_2.8.0/compile/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/core/lib/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/core/target/scala_2.8.0/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/examples/target/scala_2.8.0/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -echo $CLASSPATH - -if [ -z "$KAFKA_PERF_OPTS" ]; then - KAFKA_OPTS="-Xmx512M -server -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=3333 -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" -fi - -if [ -z "$JAVA_HOME" ]; then - JAVA="java" -else - JAVA="$JAVA_HOME/bin/java" +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" fi - -$JAVA $KAFKA_OPTS -cp $CLASSPATH kafka.examples.SimpleConsumerDemo $@ - +exec $base_dir/bin/kafka-run-class.sh kafka.examples.SimpleConsumerDemo $@ diff --git a/examples/build.sbt b/examples/build.sbt deleted file mode 100644 index d12d70142f6ec..0000000000000 --- a/examples/build.sbt +++ /dev/null @@ -1,3 +0,0 @@ -name := "kafka-java-examples" - -crossPaths := false diff --git a/examples/src/main/java/kafka/examples/Consumer.java b/examples/src/main/java/kafka/examples/Consumer.java index 13135b954f307..8af64d886d9ff 100644 --- a/examples/src/main/java/kafka/examples/Consumer.java +++ b/examples/src/main/java/kafka/examples/Consumer.java @@ -17,14 +17,15 @@ package kafka.examples; +import java.nio.ByteBuffer; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; import kafka.consumer.ConsumerConfig; -import kafka.consumer.ConsumerIterator; import kafka.consumer.KafkaStream; import kafka.javaapi.consumer.ConsumerConnector; +import kafka.message.MessageAndMetadata; public class Consumer extends Thread @@ -54,11 +55,13 @@ private static ConsumerConfig createConsumerConfig() public void run() { Map topicCountMap = new HashMap(); - topicCountMap.put(topic, new Integer(1)); + topicCountMap.put(topic, 1); Map>> consumerMap = consumer.createMessageStreams(topicCountMap); - KafkaStream stream = consumerMap.get(topic).get(0); - ConsumerIterator it = stream.iterator(); - while(it.hasNext()) - System.out.println(new String(it.next().message())); + KafkaStream stream = consumerMap.get(topic).get(0); + for (MessageAndMetadata messageAndMetadata : stream) { + System.out.println("Received message: (" + ByteBuffer.wrap(messageAndMetadata.key()).getInt() + + ", " + + "" + new String(messageAndMetadata.message()) + ")"); + } } } diff --git a/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java b/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java index 1239394190fe5..e96991a29032e 100644 --- a/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java +++ b/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java @@ -20,7 +20,8 @@ public class KafkaConsumerProducerDemo implements KafkaProperties { public static void main(String[] args) { - Producer producerThread = new Producer(KafkaProperties.topic); + final boolean isAsync = args.length > 0 ? !args[0].trim().toLowerCase().equals("sync") : true; + Producer producerThread = new Producer(KafkaProperties.topic, isAsync); producerThread.start(); Consumer consumerThread = new Consumer(KafkaProperties.topic); diff --git a/examples/src/main/java/kafka/examples/Producer.java b/examples/src/main/java/kafka/examples/Producer.java index 96e98933148d0..ccc9925caecd9 100644 --- a/examples/src/main/java/kafka/examples/Producer.java +++ b/examples/src/main/java/kafka/examples/Producer.java @@ -18,33 +18,88 @@ import java.util.Properties; -import kafka.producer.KeyedMessage; -import kafka.producer.ProducerConfig; +import java.util.concurrent.ExecutionException; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; public class Producer extends Thread { - private final kafka.javaapi.producer.Producer producer; + private final KafkaProducer producer; private final String topic; - private final Properties props = new Properties(); + private final Boolean isAsync; - public Producer(String topic) + public Producer(String topic, Boolean isAsync) { - props.put("serializer.class", "kafka.serializer.StringEncoder"); - props.put("metadata.broker.list", "localhost:9092"); - // Use random partitioner. Don't need the key type. Just set it to Integer. - // The message is of type String. - producer = new kafka.javaapi.producer.Producer(new ProducerConfig(props)); + Properties props = new Properties(); + props.put("bootstrap.servers", "localhost:9092"); + props.put("client.id", "DemoProducer"); + props.put("key.serializer", "org.apache.kafka.common.serialization.IntegerSerializer"); + props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + producer = new KafkaProducer(props); this.topic = topic; + this.isAsync = isAsync; } - + public void run() { int messageNo = 1; while(true) { - String messageStr = new String("Message_" + messageNo); - producer.send(new KeyedMessage(topic, messageStr)); - messageNo++; + String messageStr = "Message_" + messageNo; + long startTime = System.currentTimeMillis(); + if (isAsync) { // Send asynchronously + producer.send(new ProducerRecord(topic, + messageNo, + messageStr), new DemoCallBack(startTime, messageNo, messageStr)); + } else { // Send synchronously + try { + producer.send(new ProducerRecord(topic, + messageNo, + messageStr)).get(); + System.out.println("Sent message: (" + messageNo + ", " + messageStr + ")"); + } catch (InterruptedException e) { + e.printStackTrace(); + } catch (ExecutionException e) { + e.printStackTrace(); + } + } + ++messageNo; } } +} + +class DemoCallBack implements Callback { + + private long startTime; + private int key; + private String message; + public DemoCallBack(long startTime, int key, String message) { + this.startTime = startTime; + this.key = key; + this.message = message; + } + + /** + * A callback method the user can implement to provide asynchronous handling of request completion. This method will + * be called when the record sent to the server has been acknowledged. Exactly one of the arguments will be + * non-null. + * + * @param metadata The metadata for the record that was sent (i.e. the partition and offset). Null if an error + * occurred. + * @param exception The exception thrown during processing of this record. Null if no error occurred. + */ + public void onCompletion(RecordMetadata metadata, Exception exception) { + long elapsedTime = System.currentTimeMillis() - startTime; + if (metadata != null) { + System.out.println( + "message(" + key + ", " + message + ") sent to partition(" + metadata.partition() + + "), " + + "offset(" + metadata.offset() + ") in " + elapsedTime + " ms"); + } else { + exception.printStackTrace(); + } + } } diff --git a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java index c79192c5c195d..c43b46144e6d1 100644 --- a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java +++ b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java @@ -22,7 +22,7 @@ import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.message.ByteBufferMessageSet; @@ -42,9 +42,9 @@ private static void printMessages(ByteBufferMessageSet messageSet) throws Unsupp } private static void generateData() { - Producer producer2 = new Producer(KafkaProperties.topic2); + Producer producer2 = new Producer(KafkaProperties.topic2, false); producer2.start(); - Producer producer3 = new Producer(KafkaProperties.topic3); + Producer producer3 = new Producer(KafkaProperties.topic3, false); producer3.start(); try { Thread.sleep(1000); @@ -68,13 +68,12 @@ public static void main(String[] args) throws Exception { .addFetch(KafkaProperties.topic2, 0, 0L, 100) .build(); FetchResponse fetchResponse = simpleConsumer.fetch(req); - printMessages((ByteBufferMessageSet) fetchResponse.messageSet(KafkaProperties.topic2, 0)); + printMessages(fetchResponse.messageSet(KafkaProperties.topic2, 0)); System.out.println("Testing single multi-fetch"); - Map> topicMap = new HashMap>() {{ - put(KafkaProperties.topic2, new ArrayList(){{ add(0); }}); - put(KafkaProperties.topic3, new ArrayList(){{ add(0); }}); - }}; + Map> topicMap = new HashMap>(); + topicMap.put(KafkaProperties.topic2, Collections.singletonList(0)); + topicMap.put(KafkaProperties.topic3, Collections.singletonList(0)); req = new FetchRequestBuilder() .clientId(KafkaProperties.clientId) .addFetch(KafkaProperties.topic2, 0, 0L, 100) @@ -86,7 +85,7 @@ public static void main(String[] args) throws Exception { String topic = entry.getKey(); for ( Integer offset : entry.getValue()) { System.out.println("Response from fetch request no: " + ++fetchReq); - printMessages((ByteBufferMessageSet) fetchResponse.messageSet(topic, offset)); + printMessages(fetchResponse.messageSet(topic, offset)); } } } diff --git a/gradle.properties b/gradle.properties index ad7a2f0f62930..cd4050b983c4c 100644 --- a/gradle.properties +++ b/gradle.properties @@ -14,11 +14,7 @@ # limitations under the License. group=org.apache.kafka -version=0.8.1 -scalaVersion=2.8.0 +version=0.8.3-SNAPSHOT +scalaVersion=2.10.5 task=build - -#mavenUrl=file://localhost/tmp/maven -mavenUrl=http://your.maven.repository -mavenUsername=your.username -mavenPassword=your.password +org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m diff --git a/gradle/buildscript.gradle b/gradle/buildscript.gradle index 225e0a82708bc..047632b18c934 100644 --- a/gradle/buildscript.gradle +++ b/gradle/buildscript.gradle @@ -1,3 +1,18 @@ +// 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. + repositories { repositories { // For license plugin. @@ -6,7 +21,3 @@ repositories { } } } - -dependencies { - classpath 'nl.javadude.gradle.plugins:license-gradle-plugin:0.6.1' -} diff --git a/gradle/license.gradle b/gradle/license.gradle deleted file mode 100644 index b4b62ebe27771..0000000000000 --- a/gradle/license.gradle +++ /dev/null @@ -1,9 +0,0 @@ -subprojects { - apply plugin: 'license' - - license { - header rootProject.file('HEADER') - // Skip Twitter bootstrap JS and CSS. - skipExistingHeaders = true - } -} diff --git a/gradle/rat.gradle b/gradle/rat.gradle new file mode 100644 index 0000000000000..d62b3722a4f78 --- /dev/null +++ b/gradle/rat.gradle @@ -0,0 +1,115 @@ +/* + * 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. + */ + +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.Task +import org.gradle.api.internal.project.IsolatedAntBuilder + +apply plugin: RatPlugin + +class RatTask extends DefaultTask { + @Input + List excludes + + def reportPath = 'build/rat' + def stylesheet = 'gradle/resources/rat-output-to-html.xsl' + def xmlReport = reportPath + '/rat-report.xml' + def htmlReport = reportPath + '/rat-report.html' + + def generateXmlReport(File reportDir) { + def antBuilder = services.get(IsolatedAntBuilder) + def ratClasspath = project.configurations.rat + antBuilder.withClasspath(ratClasspath).execute { + ant.taskdef(resource: 'org/apache/rat/anttasks/antlib.xml') + ant.report(format: 'xml', reportFile: xmlReport) { + fileset(dir: ".") { + patternset { + excludes.each { + exclude(name: it) + } + } + } + } + } + } + + def printUnknownFiles() { + def ratXml = new XmlParser().parse(xmlReport) + def unknownLicenses = 0 + ratXml.resource.each { resource -> + if (resource.'license-approval'.@name[0] == "false") { + println('Unknown license: ' + resource.@name) + unknownLicenses++ + } + } + if (unknownLicenses > 0) { + throw new GradleException("Found " + unknownLicenses + " files with " + + "unknown licenses.") + } + } + + def generateHtmlReport() { + def antBuilder = services.get(IsolatedAntBuilder) + def ratClasspath = project.configurations.rat + antBuilder.withClasspath(ratClasspath).execute { + ant.xslt( + in: xmlReport, + style: stylesheet, + out: htmlReport, + classpath: ratClasspath) + } + println('Rat report: ' + htmlReport) + } + + @TaskAction + def rat() { + File reportDir = new File(reportPath) + if (!reportDir.exists()) { + reportDir.mkdirs() + } + generateXmlReport(reportDir) + printUnknownFiles() + generateHtmlReport() + } +} + +class RatPlugin implements Plugin { + void apply(Project project) { + configureDependencies(project) + project.plugins.apply(JavaPlugin); + Task ratTask = project.task("rat", + type: RatTask, + group: 'Build', + description: 'Runs Apache Rat checks.') + project.tasks[JavaPlugin.TEST_TASK_NAME].dependsOn ratTask + } + + void configureDependencies(final Project project) { + project.configurations { + rat + } + project.repositories { + mavenCentral() + } + project.dependencies { + rat 'org.apache.rat:apache-rat-tasks:0.11' + } + } +} diff --git a/gradle/resources/rat-output-to-html.xsl b/gradle/resources/rat-output-to-html.xsl new file mode 100644 index 0000000000000..97ea7a1d73f4e --- /dev/null +++ b/gradle/resources/rat-output-to-html.xsl @@ -0,0 +1,206 @@ + + + + + + + + + + + + + + + + + + + + + + +

            Rat Report

            +

            This HTML version (yes, it is!) is generated from the RAT xml reports using Saxon9B. All the outputs required are displayed below, similar to the .txt version. + This is obviously a work in progress; and a prettier, easier to read and manage version will be available soon

            +
            + + + + + + + + + + + + + + + + + + + + + + + + + + +
            +Table 1: A snapshot summary of this rat report. +
            Notes: Binaries: Archives: Standards:
            Apache Licensed: Generated Documents:
            Note: JavaDocs are generated and so license header is optionalNote: Generated files do not require license headers
            Unknown Licenses - or files without a license. Unknown Licenses - or files without a license.
            +
            +
            +

            Unapproved Licenses:

            + + + +
            + + +
            +
            + +

            Archives:

            + + + + +
            +
            +
            + +

            + Files with Apache License headers will be marked AL
            + Binary files (which do not require AL headers) will be marked B
            + Compressed archives will be marked A
            + Notices, licenses etc will be marked N
            +

            + + + + ! + + + + N + A + B + + !!!!! + + +
            + + +
            +
            + +

            Printing headers for files without AL header...

            + + + +

            + +
            +
            +
            + + + +
            + + +
            +

            Resource:

            + +
            +
            + + + +

            First few lines of non-compliant file

            +

            + +

            +
            +

            Other Info:

            +
            + + + Header Type: +
            +
            + + + License Family: +
            +
            + + + License Approval: +
            +
            + + + Type: +
            +
            + + + +
            diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar deleted file mode 100644 index a7634b071cb25..0000000000000 Binary files a/gradle/wrapper/gradle-wrapper.jar and /dev/null differ diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties deleted file mode 100644 index 610282a699afc..0000000000000 --- a/gradle/wrapper/gradle-wrapper.properties +++ /dev/null @@ -1,6 +0,0 @@ -#Thu Jul 11 22:18:11 PDT 2013 -distributionBase=GRADLE_USER_HOME -distributionPath=wrapper/dists -zipStoreBase=GRADLE_USER_HOME -zipStorePath=wrapper/dists -distributionUrl=http\://services.gradle.org/distributions/gradle-1.6-bin.zip diff --git a/gradlew b/gradlew index c312b910b570f..91a7e269e19df 100755 --- a/gradlew +++ b/gradlew @@ -7,7 +7,7 @@ ############################################################################## # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. -DEFAULT_JVM_OPTS="-Xmx1024m -Xms256m -XX:MaxPermSize=512m" +DEFAULT_JVM_OPTS="" APP_NAME="Gradle" APP_BASE_NAME=`basename "$0"` diff --git a/gradlew.bat b/gradlew.bat new file mode 100644 index 0000000000000..aec99730b4e8f --- /dev/null +++ b/gradlew.bat @@ -0,0 +1,90 @@ +@if "%DEBUG%" == "" @echo off +@rem ########################################################################## +@rem +@rem Gradle startup script for Windows +@rem +@rem ########################################################################## + +@rem Set local scope for the variables with windows NT shell +if "%OS%"=="Windows_NT" setlocal + +@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +set DEFAULT_JVM_OPTS= + +set DIRNAME=%~dp0 +if "%DIRNAME%" == "" set DIRNAME=. +set APP_BASE_NAME=%~n0 +set APP_HOME=%DIRNAME% + +@rem Find java.exe +if defined JAVA_HOME goto findJavaFromJavaHome + +set JAVA_EXE=java.exe +%JAVA_EXE% -version >NUL 2>&1 +if "%ERRORLEVEL%" == "0" goto init + +echo. +echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. +echo. +echo Please set the JAVA_HOME variable in your environment to match the +echo location of your Java installation. + +goto fail + +:findJavaFromJavaHome +set JAVA_HOME=%JAVA_HOME:"=% +set JAVA_EXE=%JAVA_HOME%/bin/java.exe + +if exist "%JAVA_EXE%" goto init + +echo. +echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% +echo. +echo Please set the JAVA_HOME variable in your environment to match the +echo location of your Java installation. + +goto fail + +:init +@rem Get command-line arguments, handling Windowz variants + +if not "%OS%" == "Windows_NT" goto win9xME_args +if "%@eval[2+2]" == "4" goto 4NT_args + +:win9xME_args +@rem Slurp the command line arguments. +set CMD_LINE_ARGS= +set _SKIP=2 + +:win9xME_args_slurp +if "x%~1" == "x" goto execute + +set CMD_LINE_ARGS=%* +goto execute + +:4NT_args +@rem Get arguments from the 4NT Shell from JP Software +set CMD_LINE_ARGS=%$ + +:execute +@rem Setup the command line + +set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar + +@rem Execute Gradle +"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% + +:end +@rem End local scope for the variables with windows NT shell +if "%ERRORLEVEL%"=="0" goto mainEnd + +:fail +rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of +rem the _cmd.exe /c_ return code! +if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 +exit /b 1 + +:mainEnd +if "%OS%"=="Windows_NT" endlocal + +:omega diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py new file mode 100644 index 0000000000000..4cc82e053dab2 --- /dev/null +++ b/kafka-merge-pr.py @@ -0,0 +1,442 @@ +#!/usr/bin/env python + +# +# 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. +# + +# Utility for creating well-formed pull request merges and pushing them to Apache. This script is a modified version +# of the one created by the Spark project (https://github.com/apache/spark/blob/master/dev/merge_spark_pr.py). +# +# Usage: ./kafka-merge-pr.py (see config env vars below) +# +# This utility assumes you already have local a kafka git folder and that you +# have added remotes corresponding to both: +# (i) the github apache kafka mirror and +# (ii) the apache kafka git repo. + +import json +import os +import re +import subprocess +import sys +import urllib2 + +try: + import jira.client + JIRA_IMPORTED = True +except ImportError: + JIRA_IMPORTED = False + +PROJECT_NAME = "kafka" + +CAPITALIZED_PROJECT_NAME = "kafka".upper() + +# Location of the local git repository +REPO_HOME = os.environ.get("%s_HOME" % CAPITALIZED_PROJECT_NAME, os.getcwd()) +# Remote name which points to the GitHub site +PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") +# Remote name which points to Apache git +PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") +# ASF JIRA username +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "") +# ASF JIRA password +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "") + +GITHUB_USER = os.environ.get("GITHUB_USER", "apache") +GITHUB_BASE = "https://github.com/%s/%s/pull" % (GITHUB_USER, PROJECT_NAME) +GITHUB_API_BASE = "https://api.github.com/repos/%s/%s" % (GITHUB_USER, PROJECT_NAME) +JIRA_BASE = "https://issues.apache.org/jira/browse" +JIRA_API_BASE = "https://issues.apache.org/jira" +# Prefix added to temporary branches +TEMP_BRANCH_PREFIX = "PR_TOOL" +# TODO Introduce a convention as this is too brittle +RELEASE_BRANCH_PREFIX = "0." + +DEV_BRANCH_NAME="trunk" + +def get_json(url): + try: + return json.load(urllib2.urlopen(url)) + except urllib2.HTTPError as e: + print "Unable to fetch URL, exiting: %s" % url + sys.exit(-1) + + +def fail(msg): + print msg + clean_up() + sys.exit(-1) + + +def run_cmd(cmd): + print cmd + if isinstance(cmd, list): + return subprocess.check_output(cmd) + else: + return subprocess.check_output(cmd.split(" ")) + + +def continue_maybe(prompt): + result = raw_input("\n%s (y/n): " % prompt) + if result.lower() != "y": + fail("Okay, exiting") + +def clean_up(): + print "Restoring head pointer to %s" % original_head + run_cmd("git checkout %s" % original_head) + + branches = run_cmd("git branch").replace(" ", "").split("\n") + + for branch in filter(lambda x: x.startswith(TEMP_BRANCH_PREFIX), branches): + print "Deleting local branch %s" % branch + run_cmd("git branch -D %s" % branch) + + +# merge the requested PR and return the merge hash +def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): + pr_branch_name = "%s_MERGE_PR_%s" % (TEMP_BRANCH_PREFIX, pr_num) + target_branch_name = "%s_MERGE_PR_%s_%s" % (TEMP_BRANCH_PREFIX, pr_num, target_ref.upper()) + run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) + run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) + run_cmd("git checkout %s" % target_branch_name) + + had_conflicts = False + try: + run_cmd(['git', 'merge', pr_branch_name, '--squash']) + except Exception as e: + msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" + continue_maybe(msg) + had_conflicts = True + + commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%an <%ae>']).split("\n") + distinct_authors = sorted(set(commit_authors), + key=lambda x: commit_authors.count(x), reverse=True) + primary_author = distinct_authors[0] + commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%h [%an] %s']).split("\n\n") + + merge_message_flags = [] + + merge_message_flags += ["-m", title] + if body is not None: + # We remove @ symbols from the body to avoid triggering e-mails + # to people every time someone creates a public fork of the project. + merge_message_flags += ["-m", body.replace("@", "")] + + authors = "\n".join(["Author: %s" % a for a in distinct_authors]) + + merge_message_flags += ["-m", authors] + + if had_conflicts: + committer_name = run_cmd("git config --get user.name").strip() + committer_email = run_cmd("git config --get user.email").strip() + message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % ( + committer_name, committer_email) + merge_message_flags += ["-m", message] + + # The string "Closes #%s" string is required for GitHub to correctly close the PR + merge_message_flags += [ + "-m", + "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] + for c in commits: + merge_message_flags += ["-m", c] + + run_cmd(['git', 'commit', '--author="%s"' % primary_author] + merge_message_flags) + + continue_maybe("Merge complete (local ref %s). Push to %s?" % ( + target_branch_name, PUSH_REMOTE_NAME)) + + try: + run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, target_branch_name, target_ref)) + except Exception as e: + clean_up() + fail("Exception while pushing: %s" % e) + + merge_hash = run_cmd("git rev-parse %s" % target_branch_name)[:8] + clean_up() + print("Pull request #%s merged!" % pr_num) + print("Merge hash: %s" % merge_hash) + return merge_hash + + +def cherry_pick(pr_num, merge_hash, default_branch): + pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) + if pick_ref == "": + pick_ref = default_branch + + pick_branch_name = "%s_PICK_PR_%s_%s" % (TEMP_BRANCH_PREFIX, pr_num, pick_ref.upper()) + + run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name)) + run_cmd("git checkout %s" % pick_branch_name) + + try: + run_cmd("git cherry-pick -sx %s" % merge_hash) + except Exception as e: + msg = "Error cherry-picking: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and finish the cherry-pick. Finished?" + continue_maybe(msg) + + continue_maybe("Pick complete (local ref %s). Push to %s?" % ( + pick_branch_name, PUSH_REMOTE_NAME)) + + try: + run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, pick_branch_name, pick_ref)) + except Exception as e: + clean_up() + fail("Exception while pushing: %s" % e) + + pick_hash = run_cmd("git rev-parse %s" % pick_branch_name)[:8] + clean_up() + + print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) + print("Pick hash: %s" % pick_hash) + return pick_ref + + +def fix_version_from_branch(branch, versions): + # Note: Assumes this is a sorted (newest->oldest) list of un-released versions + if branch == DEV_BRANCH_NAME: + return versions[0] + else: + return filter(lambda x: x.name.startswith(branch), versions)[-1] + + +def resolve_jira_issue(merge_branches, comment, default_jira_id=""): + asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + + jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) + if jira_id == "": + jira_id = default_jira_id + + try: + issue = asf_jira.issue(jira_id) + except Exception as e: + fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) + + cur_status = issue.fields.status.name + cur_summary = issue.fields.summary + cur_assignee = issue.fields.assignee + if cur_assignee is None: + cur_assignee = "NOT ASSIGNED!!!" + else: + cur_assignee = cur_assignee.displayName + + if cur_status == "Resolved" or cur_status == "Closed": + fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) + print ("=== JIRA %s ===" % jira_id) + print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( + cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) + + versions = asf_jira.project_versions(CAPITALIZED_PROJECT_NAME) + versions = sorted(versions, key=lambda x: x.name, reverse=True) + versions = filter(lambda x: x.raw['released'] is False, versions) + # Consider only x.y.z versions + versions = filter(lambda x: re.match('\d+\.\d+\.\d+', x.name), versions) + + default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) + for v in default_fix_versions: + # Handles the case where we have forked a release branch but not yet made the release. + # In this case, if the PR is committed to the master branch and the release branch, we + # only consider the release branch to be the fix version. E.g. it is not valid to have + # both 1.1.0 and 1.0.0 as fix versions. + (major, minor, patch) = v.split(".") + if patch == "0": + previous = "%s.%s.%s" % (major, int(minor) - 1, 0) + if previous in default_fix_versions: + default_fix_versions = filter(lambda x: x != v, default_fix_versions) + default_fix_versions = ",".join(default_fix_versions) + + fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) + if fix_versions == "": + fix_versions = default_fix_versions + fix_versions = fix_versions.replace(" ", "").split(",") + + def get_version_json(version_str): + return filter(lambda v: v.name == version_str, versions)[0].raw + + jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) + + resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] + asf_jira.transition_issue(jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) + + print "Successfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + + +def resolve_jira_issues(title, merge_branches, comment): + jira_ids = re.findall("%s-[0-9]{4,5}" % CAPITALIZED_PROJECT_NAME, title) + + if len(jira_ids) == 0: + resolve_jira_issue(merge_branches, comment) + for jira_id in jira_ids: + resolve_jira_issue(merge_branches, comment, jira_id) + + +def standardize_jira_ref(text): + """ + Standardize the jira reference commit message prefix to "PROJECT_NAME-XXX; Issue" + + >>> standardize_jira_ref("%s-5954; Top by key" % CAPITALIZED_PROJECT_NAME) + 'KAFKA-5954; Top by key' + >>> standardize_jira_ref("%s-5821; ParquetRelation2 CTAS should check if delete is successful" % PROJECT_NAME) + 'KAFKA-5821; ParquetRelation2 CTAS should check if delete is successful' + >>> standardize_jira_ref("%s-4123 [WIP] Show new dependencies added in pull requests" % PROJECT_NAME) + 'KAFKA-4123; [WIP] Show new dependencies added in pull requests' + >>> standardize_jira_ref("%s 5954: Top by key" % PROJECT_NAME) + 'KAFKA-5954; Top by key' + >>> standardize_jira_ref("%s-979 a LRU scheduler for load balancing in TaskSchedulerImpl" % PROJECT_NAME) + 'KAFKA-979; a LRU scheduler for load balancing in TaskSchedulerImpl' + >>> standardize_jira_ref("%s-1094 Support MiMa for reporting binary compatibility across versions." % CAPITALIZED_PROJECT_NAME) + 'KAFKA-1094; Support MiMa for reporting binary compatibility across versions.' + >>> standardize_jira_ref("[WIP] %s-1146; Vagrant support" % CAPITALIZED_PROJECT_NAME) + 'KAFKA-1146; [WIP] Vagrant support' + >>> standardize_jira_ref("%s-1032. If Yarn app fails before registering, app master stays aroun..." % PROJECT_NAME) + 'KAFKA-1032; If Yarn app fails before registering, app master stays aroun...' + >>> standardize_jira_ref("%s-6250 %s-6146 %s-5911: Types are now reserved words in DDL parser." % (PROJECT_NAME, PROJECT_NAME, CAPITALIZED_PROJECT_NAME)) + 'KAFKA-6250 KAFKA-6146 KAFKA-5911; Types are now reserved words in DDL parser.' + >>> standardize_jira_ref("Additional information for users building from source code") + 'Additional information for users building from source code' + """ + jira_refs = [] + components = [] + + # Extract JIRA ref(s): + pattern = re.compile(r'(%s[-\s]*[0-9]{3,6})+' % CAPITALIZED_PROJECT_NAME, re.IGNORECASE) + for ref in pattern.findall(text): + # Add brackets, replace spaces with a dash, & convert to uppercase + jira_refs.append(re.sub(r'\s+', '-', ref.upper())) + text = text.replace(ref, '') + + # Extract project name component(s): + # Look for alphanumeric chars, spaces, dashes, periods, and/or commas + pattern = re.compile(r'(\[[\w\s,-\.]+\])', re.IGNORECASE) + for component in pattern.findall(text): + components.append(component.upper()) + text = text.replace(component, '') + + # Cleanup any remaining symbols: + pattern = re.compile(r'^\W+(.*)', re.IGNORECASE) + if (pattern.search(text) is not None): + text = pattern.search(text).groups()[0] + + # Assemble full text (JIRA ref(s), module(s), remaining text) + jira_prefix = ' '.join(jira_refs).strip() + if jira_prefix: + jira_prefix = jira_prefix + "; " + clean_text = jira_prefix + ' '.join(components).strip() + " " + text.strip() + + # Replace multiple spaces with a single space, e.g. if no jira refs and/or components were included + clean_text = re.sub(r'\s+', ' ', clean_text.strip()) + + return clean_text + +def main(): + global original_head + + original_head = run_cmd("git rev-parse HEAD")[:8] + + branches = get_json("%s/branches" % GITHUB_API_BASE) + branch_names = filter(lambda x: x.startswith(RELEASE_BRANCH_PREFIX), [x['name'] for x in branches]) + # Assumes branch names can be sorted lexicographically + latest_branch = sorted(branch_names, reverse=True)[0] + + pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") + pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) + pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) + + url = pr["url"] + + # Decide whether to use the modified title or not + modified_title = standardize_jira_ref(pr["title"]) + if modified_title != pr["title"]: + print "I've re-written the title as follows to match the standard format:" + print "Original: %s" % pr["title"] + print "Modified: %s" % modified_title + result = raw_input("Would you like to use the modified title? (y/n): ") + if result.lower() == "y": + title = modified_title + print "Using modified title:" + else: + title = pr["title"] + print "Using original title:" + print title + else: + title = pr["title"] + + body = pr["body"] + target_ref = pr["base"]["ref"] + user_login = pr["user"]["login"] + base_ref = pr["head"]["ref"] + pr_repo_desc = "%s/%s" % (user_login, base_ref) + + # Merged pull requests don't appear as merged in the GitHub API; + # Instead, they're closed by asfgit. + merge_commits = \ + [e for e in pr_events if e["actor"]["login"] == "asfgit" and e["event"] == "closed"] + + if merge_commits: + merge_hash = merge_commits[0]["commit_id"] + message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] + + print "Pull request %s has already been merged, assuming you want to backport" % pr_num + commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', + "%s^{commit}" % merge_hash]).strip() != "" + if not commit_is_downloaded: + fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) + + print "Found commit %s:\n%s" % (merge_hash, message) + cherry_pick(pr_num, merge_hash, latest_branch) + sys.exit(0) + + if not bool(pr["mergeable"]): + msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ + "Continue? (experts only!)" + continue_maybe(msg) + + print ("\n=== Pull Request #%s ===" % pr_num) + print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( + title, pr_repo_desc, target_ref, url)) + continue_maybe("Proceed with merging pull request #%s?" % pr_num) + + merged_refs = [target_ref] + + merge_hash = merge_pr(pr_num, target_ref, title, body, pr_repo_desc) + + pick_prompt = "Would you like to pick %s into another branch?" % merge_hash + while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] + + if JIRA_IMPORTED: + if JIRA_USERNAME and JIRA_PASSWORD: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira_issues(title, merged_refs, jira_comment) + else: + print "JIRA_USERNAME and JIRA_PASSWORD not set" + print "Exiting without trying to close the associated JIRA." + else: + print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." + print "Exiting without trying to close the associated JIRA." + +if __name__ == "__main__": + import doctest + doctest.testmod() + + main() diff --git a/kafka-patch-review.py b/kafka-patch-review.py index dc6664d22a0b1..94873c3c29cf2 100644 --- a/kafka-patch-review.py +++ b/kafka-patch-review.py @@ -1,23 +1,59 @@ #!/usr/bin/env python +# +# 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. import argparse import sys -import os +import os import time import datetime import tempfile +import commands +import getpass from jira.client import JIRA -def get_jira(): - options = { - 'server': 'https://issues.apache.org/jira' - } +def get_jira_config(): # read the config file home=jira_home=os.getenv('HOME') home=home.rstrip('/') - jira_config = dict(line.strip().split('=') for line in open(home + '/jira.ini')) - jira = JIRA(options,basic_auth=(jira_config['user'], jira_config['password'])) - return jira + if not (os.path.isfile(home + '/jira.ini')): + jira_user=raw_input('JIRA user :') + jira_pass=getpass.getpass('JIRA password :') + jira_config = {'user':jira_user, 'password':jira_pass} + return jira_config + else: + jira_config = dict(line.strip().split('=') for line in open(home + '/jira.ini')) + return jira_config + +def get_jira(jira_config): + options = { + 'server': 'https://issues.apache.org/jira' + } + jira = JIRA(options=options,basic_auth=(jira_config['user'], jira_config['password'])) + # (Force) verify the auth was really done + jira_session=jira.session() + if (jira_session is None): + raise Exception("Failed to login to the JIRA instance") + return jira + +def cmd_exists(cmd): + status, result = commands.getstatusoutput(cmd) + return status def main(): ''' main(), shut up, pylint ''' @@ -31,6 +67,15 @@ def main(): popt.add_argument('-db', '--debug', action='store_true', required=False, help='Enable debug mode') opt = popt.parse_args() + post_review_tool = None + if (cmd_exists("post-review") == 0): + post_review_tool = "post-review" + elif (cmd_exists("rbt") == 0): + post_review_tool = "rbt post" + else: + print "please install RBTools" + sys.exit(1) + patch_file=tempfile.gettempdir() + "/" + opt.jira + ".patch" if opt.reviewboard: ts = time.time() @@ -62,9 +107,49 @@ def main(): p=os.popen(git_remote_update) p.close() - rb_command="post-review --publish --tracking-branch " + opt.branch + " --target-groups=kafka --bugs-closed=" + opt.jira + # Get JIRA configuration and login to JIRA to ensure the credentials work, before publishing the patch to the review board + print "Verifying JIRA connection configurations" + try: + jira_config=get_jira_config() + jira=get_jira(jira_config) + except: + print "Failed to login to the JIRA instance", sys.exc_info()[0], sys.exc_info()[1] + sys.exit(1) + + git_command="git format-patch " + opt.branch + " --stdout > " + patch_file + if opt.debug: + print git_command + p=os.popen(git_command) + p.close() + + print 'Getting latest patch attached to the JIRA' + tmp_dir = tempfile.mkdtemp() + get_latest_patch_command="python ./dev-utils/test-patch.py --get-latest-patch --defect " + opt.jira + " --output " + tmp_dir + " > /dev/null 2>&1" + p=os.popen(get_latest_patch_command) + p.close() + + previous_patch=tmp_dir + "/" + opt.jira + ".patch" + diff_file=tmp_dir + "/" + opt.jira + ".diff" + if os.path.isfile(previous_patch) and os.stat(previous_patch).st_size > 0: + print 'Creating diff with previous version of patch uploaded to JIRA' + diff_command = "diff " + previous_patch+ " " + patch_file + " > " + diff_file + try: + p=os.popen(diff_command) + sys.stdout.flush() + p.close() + except: + pass + print 'Diff with previous version of patch uploaded to JIRA is saved to ' + diff_file + + print 'Checking if the there are changes that need to be pushed' + if os.stat(diff_file).st_size == 0: + print 'No changes found on top of changes uploaded to JIRA' + print 'Aborting' + sys.exit(1) + + rb_command= post_review_tool + " --publish --tracking-branch " + opt.branch + " --target-groups=kafka --bugs-closed=" + opt.jira if opt.debug: - rb_command=rb_command + " --debug" + rb_command=rb_command + " --debug" summary="Patch for " + opt.jira if opt.summary: summary=opt.summary @@ -91,33 +176,38 @@ def main(): print 'ERROR: Your reviewboard was not created/updated. Please run the script with the --debug option to troubleshoot the problem' p.close() sys.exit(1) - p.close() - if opt.debug: - print 'rb url=',rb_url - - git_command="git diff " + opt.branch + " > " + patch_file + if p.close() != None: + print 'ERROR: reviewboard update failed. Exiting.' + sys.exit(1) if opt.debug: - print git_command - p=os.popen(git_command) - p.close() + print 'rb url=',rb_url print 'Creating diff against', opt.branch, 'and uploading patch to JIRA',opt.jira - jira=get_jira() issue = jira.issue(opt.jira) attachment=open(patch_file) jira.add_attachment(issue,attachment) attachment.close() - comment="Created reviewboard " + 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' + print 'Updated reviewboard',rb_url comment="Updated reviewboard " - comment = comment + rb_url + ' against branch ' + opt.branch + comment = comment + rb_url + ' against branch ' + opt.branch jira.add_comment(opt.jira, comment) + #update the JIRA status to PATCH AVAILABLE + transitions = jira.transitions(issue) + transitionsMap ={} + + for t in transitions: + transitionsMap[t['name']] = t['id'] + + if('Submit Patch' in transitionsMap): + jira.transition_issue(issue, transitionsMap['Submit Patch'] , assignee={'name': jira_config['user']} ) + + if __name__ == '__main__': sys.exit(main()) - diff --git a/lib/apache-rat-0.8.jar b/lib/apache-rat-0.8.jar deleted file mode 100644 index bdc43726d6369..0000000000000 Binary files a/lib/apache-rat-0.8.jar and /dev/null differ diff --git a/lib/sbt-launch.jar b/lib/sbt-launch.jar deleted file mode 100644 index 06ad8d880592a..0000000000000 Binary files a/lib/sbt-launch.jar and /dev/null differ diff --git a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java new file mode 100644 index 0000000000000..628ff53ff70b8 --- /dev/null +++ b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java @@ -0,0 +1,167 @@ +/** + * 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 org.apache.kafka.log4jappender; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.config.ConfigException; +import org.apache.log4j.AppenderSkeleton; +import org.apache.log4j.helpers.LogLog; +import org.apache.log4j.spi.LoggingEvent; + +import java.util.Date; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * A log4j appender that produces log messages to Kafka + */ +public class KafkaLog4jAppender extends AppenderSkeleton { + + private static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + private static final String COMPRESSION_TYPE_CONFIG = "compression.type"; + private static final String ACKS_CONFIG = "acks"; + private static final String RETRIES_CONFIG = "retries"; + private static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; + private static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; + + private String brokerList = null; + private String topic = null; + private String compressionType = null; + + private int retries = 0; + private int requiredNumAcks = Integer.MAX_VALUE; + private boolean syncSend = false; + private Producer producer = null; + + public Producer getProducer() { + return producer; + } + + public String getBrokerList() { + return brokerList; + } + + public void setBrokerList(String brokerList) { + this.brokerList = brokerList; + } + + public int getRequiredNumAcks() { + return requiredNumAcks; + } + + public void setRequiredNumAcks(int requiredNumAcks) { + this.requiredNumAcks = requiredNumAcks; + } + + public int getRetries() { + return retries; + } + + public void setRetries(int retries) { + this.retries = retries; + } + + public String getCompressionType() { + return compressionType; + } + + public void setCompressionType(String compressionType) { + this.compressionType = compressionType; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public boolean getSyncSend() { + return syncSend; + } + + public void setSyncSend(boolean syncSend) { + this.syncSend = syncSend; + } + + @Override + public void activateOptions() { + // check for config parameter validity + Properties props = new Properties(); + if (brokerList != null) + props.put(BOOTSTRAP_SERVERS_CONFIG, brokerList); + if (props.isEmpty()) + throw new ConfigException("The bootstrap servers property should be specified"); + if (topic == null) + throw new ConfigException("Topic must be specified by the Kafka log4j appender"); + if (compressionType != null) + props.put(COMPRESSION_TYPE_CONFIG, compressionType); + if (requiredNumAcks != Integer.MAX_VALUE) + props.put(ACKS_CONFIG, requiredNumAcks); + if (retries > 0) + props.put(RETRIES_CONFIG, retries); + + props.put(KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + props.put(VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + this.producer = getKafkaProducer(props); + LogLog.debug("Kafka producer connected to " + brokerList); + LogLog.debug("Logging for topic: " + topic); + } + + protected Producer getKafkaProducer(Properties props) { + return new KafkaProducer(props); + } + + @Override + protected void append(LoggingEvent event) { + String message = subAppend(event); + LogLog.debug("[" + new Date(event.getTimeStamp()) + "]" + message); + Future response = producer.send(new ProducerRecord(topic, message.getBytes())); + if (syncSend) { + try { + response.get(); + } catch (InterruptedException ex) { + throw new RuntimeException(ex); + } catch (ExecutionException ex) { + throw new RuntimeException(ex); + } + } + } + + private String subAppend(LoggingEvent event) { + return (this.layout == null) ? event.getRenderedMessage() : this.layout.format(event); + } + + @Override + public void close() { + if (!this.closed) { + this.closed = true; + producer.close(); + } + } + + @Override + public boolean requiresLayout() { + return true; + } +} diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java new file mode 100644 index 0000000000000..71bdd94360c7e --- /dev/null +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java @@ -0,0 +1,98 @@ +/** + * 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 org.apache.kafka.log4jappender; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.log4j.Logger; +import org.apache.log4j.PropertyConfigurator; +import org.junit.Assert; +import org.junit.Test; + +import java.io.UnsupportedEncodingException; +import java.util.Properties; + +public class KafkaLog4jAppenderTest { + + Logger logger = Logger.getLogger(KafkaLog4jAppenderTest.class); + + @Test + public void testKafkaLog4jConfigs() { + // host missing + Properties props = new Properties(); + props.put("log4j.rootLogger", "INFO"); + props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender"); + props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.put("log4j.appender.KAFKA.Topic", "test-topic"); + props.put("log4j.logger.kafka.log4j", "INFO, KAFKA"); + + try { + PropertyConfigurator.configure(props); + Assert.fail("Missing properties exception was expected !"); + } catch (ConfigException ex) { + // It's OK! + } + + // topic missing + props = new Properties(); + props.put("log4j.rootLogger", "INFO"); + props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.KafkaLog4jAppender"); + props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.put("log4j.appender.KAFKA.brokerList", "127.0.0.1:9093"); + props.put("log4j.logger.kafka.log4j", "INFO, KAFKA"); + + try { + PropertyConfigurator.configure(props); + Assert.fail("Missing properties exception was expected !"); + } catch (ConfigException ex) { + // It's OK! + } + } + + + @Test + public void testLog4jAppends() throws UnsupportedEncodingException { + PropertyConfigurator.configure(getLog4jConfig()); + + for (int i = 1; i <= 5; ++i) { + logger.error(getMessage(i)); + } + + Assert.assertEquals( + 5, ((MockKafkaLog4jAppender) (logger.getRootLogger().getAppender("KAFKA"))).getHistory().size()); + } + + private byte[] getMessage(int i) throws UnsupportedEncodingException { + return ("test_" + i).getBytes("UTF-8"); + } + + private Properties getLog4jConfig() { + Properties props = new Properties(); + props.put("log4j.rootLogger", "INFO, KAFKA"); + props.put("log4j.appender.KAFKA", "org.apache.kafka.log4jappender.MockKafkaLog4jAppender"); + props.put("log4j.appender.KAFKA.layout", "org.apache.log4j.PatternLayout"); + props.put("log4j.appender.KAFKA.layout.ConversionPattern", "%-5p: %c - %m%n"); + props.put("log4j.appender.KAFKA.BrokerList", "127.0.0.1:9093"); + props.put("log4j.appender.KAFKA.Topic", "test-topic"); + props.put("log4j.appender.KAFKA.RequiredNumAcks", "1"); + props.put("log4j.appender.KAFKA.SyncSend", "false"); + props.put("log4j.logger.kafka.log4j", "INFO, KAFKA"); + return props; + } +} + diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java new file mode 100644 index 0000000000000..c35f26adb33f7 --- /dev/null +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java @@ -0,0 +1,47 @@ +/** + * 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 org.apache.kafka.log4jappender; + +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.test.MockSerializer; +import org.apache.log4j.spi.LoggingEvent; + +import java.util.Properties; + +public class MockKafkaLog4jAppender extends KafkaLog4jAppender { + private MockProducer mockProducer = + new MockProducer(false, new MockSerializer(), new MockSerializer()); + + @Override + protected Producer getKafkaProducer(Properties props) { + return mockProducer; + } + + @Override + protected void append(LoggingEvent event) { + if (super.getProducer() == null) { + activateOptions(); + } + super.append(event); + } + + protected java.util.List> getHistory() { + return mockProducer.history(); + } +} diff --git a/perf/build.sbt b/perf/build.sbt deleted file mode 100644 index 8aa72f4a13eda..0000000000000 --- a/perf/build.sbt +++ /dev/null @@ -1 +0,0 @@ -name := "kafka-perf" diff --git a/perf/config/log4j.properties b/perf/config/log4j.properties deleted file mode 100644 index 542b7391550b2..0000000000000 --- a/perf/config/log4j.properties +++ /dev/null @@ -1,24 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -log4j.rootLogger=INFO, fileAppender - -log4j.appender.fileAppender=org.apache.log4j.FileAppender -log4j.appender.fileAppender.File=perf.log -log4j.appender.fileAppender.layout=org.apache.log4j.PatternLayout -log4j.appender.fileAppender.layout.ConversionPattern=%m %n - -# Turn on all our debugging info -log4j.logger.kafka=INFO - diff --git a/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala b/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala deleted file mode 100644 index 55ee01b40afa4..0000000000000 --- a/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala +++ /dev/null @@ -1,206 +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.perf - -import java.util.concurrent.CountDownLatch -import java.util.concurrent.atomic.AtomicLong -import java.nio.channels.ClosedByInterruptException -import org.apache.log4j.Logger -import kafka.message.Message -import kafka.utils.ZkUtils -import java.util.{Random, Properties} -import kafka.consumer._ -import java.text.SimpleDateFormat - -/** - * Performance test for the full zookeeper consumer - */ -object ConsumerPerformance { - private val logger = Logger.getLogger(getClass()) - - def main(args: Array[String]): Unit = { - - val config = new ConsumerPerfConfig(args) - logger.info("Starting consumer...") - var totalMessagesRead = new AtomicLong(0) - var totalBytesRead = new AtomicLong(0) - - if(!config.hideHeader) { - if(!config.showDetailedStats) - println("start.time, end.time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") - else - println("time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") - } - - // clean up zookeeper state for this group id for every perf run - ZkUtils.maybeDeletePath(config.consumerConfig.zkConnect, "/consumers/" + config.consumerConfig.groupId) - - val consumerConnector: ConsumerConnector = Consumer.create(config.consumerConfig) - - val topicMessageStreams = consumerConnector.createMessageStreams(Map(config.topic -> config.numThreads)) - var threadList = List[ConsumerPerfThread]() - for ((topic, streamList) <- topicMessageStreams) - for (i <- 0 until streamList.length) - threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, - totalMessagesRead, totalBytesRead) - - logger.info("Sleeping for 1 second.") - Thread.sleep(1000) - logger.info("starting threads") - val startMs = System.currentTimeMillis - for (thread <- threadList) - thread.start - - for (thread <- threadList) - thread.shutdown - - val endMs = System.currentTimeMillis - val elapsedSecs = (endMs - startMs - config.consumerConfig.consumerTimeoutMs) / 1000.0 - if(!config.showDetailedStats) { - val totalMBRead = (totalBytesRead.get*1.0)/(1024*1024) - println(("%s, %s, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs), - config.consumerConfig.fetchMessageMaxBytes, totalMBRead, totalMBRead/elapsedSecs, totalMessagesRead.get, - totalMessagesRead.get/elapsedSecs)) - } - System.exit(0) - } - - class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) { - 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 topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - val groupIdOpt = parser.accepts("group", "The group id to consume on.") - .withRequiredArg - .describedAs("gid") - .defaultsTo("perf-consumer-" + new Random().nextInt(100000)) - .ofType(classOf[String]) - val fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.") - .withRequiredArg - .describedAs("size") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1024 * 1024) - val resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " + - "offset to consume from, start with the latest message present in the log rather than the earliest message.") - val socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.") - .withRequiredArg - .describedAs("size") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(2 * 1024 * 1024) - val numThreadsOpt = parser.accepts("threads", "Number of processing threads.") - .withRequiredArg - .describedAs("count") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(10) - val numFetchersOpt = parser.accepts("num-fetch-threads", "Number of fetcher threads.") - .withRequiredArg - .describedAs("count") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) - - 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 props = new Properties - props.put("group.id", options.valueOf(groupIdOpt)) - props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) - props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString) - props.put("auto.offset.reset", if(options.has(resetBeginningOffsetOpt)) "largest" else "smallest") - props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) - props.put("consumer.timeout.ms", "5000") - props.put("num.consumer.fetchers", options.valueOf(numFetchersOpt).toString) - val consumerConfig = new ConsumerConfig(props) - val numThreads = options.valueOf(numThreadsOpt).intValue - val topic = options.valueOf(topicOpt) - val numMessages = options.valueOf(numMessagesOpt).longValue - val reportingInterval = options.valueOf(reportingIntervalOpt).intValue - val showDetailedStats = options.has(showDetailedStatsOpt) - val dateFormat = new SimpleDateFormat(options.valueOf(dateFormatOpt)) - val hideHeader = options.has(hideHeaderOpt) - } - - class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaStream[Array[Byte], Array[Byte]], - config:ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) - extends Thread(name) { - private val shutdownLatch = new CountDownLatch(1) - - def shutdown(): Unit = { - shutdownLatch.await - } - - override def run() { - var bytesRead = 0L - var messagesRead = 0L - val startMs = System.currentTimeMillis - var lastReportTime: Long = startMs - var lastBytesRead = 0L - var lastMessagesRead = 0L - - try { - for (messageAndMetadata <- stream if messagesRead < config.numMessages) { - messagesRead += 1 - bytesRead += messageAndMetadata.message.length - - if (messagesRead % config.reportingInterval == 0) { - if(config.showDetailedStats) - printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis) - lastReportTime = System.currentTimeMillis - lastMessagesRead = messagesRead - lastBytesRead = bytesRead - } - } - } - catch { - case _: InterruptedException => - case _: ClosedByInterruptException => - case _: ConsumerTimeoutException => - case e: Throwable => throw e - } - totalMessagesRead.addAndGet(messagesRead) - totalBytesRead.addAndGet(bytesRead) - if(config.showDetailedStats) - printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis) - shutdownComplete - } - - private def printMessage(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, lastMessagesRead: Long, - startMs: Long, endMs: Long) = { - val elapsedMs = endMs - startMs - val totalMBRead = (bytesRead*1.0)/(1024*1024) - val mbRead = ((bytesRead - lastBytesRead)*1.0)/(1024*1024) - println(("%s, %d, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(endMs), id, - config.consumerConfig.fetchMessageMaxBytes, totalMBRead, - 1000.0*(mbRead/elapsedMs), messagesRead, ((messagesRead - lastMessagesRead)/elapsedMs)*1000.0)) - } - - private def shutdownComplete() = shutdownLatch.countDown - } - -} diff --git a/project/Build.scala b/project/Build.scala deleted file mode 100644 index ddcfc4176e689..0000000000000 --- a/project/Build.scala +++ /dev/null @@ -1,152 +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. - */ - -import sbt._ -import Keys._ -import Process._ - -import scala.xml.{Node, Elem} -import scala.xml.transform.{RewriteRule, RuleTransformer} - -object KafkaBuild extends Build { - val buildNumber = SettingKey[String]("build-number", "Build number defaults to $BUILD_NUMBER environment variable") - val releaseName = SettingKey[String]("release-name", "the full name of this release") - val commonSettings = Seq( - organization := "org.apache.kafka", - pomExtra := - - org.apache - apache - 10 - - - - Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt - repo - -, - scalacOptions ++= Seq("-deprecation", "-unchecked", "-g:none"), - crossScalaVersions := Seq("2.8.0","2.8.2", "2.9.1", "2.9.2", "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.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", ""), - version <<= (buildNumber, version) { (build, version) => if (build == "") version else version + "+" + build}, - releaseName <<= (name, version, scalaVersion) {(name, version, scalaVersion) => name + "_" + scalaVersion + "-" + version}, - 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", - "org.slf4j" % "slf4j-simple" % "1.6.4" - ), - // The issue is going from log4j 1.2.14 to 1.2.15, the developers added some features which required - // some dependencies on various sun and javax packages. - ivyXML := - - - - - - - - - - , - mappings in packageBin in Compile += file("LICENSE") -> "LICENSE", - mappings in packageBin in Compile += file("NOTICE") -> "NOTICE" - ) - - val hadoopSettings = Seq( - javacOptions in compile ++= Seq("-Xlint:deprecation"), - libraryDependencies ++= Seq( - "org.apache.avro" % "avro" % "1.4.0", - "org.apache.pig" % "pig" % "0.8.0", - "commons-logging" % "commons-logging" % "1.0.4", - "org.codehaus.jackson" % "jackson-core-asl" % "1.5.5", - "org.codehaus.jackson" % "jackson-mapper-asl" % "1.5.5", - "org.apache.hadoop" % "hadoop-core" % "0.20.2" - ), - ivyXML := - - - - - - - - - - - - - - - ) - - - val runRat = TaskKey[Unit]("run-rat-task", "Runs Apache rat on Kafka") - val runRatTask = runRat := { - "bin/run-rat.sh" ! - } - - val release = TaskKey[Unit]("release", "Creates a deployable release directory file with dependencies, config, and scripts.") - val releaseTask = release <<= ( packageBin in (core, Compile), dependencyClasspath in (core, Runtime), exportedProducts in Compile, - target, releaseName in core ) map { (packageBin, deps, products, target, releaseName) => - val jarFiles = deps.files.filter(f => !products.files.contains(f) && f.getName.endsWith(".jar")) - val destination = target / "RELEASE" / releaseName - IO.copyFile(packageBin, destination / packageBin.getName) - IO.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") - for {file <- (destination / "bin").listFiles} { file.setExecutable(true, true) } - } - - val releaseZip = TaskKey[Unit]("release-zip", "Creates a deployable zip file with dependencies, config, and scripts.") - val releaseZipTask = releaseZip <<= (release, target, releaseName in core) map { (release, target, releaseName) => - val zipPath = target / "RELEASE" / "%s.zip".format(releaseName) - IO.delete(zipPath) - IO.zip((target/"RELEASE" ** releaseName ***) x relativeTo(target/"RELEASE"), zipPath) - } - - val releaseTar = TaskKey[Unit]("release-tar", "Creates a deployable tar.gz file with dependencies, config, and scripts.") - val releaseTarTask = releaseTar <<= ( release, target, releaseName in core) map { (release, target, releaseName) => - Process(Seq("tar", "czf", "%s.tar.gz".format(releaseName), releaseName), target / "RELEASE").! match { - case 0 => () - case n => sys.error("Failed to run native tar application!") - } - } - - lazy val kafka = Project(id = "Kafka", base = file(".")).aggregate(core, examples, contrib, perf).settings((commonSettings ++ - runRatTask ++ releaseTask ++ releaseZipTask ++ releaseTarTask): _*) - lazy val core = Project(id = "core", base = file("core")).settings(commonSettings: _*) - lazy val examples = Project(id = "java-examples", base = file("examples")).settings(commonSettings :_*) dependsOn (core) - lazy val perf = Project(id = "perf", base = file("perf")).settings((Seq(name := "kafka-perf") ++ commonSettings):_*) dependsOn (core) - - lazy val contrib = Project(id = "contrib", base = file("contrib")).aggregate(hadoopProducer, hadoopConsumer).settings(commonSettings :_*) - lazy val hadoopProducer = Project(id = "hadoop-producer", base = file("contrib/hadoop-producer")).settings(hadoopSettings ++ commonSettings: _*) dependsOn (core) - lazy val hadoopConsumer = Project(id = "hadoop-consumer", base = file("contrib/hadoop-consumer")).settings(hadoopSettings ++ commonSettings: _*) dependsOn (core) - lazy val clients = Project(id = "kafka-clients", base = file("clients")) - -} diff --git a/project/build/KafkaProject.scala b/project/build/KafkaProject.scala deleted file mode 100644 index f29c67a47fe89..0000000000000 --- a/project/build/KafkaProject.scala +++ /dev/null @@ -1,251 +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. - */ - -import sbt._ -import scala.xml.{Node, Elem} -import scala.xml.transform.{RewriteRule, RuleTransformer} - -class KafkaProject(info: ProjectInfo) extends ParentProject(info) with IdeaProject { - override def managedStyle = ManagedStyle.Maven - val publishTo = "Maven Repo" at "http://maven/content/repositories/repository.snapshots" - Credentials(Path.userHome / ".m2" / ".credentials", log) - - lazy val core = project("core", "core-kafka", new CoreKafkaProject(_)) - lazy val examples = project("examples", "java-examples", new KafkaExamplesProject(_), core) - lazy val contrib = project("contrib", "contrib", new ContribProject(_)) - lazy val perf = project("perf", "perf", new KafkaPerfProject(_)) - - lazy val releaseZipTask = core.packageDistTask - - val releaseZipDescription = "Compiles every sub project, runs unit tests, creates a deployable release zip file with dependencies, config, and scripts." - lazy val releaseZip = releaseZipTask dependsOn(core.corePackageAction, core.test, examples.examplesPackageAction, - contrib.producerPackageAction, contrib.consumerPackageAction) describedAs releaseZipDescription - - val runRatDescription = "Runs Apache rat on Kafka" - lazy val runRatTask = task { - Runtime.getRuntime().exec("bin/run-rat.sh") - None - } describedAs runRatDescription - - val rat = "org.apache.rat" % "apache-rat" % "0.8" - - class CoreKafkaProject(info: ProjectInfo) extends DefaultProject(info) - with IdeaProject with CoreDependencies with TestDependencies with CompressionDependencies { - val corePackageAction = packageAllAction - - //The issue is going from log4j 1.2.14 to 1.2.15, the developers added some features which required - // some dependencies on various sun and javax packages. - override def ivyXML = - - - - - - - - - - - - - override def organization = "org.apache" - override def filterScalaJars = false - - // build the executable jar's classpath. - // (why is it necessary to explicitly remove the target/{classes,resources} paths? hm.) - def dependentJars = { - val jars = - publicClasspath +++ mainDependencies.scalaJars --- mainCompilePath --- mainResourcesOutputPath - if (jars.get.find { jar => jar.name.startsWith("scala-library-") }.isDefined) { - // workaround bug in sbt: if the compiler is explicitly included, don't include 2 versions - // of the library. - jars --- jars.filter { jar => - jar.absolutePath.contains("/boot/") && jar.name == "scala-library.jar" - } - } else { - jars - } - } - - def dependentJarNames = dependentJars.getFiles.map(_.getName).filter(_.endsWith(".jar")) - override def manifestClassPath = Some(dependentJarNames.map { "libs/" + _ }.mkString(" ")) - - def distName = (artifactID + "-" + projectVersion.value) - def distPath = "dist" / distName ## - - def configPath = "config" ## - def configOutputPath = distPath / "config" - - def binPath = "bin" ## - def binOutputPath = distPath / "bin" - - def distZipName = { - "%s-%s.zip".format(artifactID, projectVersion.value) - } - - lazy val packageDistTask = task { - distPath.asFile.mkdirs() - (distPath / "libs").asFile.mkdirs() - binOutputPath.asFile.mkdirs() - configOutputPath.asFile.mkdirs() - - FileUtilities.copyFlat(List(jarPath), distPath, log).left.toOption orElse - FileUtilities.copyFlat(dependentJars.get, distPath / "libs", log).left.toOption orElse - FileUtilities.copy((configPath ***).get, configOutputPath, log).left.toOption orElse - FileUtilities.copy((binPath ***).get, binOutputPath, log).left.toOption orElse - FileUtilities.zip((("dist" / distName) ##).get, "dist" / distZipName, true, log) - None - } - - val PackageDistDescription = "Creates a deployable zip file with dependencies, config, and scripts." - lazy val packageDist = packageDistTask dependsOn(`package`, `test`) describedAs PackageDistDescription - - val cleanDist = cleanTask("dist" ##) describedAs("Erase any packaged distributions.") - override def cleanAction = super.cleanAction dependsOn(cleanDist) - - override def javaCompileOptions = super.javaCompileOptions ++ - List(JavaCompileOption("-source"), JavaCompileOption("1.5")) - - override def packageAction = super.packageAction dependsOn (testCompileAction, packageTestAction) - - } - - class KafkaPerfProject(info: ProjectInfo) extends DefaultProject(info) - with IdeaProject - with CoreDependencies { - val perfPackageAction = packageAllAction - val dependsOnCore = core - - //The issue is going from log4j 1.2.14 to 1.2.15, the developers added some features which required - // some dependencies on various sun and javax packages. - override def ivyXML = - - - - - - - - - override def artifactID = "kafka-perf" - override def filterScalaJars = false - override def javaCompileOptions = super.javaCompileOptions ++ - List(JavaCompileOption("-Xlint:unchecked")) - } - - class KafkaExamplesProject(info: ProjectInfo) extends DefaultProject(info) - with IdeaProject - with CoreDependencies { - val examplesPackageAction = packageAllAction - val dependsOnCore = core - //The issue is going from log4j 1.2.14 to 1.2.15, the developers added some features which required - // some dependencies on various sun and javax packages. - override def ivyXML = - - - - - - - - - override def artifactID = "kafka-java-examples" - override def filterScalaJars = false - override def javaCompileOptions = super.javaCompileOptions ++ - List(JavaCompileOption("-Xlint:unchecked")) - } - - class ContribProject(info: ProjectInfo) extends ParentProject(info) with IdeaProject { - lazy val hadoopProducer = project("hadoop-producer", "hadoop producer", - new HadoopProducerProject(_), core) - lazy val hadoopConsumer = project("hadoop-consumer", "hadoop consumer", - new HadoopConsumerProject(_), core) - - val producerPackageAction = hadoopProducer.producerPackageAction - val consumerPackageAction = hadoopConsumer.consumerPackageAction - - class HadoopProducerProject(info: ProjectInfo) extends DefaultProject(info) - with IdeaProject - with CoreDependencies with HadoopDependencies { - val producerPackageAction = packageAllAction - override def ivyXML = - - - - - - - - - - - - - - - - } - - class HadoopConsumerProject(info: ProjectInfo) extends DefaultProject(info) - with IdeaProject - with CoreDependencies { - val consumerPackageAction = packageAllAction - override def ivyXML = - - - - - - - - - - - - - - - - - val jodaTime = "joda-time" % "joda-time" % "1.6" - } - } - - trait TestDependencies { - val easymock = "org.easymock" % "easymock" % "3.0" % "test" - val junit = "junit" % "junit" % "4.1" % "test" - val scalaTest = "org.scalatest" % "scalatest" % "1.2" % "test" - } - - trait CoreDependencies { - val log4j = "log4j" % "log4j" % "1.2.15" - val jopt = "net.sf.jopt-simple" % "jopt-simple" % "3.2" - val slf4jSimple = "org.slf4j" % "slf4j-simple" % "1.6.4" - } - - trait HadoopDependencies { - val avro = "org.apache.avro" % "avro" % "1.4.0" - val commonsLogging = "commons-logging" % "commons-logging" % "1.0.4" - val jacksonCore = "org.codehaus.jackson" % "jackson-core-asl" % "1.5.5" - val jacksonMapper = "org.codehaus.jackson" % "jackson-mapper-asl" % "1.5.5" - val hadoop = "org.apache.hadoop" % "hadoop-core" % "0.20.2" - } - - trait CompressionDependencies { - val snappy = "org.xerial.snappy" % "snappy-java" % "1.0.5" - } -} diff --git a/project/plugins.sbt b/project/plugins.sbt deleted file mode 100644 index 7abe703893ba0..0000000000000 --- a/project/plugins.sbt +++ /dev/null @@ -1,9 +0,0 @@ -resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns) - -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.8") - -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") - -resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) - -addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6") \ No newline at end of file diff --git a/scala.gradle b/scala.gradle new file mode 100644 index 0000000000000..5eb2a656a05f5 --- /dev/null +++ b/scala.gradle @@ -0,0 +1,26 @@ +// 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. + +if (!hasProperty('scalaVersion')) { + ext.scalaVersion = '2.10.5' +} +ext.defaultScalaVersion = '2.10.5' +if (scalaVersion.startsWith('2.10')) { + ext.baseScalaVersion = '2.10' +} else if (scalaVersion.startsWith('2.11')) { + ext.baseScalaVersion = '2.11' +} else { + ext.baseScalaVersion = scalaVersion +} diff --git a/settings.gradle b/settings.gradle index 74e591a8a4afc..3b6a952cff2b3 100644 --- a/settings.gradle +++ b/settings.gradle @@ -13,4 +13,5 @@ // See the License for the specific language governing permissions and // limitations under the License. -include 'core', 'perf', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients' +apply from: file('scala.gradle') +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'log4j-appender' diff --git a/system_test/README.txt b/system_test/README.txt index d89ad9d7d5f0c..e96d15dd55846 100644 --- a/system_test/README.txt +++ b/system_test/README.txt @@ -1,7 +1,23 @@ # ========================== -# Known Issues: +# Quick Start # ========================== -1. This test framework currently doesn't support MacOS due to different "ps" argument options from Linux. The correct ps execution is required to terminate the background running processes properly. + +* Please note that the following commands should be executed after downloading the kafka source code to build all the required binaries: + 1. / $ ./gradlew jar + + Now you are ready to follow the steps below. + 1. Update system_test/cluster_config.json for "kafka_home" & "java_home" specific to your environment + 2. Edit system_test/replication_testsuite/testcase_1/testcase_1_properties.json and update "broker-list" to the proper settings of your environment. (If this test is to be run in a single localhost, no change is required for this.) + 3. Create testcase_to_run.json file with the tests you wish to run. You can start by just copying one of our preset test suites. For example: + cp testcase_to_run_sanity.json testcase_to_run.json + 4. To run the test, go to /system_test and run the following command: + $ python -u -B system_test_runner.py 2>&1 | tee system_test_output.log + 5. To turn on debugging, update system_test/logging.conf by changing the level in handlers session from INFO to DEBUG. + + We also have three built-in test suites you can use after you set your environment (steps 1 and 2 above): + * run_sanity.sh - will run a single basic replication test + * run_all_replica.sh - will run all replication tests + * run_all.sh - will run all replication and mirror_maker tests # ========================== # Overview @@ -43,21 +59,6 @@ The framework has the following levels: ** Please note the test framework will look for a specific prefix of the directories under system_test// to determine what test cases are available. The prefix of can be defined in SystemTestEnv class (system_test_env.py) -# ========================== -# Quick Start -# ========================== - -* Please note that the following commands should be executed after downloading the kafka source code to build all the required binaries: - 1. / $ ./sbt update package - - Now you are ready to follow the steps below. - 1. Update system_test/cluster_config.json for "kafka_home" & "java_home" specific to your environment - 2. Edit system_test/replication_testsuite/testcase_1/testcase_1_properties.json and update "broker-list" to the proper settings of your environment. (If this test is to be run in a single localhost, no change is required for this.) - 3. To run the test, go to /system_test and run the following command: - $ python -B system_test_runner.py - 4. To turn on debugging, update system_test/system_test_runner.py and uncomment the following line: - namedLogger.setLevel(logging.DEBUG) - # ========================== # Adding Test Case # ========================== @@ -76,4 +77,7 @@ To create a new test case under "replication_testsuite", please do the following 2. Rename system_test/replication_testsuite/testcase_2/testcase_1_properties.json => system_test/replication_testsuite/testcase_2/testcase_2_properties.json 3. Update system_test/replication_testsuite/testcase_2/testcase_2_properties.json with the corresponding settings for testcase 2. - +Note: +The following testcases are for the old producer and the old mirror maker. We can remove them once we phase out the old producer client. + replication_testsuite: testcase_{10101 - 10110} testcase_{10131 - 10134} + mirror_maker_testsuite: testcase_{15001 - 15006} diff --git a/system_test/broker_failure/bin/run-test.sh b/system_test/broker_failure/bin/run-test.sh index 1f11180e33974..549cd1f4eafd8 100755 --- a/system_test/broker_failure/bin/run-test.sh +++ b/system_test/broker_failure/bin/run-test.sh @@ -5,9 +5,9 @@ # 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. @@ -17,7 +17,7 @@ # =========== # run-test.sh # =========== - + # ==================================== # Do not change the followings # (keep this section at the beginning @@ -52,9 +52,9 @@ readonly source_console_consumer_grp=source readonly target_console_consumer_grp=target readonly message_size=100 readonly console_consumer_timeout_ms=15000 -readonly num_kafka_source_server=4 # requires same no. of property files such as: +readonly num_kafka_source_server=4 # requires same no. of property files such as: # $base_dir/config/server_source{1..4}.properties -readonly num_kafka_target_server=3 # requires same no. of property files such as: +readonly num_kafka_target_server=3 # requires same no. of property files such as: # $base_dir/config/server_target{1..3}.properties readonly num_kafka_mirror_maker=3 # any values greater than 0 readonly wait_time_after_killing_broker=0 # wait after broker is stopped but before starting again @@ -65,8 +65,8 @@ readonly wait_time_after_restarting_broker=10 # ==================================== num_msg_per_batch=500 # no. of msg produced in each calling of ProducerPerformance num_producer_threads=5 # no. of producer threads to send msg -producer_sleep_min=5 # min & max sleep time (in sec) between each -producer_sleep_max=5 # batch of messages sent from producer +producer_sleep_min=5 # min & max sleep time (in sec) between each +producer_sleep_max=5 # batch of messages sent from producer # ==================================== # zookeeper @@ -255,7 +255,7 @@ create_topic() { --topic $this_topic_to_create \ --zookeeper $this_zk_conn_str \ --replica $this_replica_factor \ - 2> $kafka_topic_creation_log_file + 2> $kafka_topic_creation_log_file } # ========================================= @@ -281,7 +281,7 @@ start_zk() { start_source_servers_cluster() { info "starting source cluster" - for ((i=1; i<=$num_kafka_source_server; i++)) + for ((i=1; i<=$num_kafka_source_server; i++)) do start_source_server $i done @@ -367,13 +367,13 @@ start_console_consumer() { info "starting console consumers for $this_consumer_grp" - $base_dir/bin/kafka-run-class.sh kafka.consumer.ConsoleConsumer \ + $base_dir/bin/kafka-run-class.sh kafka.tools.ConsoleConsumer \ --zookeeper localhost:$this_consumer_zk_port \ --topic $test_topic \ --group $this_consumer_grp \ --from-beginning \ --consumer-timeout-ms $console_consumer_timeout_ms \ - --formatter "kafka.consumer.ConsoleConsumer\$${this_msg_formatter}" \ + --formatter "kafka.tools.ConsoleConsumer\$${this_msg_formatter}" \ 2>&1 > ${this_consumer_log} & console_consumer_pid=$! @@ -448,7 +448,7 @@ start_background_producer() { info "producing $num_msg_per_batch messages on topic '$topic'" $base_dir/bin/kafka-run-class.sh \ - kafka.perf.ProducerPerformance \ + kafka.tools.ProducerPerformance \ --brokerinfo zk.connect=localhost:2181 \ --topics $topic \ --messages $num_msg_per_batch \ @@ -499,7 +499,7 @@ cmp_checksum() { crc_only_in_producer=`comm -23 $producer_performance_mid_sorted_uniq_log $console_consumer_source_mid_sorted_uniq_log` - duplicate_mirror_mid=`comm -23 $console_consumer_target_mid_sorted_log $console_consumer_target_mid_sorted_uniq_log` + duplicate_mirror_mid=`comm -23 $console_consumer_target_mid_sorted_log $console_consumer_target_mid_sorted_uniq_log` no_of_duplicate_msg=$(( $msg_count_from_mirror_consumer - $uniq_msg_count_from_mirror_consumer \ + $msg_count_from_source_consumer - $uniq_msg_count_from_source_consumer - \ 2*$duplicate_msg_in_producer )) @@ -521,19 +521,19 @@ cmp_checksum() { echo "" echo "========================================================" >> $checksum_diff_log - echo "crc only in producer" >> $checksum_diff_log + echo "crc only in producer" >> $checksum_diff_log echo "========================================================" >> $checksum_diff_log - echo "${crc_only_in_producer}" >> $checksum_diff_log + echo "${crc_only_in_producer}" >> $checksum_diff_log echo "" >> $checksum_diff_log echo "========================================================" >> $checksum_diff_log - echo "crc only in source consumer" >> $checksum_diff_log + echo "crc only in source consumer" >> $checksum_diff_log echo "========================================================" >> $checksum_diff_log - echo "${crc_only_in_source_consumer}" >> $checksum_diff_log + echo "${crc_only_in_source_consumer}" >> $checksum_diff_log echo "" >> $checksum_diff_log echo "========================================================" >> $checksum_diff_log echo "crc only in mirror consumer" >> $checksum_diff_log echo "========================================================" >> $checksum_diff_log - echo "${crc_only_in_mirror_consumer}" >> $checksum_diff_log + echo "${crc_only_in_mirror_consumer}" >> $checksum_diff_log echo "" >> $checksum_diff_log echo "========================================================" >> $checksum_diff_log echo "duplicate crc in mirror consumer" >> $checksum_diff_log @@ -583,8 +583,8 @@ start_test() { info "Started background producer pid [${background_producer_pid}]" sleep 5 - - # loop for no. of iterations specified in $num_iterations + + # loop for no. of iterations specified in $num_iterations while [ $num_iterations -ge $iter ] do # if $svr_to_bounce is '0', it means no bouncing diff --git a/system_test/metrics.json b/system_test/metrics.json index cd3fc142176b8..30dabe596871f 100644 --- a/system_test/metrics.json +++ b/system_test/metrics.json @@ -78,13 +78,13 @@ { "graph_name": "ProducePurgatoryQueueSize", "y_label": "size", - "bean_name": "kafka.server:type=ProducerRequestPurgatory,name=NumDelayedRequests", + "bean_name": "kafka.server:type=ProducerRequestPurgatory,name=NumDelayedOperations", "attributes": "Value" }, { "graph_name": "FetchPurgatoryQueueSize", "y_label": "size", - "bean_name": "kafka.server:type=FetchRequestPurgatory,name=NumDelayedRequests", + "bean_name": "kafka.server:type=FetchRequestPurgatory,name=NumDelayedOperations", "attributes": "Value" }, { 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 deleted file mode 100755 index ec92a343ebfee..0000000000000 --- a/system_test/migration_tool_testsuite/0.7/bin/kafka-run-class.sh +++ /dev/null @@ -1,87 +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. - -if [ $# -lt 1 ]; -then - echo "USAGE: $0 classname [opts]" - exit 1 -fi - -snappy=~/.ivy2/cache/org.xerial.snappy/snappy-java/bundles/snappy-java-1.0.5.jar -CLASSPATH=$CLASSPATH:$snappy -library=~/.ivy2/cache/org.scala-lang/scala-library/jars/scala-library-2.8.0.jar -CLASSPATH=$CLASSPATH:$library -compiler=~/.ivy2/cache/org.scala-lang/scala-compiler/jars/scala-compiler-2.8.0.jar -CLASSPATH=$CLASSPATH:$compiler -log4j=~/.ivy2/cache/log4j/log4j/jars/log4j-1.2.15.jar -CLASSPATH=$CLASSPATH:$log4j -slf=~/.ivy2/cache/org.slf4j/slf4j-api/jars/slf4j-api-1.6.4.jar -CLASSPATH=$CLASSPATH:$slf -zookeeper=~/.ivy2/cache/org.apache.zookeeper/zookeeper/jars/zookeeper-3.3.4.jar -CLASSPATH=$CLASSPATH:$zookeeper -jopt=~/.ivy2//cache/net.sf.jopt-simple/jopt-simple/jars/jopt-simple-3.2.jar -CLASSPATH=$CLASSPATH:$jopt - -base_dir=$(dirname $0)/../../../.. -kafka_07_lib_dir=$(dirname $0)/../lib - -# 0.8 - scala jars -for file in $base_dir/project/boot/scala-2.8.0/lib/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -# 0.7 - kafka-0.7.jar, zkclient-0.1.jar, kafka-perf-0.7.0.jar -for file in ${kafka_07_lib_dir}/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -# 0.8 - metrics jar -for file in $base_dir/core/lib/metrics*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -# 0.8 - misc jars -for file in $base_dir/core/lib_managed/scala_2.8.0/compile/*.jar; -do - if [ ${file##*/} != "sbt-launch.jar" ]; then - CLASSPATH=$CLASSPATH:$file - fi -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 $KAFKA_LOG4J_OPTS" -fi -if [ $JMX_PORT ]; then - KAFKA_JMX_OPTS="$KAFKA_JMX_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT " -fi -if [ -z "$JAVA_HOME" ]; then - JAVA="java" -else - JAVA="$JAVA_HOME/bin/java" -fi - -$JAVA $KAFKA_OPTS $KAFKA_JMX_OPTS -cp $CLASSPATH $@ 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 deleted file mode 100644 index a3ae33f20e4b7..0000000000000 --- a/system_test/migration_tool_testsuite/0.7/config/test-log4j.properties +++ /dev/null @@ -1,68 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -log4j.rootLogger=INFO, 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/0.7/lib/kafka-0.7.0.jar b/system_test/migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar deleted file mode 100644 index 982ddb953aaa1..0000000000000 Binary files a/system_test/migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar and /dev/null differ diff --git a/system_test/migration_tool_testsuite/0.7/lib/kafka-perf-0.7.0.jar b/system_test/migration_tool_testsuite/0.7/lib/kafka-perf-0.7.0.jar deleted file mode 100644 index d4f89d54907af..0000000000000 Binary files a/system_test/migration_tool_testsuite/0.7/lib/kafka-perf-0.7.0.jar and /dev/null differ diff --git a/system_test/migration_tool_testsuite/0.7/lib/zkclient-0.1.jar b/system_test/migration_tool_testsuite/0.7/lib/zkclient-0.1.jar deleted file mode 100644 index aebcc3491adb3..0000000000000 Binary files a/system_test/migration_tool_testsuite/0.7/lib/zkclient-0.1.jar and /dev/null differ diff --git a/system_test/migration_tool_testsuite/__init__.py b/system_test/migration_tool_testsuite/__init__.py deleted file mode 100644 index 8d1c8b69c3fce..0000000000000 --- a/system_test/migration_tool_testsuite/__init__.py +++ /dev/null @@ -1 +0,0 @@ - diff --git a/system_test/migration_tool_testsuite/config/migration_consumer.properties b/system_test/migration_tool_testsuite/config/migration_consumer.properties deleted file mode 100644 index 184f1dea16475..0000000000000 --- a/system_test/migration_tool_testsuite/config/migration_consumer.properties +++ /dev/null @@ -1,29 +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. -# see kafka.consumer.ConsumerConfig for more details - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=127.0.0.1:2188 - -# timeout in ms for connecting to zookeeper -zk.connectiontimeout.ms=1000000 - -#consumer group id -groupid=test-consumer-group - -#consumer timeout -#consumer.timeout.ms=5000 diff --git a/system_test/migration_tool_testsuite/config/migration_producer.properties b/system_test/migration_tool_testsuite/config/migration_producer.properties deleted file mode 100644 index 17b5928a1213f..0000000000000 --- a/system_test/migration_tool_testsuite/config/migration_producer.properties +++ /dev/null @@ -1,66 +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. -# see kafka.producer.ProducerConfig for more details - -############################# Producer Basics ############################# - -# need to set either broker.list or zk.connect - -# configure brokers statically -# format: host1:port1,host2:port2 ... -metadata.broker.list=localhost:9094,localhost:9095,localhost:9096 - -# discover brokers from ZK -#zk.connect= - -# zookeeper session timeout; default is 6000 -#zk.session.timeout.ms= - -# the max time that the client waits to establish a connection to zookeeper; default is 6000 -#zk.connection.timeout.ms - -# name of the partitioner class for partitioning events; default partition spreads data randomly -#partitioner.class= - -# specifies whether the messages are sent asynchronously (async) or synchronously (sync) -producer.type=sync - -# specify the compression codec for all data generated: 0: no compression, 1: gzip -compression.codec=0 - -# message encoder -serializer.class=kafka.serializer.DefaultEncoder - -# allow topic level compression -#compressed.topics= - -############################# Async Producer ############################# -# maximum time, in milliseconds, for buffering data on the producer queue -#queue.buffering.max.ms= - -# the maximum size of the blocking queue for buffering on the producer -#queue.buffering.max.messages= - -# Timeout for event enqueue: -# 0: events will be enqueued immediately or dropped if the queue is full -# -ve: enqueue will block indefinitely if the queue is full -# +ve: enqueue will block up to this many milliseconds if the queue is full -#queue.enqueue.timeout.ms= - -# the number of messages batched at the producer -#batch.num.messages= - -message.send.max.retries=3 -request.required.acks=1 diff --git a/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json b/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json deleted file mode 100644 index 608e3bdf79cc4..0000000000000 --- a/system_test/migration_tool_testsuite/testcase_9001/testcase_9001_properties.json +++ /dev/null @@ -1,125 +0,0 @@ -{ - "description": {"01":"To Test : 'Replication with Migration Tool'", - "02":"Set up 2 clusters such as : SOURCE => Migration Tool => TARGET", - "03":"Produce and consume messages to a single topic - single partition.", - "04":"This test sends messages to 3 replicas", - "05":"At the end it verifies the log size and contents", - "06":"Use a consumer to verify no message loss in TARGET cluster.", - "07":"Producer dimensions : mode:sync, acks:-1, comp:0", - "08":"Log segment size : 51200" - }, - "testcase_args": { - "bounce_migration_tool": "false", - "replica_factor": "3", - "num_partition": "1", - "num_iteration": "1", - "sleep_seconds_between_producer_calls": "1", - "message_producing_free_time_sec": "30", - "num_messages_to_produce_per_producer_call": "50" - }, - "entities": [ - { - "entity_id": "0", - "clientPort": "2188", - "dataDir": "/tmp/zookeeper_0", - "log_filename": "zookeeper_0.log", - "config_filename": "zookeeper_0.properties" - }, - { - "entity_id": "1", - "port": "9091", - "brokerid": "1", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_1_logs", - "log_filename": "kafka_server_1.log", - "config_filename": "kafka_server_1.properties" - }, - { - "entity_id": "2", - "port": "9092", - "brokerid": "2", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_2_logs", - "log_filename": "kafka_server_2.log", - "config_filename": "kafka_server_2.properties" - }, - { - "entity_id": "3", - "port": "9093", - "brokerid": "3", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_3_logs", - "log_filename": "kafka_server_3.log", - "config_filename": "kafka_server_3.properties" - }, - { - "entity_id": "4", - "port": "9094", - "broker.id": "4", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_4_logs", - "log_filename": "kafka_server_4.log", - "config_filename": "kafka_server_4.properties" - }, - { - "entity_id": "5", - "port": "9095", - "broker.id": "5", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_5_logs", - "log_filename": "kafka_server_5.log", - "config_filename": "kafka_server_5.properties" - }, - { - "entity_id": "6", - "port": "9096", - "broker.id": "6", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_6_logs", - "log_filename": "kafka_server_6.log", - "config_filename": "kafka_server_6.properties" - }, - { - "entity_id": "7", - "topic": "test_1", - "threads": "5", - "compression-codec": "0", - "message-size": "500", - "message": "500", - "request-num-acks": "-1", - "async": "false", - "log_filename": "producer_performance_7.log", - "config_filename": "producer_performance_7.properties" - }, - { - "entity_id": "8", - "topic": "test_1", - "group.id": "mytestgroup", - "consumer-timeout-ms": "10000", - "log_filename": "console_consumer_8.log", - "config_filename": "console_consumer_8.properties" - }, - { - "entity_id": "9", - "clientPort": "2191", - "dataDir": "/tmp/zookeeper_9", - "log_filename": "zookeeper_9.log", - "config_filename": "zookeeper_9.properties" - }, - { - "entity_id": "10", - "whitelist": ".*", - "num.producers": "2", - "num.streams": "2", - "producer.config": "migration_tool_testsuite/config/migration_producer.properties", - "consumer.config": "migration_tool_testsuite/config/migration_consumer.properties", - "zkclient.01.jar": "migration_tool_testsuite/0.7/lib/zkclient-0.1.jar", - "kafka.07.jar" : "migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar", - "log_filename": "migration_tool.log", - "config_filename": "migration_tool.properties" - } - ] -} diff --git a/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json b/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json deleted file mode 100644 index 333256c8f53a6..0000000000000 --- a/system_test/migration_tool_testsuite/testcase_9003/testcase_9003_properties.json +++ /dev/null @@ -1,138 +0,0 @@ -{ - "description": {"01":"To Test : 'Replication with Migration Tool'", - "02":"Set up 2 clusters such as : SOURCE => Migration Tool => TARGET", - "03":"Produce and consume messages to a single topic - single partition.", - "04":"This test sends messages to 3 replicas", - "05":"At the end it verifies the log size and contents", - "06":"Use a consumer to verify no message loss in TARGET cluster.", - "07":"Producer dimensions : mode:async, acks:-1, comp:1", - "08":"Log segment size : 51200" - }, - "testcase_args": { - "bounce_migration_tool": "true", - "bounced_entity_downtime_sec": "30", - "replica_factor": "3", - "num_partition": "1", - "num_iteration": "1", - "sleep_seconds_between_producer_calls": "1", - "message_producing_free_time_sec": "30", - "num_messages_to_produce_per_producer_call": "50" - }, - "entities": [ - { - "entity_id": "0", - "clientPort": "2188", - "dataDir": "/tmp/zookeeper_0", - "log_filename": "zookeeper_0.log", - "config_filename": "zookeeper_0.properties" - }, - { - "entity_id": "1", - "port": "9091", - "brokerid": "1", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_1_logs", - "log_filename": "kafka_server_1.log", - "config_filename": "kafka_server_1.properties" - }, - { - "entity_id": "2", - "port": "9092", - "brokerid": "2", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_2_logs", - "log_filename": "kafka_server_2.log", - "config_filename": "kafka_server_2.properties" - }, - { - "entity_id": "3", - "port": "9093", - "brokerid": "3", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_3_logs", - "log_filename": "kafka_server_3.log", - "config_filename": "kafka_server_3.properties" - }, - { - "entity_id": "4", - "port": "9094", - "broker.id": "4", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_4_logs", - "log_filename": "kafka_server_4.log", - "config_filename": "kafka_server_4.properties" - }, - { - "entity_id": "5", - "port": "9095", - "broker.id": "5", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_5_logs", - "log_filename": "kafka_server_5.log", - "config_filename": "kafka_server_5.properties" - }, - { - "entity_id": "6", - "port": "9096", - "broker.id": "6", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_6_logs", - "log_filename": "kafka_server_6.log", - "config_filename": "kafka_server_6.properties" - }, - { - "entity_id": "7", - "topic": "test_1", - "threads": "5", - "compression-codec": "1", - "message-size": "500", - "message": "500", - "request-num-acks": "-1", - "async": "true", - "log_filename": "producer_performance_7.log", - "config_filename": "producer_performance_7.properties" - }, - { - "entity_id": "8", - "topic": "test_1", - "group.id": "mytestgroup", - "consumer-timeout-ms": "10000", - "log_filename": "console_consumer_8.log", - "config_filename": "console_consumer_8.properties" - }, - { - "entity_id": "9", - "clientPort": "2191", - "dataDir": "/tmp/zookeeper_9", - "log_filename": "zookeeper_9.log", - "config_filename": "zookeeper_9.properties" - }, - { - "entity_id": "10", - "whitelist": ".*", - "num.producers": "2", - "num.streams": "2", - "producer.config": "migration_tool_testsuite/config/migration_producer.properties", - "consumer.config": "migration_tool_testsuite/config/migration_consumer.properties", - "zkclient.01.jar": "migration_tool_testsuite/0.7/lib/zkclient-0.1.jar", - "kafka.07.jar" : "migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar", - "log_filename": "migration_tool_10.log", - "config_filename": "migration_tool_10.properties" - }, - { - "entity_id": "11", - "whitelist": ".*", - "num.producers": "2", - "num.streams": "2", - "producer.config": "migration_tool_testsuite/config/migration_producer.properties", - "consumer.config": "migration_tool_testsuite/config/migration_consumer.properties", - "zkclient.01.jar": "migration_tool_testsuite/0.7/lib/zkclient-0.1.jar", - "kafka.07.jar" : "migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar", - "log_filename": "migration_tool_11.log", - "config_filename": "migration_tool_11.properties" - } - ] -} diff --git a/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json b/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json deleted file mode 100644 index b2a6e85e575b4..0000000000000 --- a/system_test/migration_tool_testsuite/testcase_9004/testcase_9004_properties.json +++ /dev/null @@ -1,138 +0,0 @@ -{ - "description": {"01":"To Test : 'Replication with Migration Tool'", - "02":"Set up 2 clusters such as : SOURCE => Migration Tool => TARGET", - "03":"Produce and consume messages to a single topic - single partition.", - "04":"This test sends messages to 3 replicas", - "05":"At the end it verifies the log size and contents", - "06":"Use a consumer to verify no message loss in TARGET cluster.", - "07":"Producer dimensions : mode:async, acks:1, comp:1", - "08":"Log segment size : 51200" - }, - "testcase_args": { - "bounce_migration_tool": "true", - "bounced_entity_downtime_sec": "30", - "replica_factor": "3", - "num_partition": "1", - "num_iteration": "1", - "sleep_seconds_between_producer_calls": "1", - "message_producing_free_time_sec": "30", - "num_messages_to_produce_per_producer_call": "50" - }, - "entities": [ - { - "entity_id": "0", - "clientPort": "2188", - "dataDir": "/tmp/zookeeper_0", - "log_filename": "zookeeper_0.log", - "config_filename": "zookeeper_0.properties" - }, - { - "entity_id": "1", - "port": "9091", - "brokerid": "1", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_1_logs", - "log_filename": "kafka_server_1.log", - "config_filename": "kafka_server_1.properties" - }, - { - "entity_id": "2", - "port": "9092", - "brokerid": "2", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_2_logs", - "log_filename": "kafka_server_2.log", - "config_filename": "kafka_server_2.properties" - }, - { - "entity_id": "3", - "port": "9093", - "brokerid": "3", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_3_logs", - "log_filename": "kafka_server_3.log", - "config_filename": "kafka_server_3.properties" - }, - { - "entity_id": "4", - "port": "9094", - "broker.id": "4", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_4_logs", - "log_filename": "kafka_server_4.log", - "config_filename": "kafka_server_4.properties" - }, - { - "entity_id": "5", - "port": "9095", - "broker.id": "5", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_5_logs", - "log_filename": "kafka_server_5.log", - "config_filename": "kafka_server_5.properties" - }, - { - "entity_id": "6", - "port": "9096", - "broker.id": "6", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_6_logs", - "log_filename": "kafka_server_6.log", - "config_filename": "kafka_server_6.properties" - }, - { - "entity_id": "7", - "topic": "test_1", - "threads": "5", - "compression-codec": "1", - "message-size": "500", - "message": "500", - "request-num-acks": "1", - "async": "true", - "log_filename": "producer_performance_7.log", - "config_filename": "producer_performance_7.properties" - }, - { - "entity_id": "8", - "topic": "test_1", - "group.id": "mytestgroup", - "consumer-timeout-ms": "10000", - "log_filename": "console_consumer_8.log", - "config_filename": "console_consumer_8.properties" - }, - { - "entity_id": "9", - "clientPort": "2191", - "dataDir": "/tmp/zookeeper_9", - "log_filename": "zookeeper_9.log", - "config_filename": "zookeeper_9.properties" - }, - { - "entity_id": "10", - "whitelist": ".*", - "num.producers": "2", - "num.streams": "2", - "producer.config": "migration_tool_testsuite/config/migration_producer.properties", - "consumer.config": "migration_tool_testsuite/config/migration_consumer.properties", - "zkclient.01.jar": "migration_tool_testsuite/0.7/lib/zkclient-0.1.jar", - "kafka.07.jar" : "migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar", - "log_filename": "migration_tool_10.log", - "config_filename": "migration_tool_10.properties" - }, - { - "entity_id": "11", - "whitelist": ".*", - "num.producers": "2", - "num.streams": "2", - "producer.config": "migration_tool_testsuite/config/migration_producer.properties", - "consumer.config": "migration_tool_testsuite/config/migration_consumer.properties", - "zkclient.01.jar": "migration_tool_testsuite/0.7/lib/zkclient-0.1.jar", - "kafka.07.jar" : "migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar", - "log_filename": "migration_tool_11.log", - "config_filename": "migration_tool_11.properties" - } - ] -} diff --git a/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json b/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json deleted file mode 100644 index ddbc90506902c..0000000000000 --- a/system_test/migration_tool_testsuite/testcase_9005/testcase_9005_properties.json +++ /dev/null @@ -1,168 +0,0 @@ -{ - "description": {"01":"To Test : 'Replication with Migration Tool'", - "02":"Set up 2 clusters such as : SOURCE => Migration Tool => TARGET", - "03":"Produce and consume messages to 2 topics - 2 partitions.", - "04":"This test sends messages to 3 replicas", - "05":"At the end it verifies the log size and contents", - "06":"Use a consumer to verify no message loss in TARGET cluster.", - "07":"Producer dimensions : mode:async, acks:-1, comp:1", - "08":"Log segment size : 51200" - }, - "testcase_args": { - "bounce_migration_tool": "true", - "bounced_entity_downtime_sec": "30", - "replica_factor": "3", - "num_partition": "2", - "num_iteration": "1", - "sleep_seconds_between_producer_calls": "1", - "message_producing_free_time_sec": "30", - "num_messages_to_produce_per_producer_call": "50" - }, - "entities": [ - { - "entity_id": "0", - "clientPort": "2188", - "dataDir": "/tmp/zookeeper_0", - "log_filename": "zookeeper_0.log", - "config_filename": "zookeeper_0.properties" - }, - { - "entity_id": "1", - "clientPort": "2191", - "dataDir": "/tmp/zookeeper_1", - "log_filename": "zookeeper_1.log", - "config_filename": "zookeeper_1.properties" - }, - - - { - "entity_id": "2", - "port": "9091", - "brokerid": "1", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_2_logs", - "log_filename": "kafka_server_2.log", - "config_filename": "kafka_server_2.properties" - }, - { - "entity_id": "3", - "port": "9092", - "brokerid": "2", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_3_logs", - "log_filename": "kafka_server_3.log", - "config_filename": "kafka_server_3.properties" - }, - { - "entity_id": "4", - "port": "9093", - "brokerid": "3", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_4_logs", - "log_filename": "kafka_server_4.log", - "config_filename": "kafka_server_4.properties" - }, - - - { - "entity_id": "5", - "port": "9094", - "broker.id": "4", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_5_logs", - "log_filename": "kafka_server_5.log", - "config_filename": "kafka_server_5.properties" - }, - { - "entity_id": "6", - "port": "9095", - "broker.id": "5", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_6_logs", - "log_filename": "kafka_server_6.log", - "config_filename": "kafka_server_6.properties" - }, - { - "entity_id": "7", - "port": "9096", - "broker.id": "6", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_7_logs", - "log_filename": "kafka_server_7.log", - "config_filename": "kafka_server_7.properties" - }, - - - { - "entity_id": "8", - "topic": "test_1", - "threads": "5", - "compression-codec": "1", - "message-size": "500", - "message": "500", - "request-num-acks": "-1", - "async": "true", - "log_filename": "producer_performance_8.log", - "config_filename": "producer_performance_8.properties" - }, - { - "entity_id": "9", - "topic": "test_2", - "threads": "5", - "compression-codec": "1", - "message-size": "500", - "message": "500", - "request-num-acks": "-1", - "async": "true", - "log_filename": "producer_performance_9.log", - "config_filename": "producer_performance_9.properties" - }, - - - { - "entity_id": "10", - "topic": "test_1", - "group.id": "mytestgroup", - "consumer-timeout-ms": "10000", - "log_filename": "console_consumer_10.log", - "config_filename": "console_consumer_10.properties" - }, - { - "entity_id": "11", - "topic": "test_2", - "group.id": "mytestgroup", - "consumer-timeout-ms": "10000", - "log_filename": "console_consumer_11.log", - "config_filename": "console_consumer_11.properties" - }, - - - { - "entity_id": "12", - "whitelist": ".*", - "num.producers": "2", - "num.streams": "2", - "producer.config": "migration_tool_testsuite/config/migration_producer.properties", - "consumer.config": "migration_tool_testsuite/config/migration_consumer.properties", - "zkclient.01.jar": "migration_tool_testsuite/0.7/lib/zkclient-0.1.jar", - "kafka.07.jar" : "migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar", - "log_filename": "migration_tool_12.log", - "config_filename": "migration_tool_12.properties" - }, - { - "entity_id": "13", - "whitelist": ".*", - "num.producers": "2", - "num.streams": "2", - "producer.config": "migration_tool_testsuite/config/migration_producer.properties", - "consumer.config": "migration_tool_testsuite/config/migration_consumer.properties", - "zkclient.01.jar": "migration_tool_testsuite/0.7/lib/zkclient-0.1.jar", - "kafka.07.jar" : "migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar", - "log_filename": "migration_tool_13.log", - "config_filename": "migration_tool_13.properties" - } - ] -} diff --git a/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json b/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json deleted file mode 100644 index 21b4c40e207ac..0000000000000 --- a/system_test/migration_tool_testsuite/testcase_9006/testcase_9006_properties.json +++ /dev/null @@ -1,168 +0,0 @@ -{ - "description": {"01":"To Test : 'Replication with Migration Tool'", - "02":"Set up 2 clusters such as : SOURCE => Migration Tool => TARGET", - "03":"Produce and consume messages to 2 topics - 2 partitions.", - "04":"This test sends messages to 3 replicas", - "05":"At the end it verifies the log size and contents", - "06":"Use a consumer to verify no message loss in TARGET cluster.", - "07":"Producer dimensions : mode:async, acks:1, comp:1", - "08":"Log segment size : 51200" - }, - "testcase_args": { - "bounce_migration_tool": "true", - "bounced_entity_downtime_sec": "30", - "replica_factor": "3", - "num_partition": "2", - "num_iteration": "1", - "sleep_seconds_between_producer_calls": "1", - "message_producing_free_time_sec": "30", - "num_messages_to_produce_per_producer_call": "50" - }, - "entities": [ - { - "entity_id": "0", - "clientPort": "2188", - "dataDir": "/tmp/zookeeper_0", - "log_filename": "zookeeper_0.log", - "config_filename": "zookeeper_0.properties" - }, - { - "entity_id": "1", - "clientPort": "2191", - "dataDir": "/tmp/zookeeper_1", - "log_filename": "zookeeper_1.log", - "config_filename": "zookeeper_1.properties" - }, - - - { - "entity_id": "2", - "port": "9091", - "brokerid": "1", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_2_logs", - "log_filename": "kafka_server_2.log", - "config_filename": "kafka_server_2.properties" - }, - { - "entity_id": "3", - "port": "9092", - "brokerid": "2", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_3_logs", - "log_filename": "kafka_server_3.log", - "config_filename": "kafka_server_3.properties" - }, - { - "entity_id": "4", - "port": "9093", - "brokerid": "3", - "version": "0.7", - "log.file.size": "51200", - "log.dir": "/tmp/kafka_server_4_logs", - "log_filename": "kafka_server_4.log", - "config_filename": "kafka_server_4.properties" - }, - - - { - "entity_id": "5", - "port": "9094", - "broker.id": "4", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_5_logs", - "log_filename": "kafka_server_5.log", - "config_filename": "kafka_server_5.properties" - }, - { - "entity_id": "6", - "port": "9095", - "broker.id": "5", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_6_logs", - "log_filename": "kafka_server_6.log", - "config_filename": "kafka_server_6.properties" - }, - { - "entity_id": "7", - "port": "9096", - "broker.id": "6", - "log.segment.bytes": "51200", - "log.dir": "/tmp/kafka_server_7_logs", - "log_filename": "kafka_server_7.log", - "config_filename": "kafka_server_7.properties" - }, - - - { - "entity_id": "8", - "topic": "test_1", - "threads": "5", - "compression-codec": "1", - "message-size": "500", - "message": "500", - "request-num-acks": "1", - "async": "true", - "log_filename": "producer_performance_8.log", - "config_filename": "producer_performance_8.properties" - }, - { - "entity_id": "9", - "topic": "test_2", - "threads": "5", - "compression-codec": "1", - "message-size": "500", - "message": "500", - "request-num-acks": "1", - "async": "true", - "log_filename": "producer_performance_9.log", - "config_filename": "producer_performance_9.properties" - }, - - - { - "entity_id": "10", - "topic": "test_1", - "group.id": "mytestgroup", - "consumer-timeout-ms": "10000", - "log_filename": "console_consumer_10.log", - "config_filename": "console_consumer_10.properties" - }, - { - "entity_id": "11", - "topic": "test_2", - "group.id": "mytestgroup", - "consumer-timeout-ms": "10000", - "log_filename": "console_consumer_11.log", - "config_filename": "console_consumer_11.properties" - }, - - - { - "entity_id": "12", - "whitelist": ".*", - "num.producers": "2", - "num.streams": "2", - "producer.config": "migration_tool_testsuite/config/migration_producer.properties", - "consumer.config": "migration_tool_testsuite/config/migration_consumer.properties", - "zkclient.01.jar": "migration_tool_testsuite/0.7/lib/zkclient-0.1.jar", - "kafka.07.jar" : "migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar", - "log_filename": "migration_tool_12.log", - "config_filename": "migration_tool_12.properties" - }, - { - "entity_id": "13", - "whitelist": ".*", - "num.producers": "2", - "num.streams": "2", - "producer.config": "migration_tool_testsuite/config/migration_producer.properties", - "consumer.config": "migration_tool_testsuite/config/migration_consumer.properties", - "zkclient.01.jar": "migration_tool_testsuite/0.7/lib/zkclient-0.1.jar", - "kafka.07.jar" : "migration_tool_testsuite/0.7/lib/kafka-0.7.0.jar", - "log_filename": "migration_tool_13.log", - "config_filename": "migration_tool_13.properties" - } - ] -} diff --git a/system_test/mirror_maker/README b/system_test/mirror_maker/README deleted file mode 100644 index da53c14446162..0000000000000 --- a/system_test/mirror_maker/README +++ /dev/null @@ -1,22 +0,0 @@ -This test replicates messages from two source kafka clusters into one target -kafka cluster using the mirror-maker tool. At the end, the messages produced -at the source brokers should match that at the target brokers. - -To run this test, do -bin/run-test.sh - -In the event of failure, by default the brokers and zookeepers remain running -to make it easier to debug the issue - hit Ctrl-C to shut them down. You can -change this behavior by setting the action_on_fail flag in the script to "exit" -or "proceed", in which case a snapshot of all the logs and directories is -placed in the test's base directory. - -It is a good idea to run the test in a loop. E.g.: - -:>/tmp/mirrormaker_test.log -for i in {1..10}; do echo "run $i"; ./bin/run-test.sh 2>1 >> /tmp/mirrormaker_test.log; done -tail -F /tmp/mirrormaker_test.log - -grep -ic passed /tmp/mirrormaker_test.log -grep -ic failed /tmp/mirrormaker_test.log - diff --git a/system_test/mirror_maker/bin/expected.out b/system_test/mirror_maker/bin/expected.out deleted file mode 100644 index 0a1bbafcac4a1..0000000000000 --- a/system_test/mirror_maker/bin/expected.out +++ /dev/null @@ -1,18 +0,0 @@ -start the servers ... -start producing messages ... -wait for consumer to finish consuming ... -[2011-05-17 14:49:11,605] INFO Creating async producer for broker id = 2 at localhost:9091 (kafka.producer.ProducerPool) -[2011-05-17 14:49:11,606] INFO Creating async producer for broker id = 1 at localhost:9092 (kafka.producer.ProducerPool) -[2011-05-17 14:49:11,607] INFO Creating async producer for broker id = 3 at localhost:9090 (kafka.producer.ProducerPool) -thread 0: 400000 messages sent 3514012.1233 nMsg/sec 3.3453 MBs/sec -[2011-05-17 14:49:34,382] INFO Closing all async producers (kafka.producer.ProducerPool) -[2011-05-17 14:49:34,383] INFO Closed AsyncProducer (kafka.producer.async.AsyncProducer) -[2011-05-17 14:49:34,384] INFO Closed AsyncProducer (kafka.producer.async.AsyncProducer) -[2011-05-17 14:49:34,385] INFO Closed AsyncProducer (kafka.producer.async.AsyncProducer) -Total Num Messages: 400000 bytes: 79859641 in 22.93 secs -Messages/sec: 17444.3960 -MB/sec: 3.3214 -test passed -stopping the servers -bin/../../../bin/zookeeper-server-start.sh: line 9: 22584 Terminated $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@ -bin/../../../bin/zookeeper-server-start.sh: line 9: 22585 Terminated $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@ diff --git a/system_test/mirror_maker/bin/run-test.sh b/system_test/mirror_maker/bin/run-test.sh deleted file mode 100644 index e5e6c08f2c153..0000000000000 --- a/system_test/mirror_maker/bin/run-test.sh +++ /dev/null @@ -1,357 +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. - -readonly num_messages=10000 -readonly message_size=100 -readonly action_on_fail="proceed" -# readonly action_on_fail="wait" - -readonly test_start_time="$(date +%s)" - -readonly base_dir=$(dirname $0)/.. - -info() { - echo -e "$(date +"%Y-%m-%d %H:%M:%S") $*" -} - -kill_child_processes() { - isTopmost=$1 - curPid=$2 - childPids=$(ps a -o pid= -o ppid= | grep "${curPid}$" | awk '{print $1;}') - for childPid in $childPids - do - kill_child_processes 0 $childPid - done - if [ $isTopmost -eq 0 ]; then - kill -15 $curPid 2> /dev/null - fi -} - -cleanup() { - info "cleaning up" - - pid_zk_source1= - pid_zk_source2= - pid_zk_target= - pid_kafka_source_1_1= - pid_kafka_source_1_2= - pid_kafka_source_2_1= - pid_kafka_source_2_2= - pid_kafka_target_1_1= - pid_kafka_target_1_2= - pid_producer= - pid_mirrormaker_1= - pid_mirrormaker_2= - - rm -rf /tmp/zookeeper* - - rm -rf /tmp/kafka* -} - -begin_timer() { - t_begin=$(date +%s) -} - -end_timer() { - t_end=$(date +%s) -} - -start_zk() { - info "starting zookeepers" - $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_1.properties 2>&1 > $base_dir/zookeeper_source-1.log & - pid_zk_source1=$! - $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_2.properties 2>&1 > $base_dir/zookeeper_source-2.log & - pid_zk_source2=$! - $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_target.properties 2>&1 > $base_dir/zookeeper_target.log & - pid_zk_target=$! -} - -start_source_servers() { - info "starting source cluster" - - JMX_PORT=1111 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_1.properties 2>&1 > $base_dir/kafka_source-1-1.log & - pid_kafka_source_1_1=$! - JMX_PORT=2222 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_2.properties 2>&1 > $base_dir/kafka_source-1-2.log & - pid_kafka_source_1_2=$! - JMX_PORT=3333 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_1.properties 2>&1 > $base_dir/kafka_source-2-1.log & - pid_kafka_source_2_1=$! - JMX_PORT=4444 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_2.properties 2>&1 > $base_dir/kafka_source-2-2.log & - pid_kafka_source_2_2=$! -} - -start_target_servers() { - info "starting mirror cluster" - JMX_PORT=5555 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_1.properties 2>&1 > $base_dir/kafka_target-1-1.log & - pid_kafka_target_1_1=$! - JMX_PORT=6666 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_2.properties 2>&1 > $base_dir/kafka_target-1-2.log & - pid_kafka_target_1_2=$! -} - -shutdown_servers() { - info "stopping mirror-maker" - if [ "x${pid_mirrormaker_1}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_1}; fi - # sleep to avoid rebalancing during shutdown - sleep 2 - if [ "x${pid_mirrormaker_2}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_2}; fi - - info "stopping producer" - if [ "x${pid_producer}" != "x" ]; then kill_child_processes 0 ${pid_producer}; fi - - info "shutting down target servers" - if [ "x${pid_kafka_target_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_1}; fi - if [ "x${pid_kafka_target_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_2}; fi - sleep 2 - - info "shutting down source servers" - if [ "x${pid_kafka_source_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_1}; fi - if [ "x${pid_kafka_source_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_2}; fi - if [ "x${pid_kafka_source_2_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_1}; fi - if [ "x${pid_kafka_source_2_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_2}; fi - - info "shutting down zookeeper servers" - if [ "x${pid_zk_target}" != "x" ]; then kill_child_processes 0 ${pid_zk_target}; fi - if [ "x${pid_zk_source1}" != "x" ]; then kill_child_processes 0 ${pid_zk_source1}; fi - if [ "x${pid_zk_source2}" != "x" ]; then kill_child_processes 0 ${pid_zk_source2}; fi -} - -start_producer() { - topic=$1 - zk=$2 - info "start producing messages for topic $topic to zookeeper $zk ..." - $base_dir/../../bin/kafka-run-class.sh kafka.perf.ProducerPerformance --brokerinfo zk.connect=$zk --topics $topic --messages $num_messages --message-size $message_size --batch-size 200 --vary-message-size --threads 1 --reporting-interval $num_messages --async 2>&1 > $base_dir/producer_performance.log & - pid_producer=$! -} - -# Usage: wait_partition_done ([kafka-server] [topic] [partition-id])+ -wait_partition_done() { - n_tuples=$(($# / 3)) - - i=1 - while (($#)); do - kafka_server[i]=$1 - topic[i]=$2 - partitionid[i]=$3 - prev_offset[i]=0 - info "\twaiting for partition on server ${kafka_server[i]}, topic ${topic[i]}, partition ${partitionid[i]}" - i=$((i+1)) - shift 3 - done - - all_done=0 - - # set -x - while [[ $all_done != 1 ]]; do - sleep 4 - i=$n_tuples - all_done=1 - for ((i=1; i <= $n_tuples; i++)); do - cur_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server ${kafka_server[i]} --topic ${topic[i]} --partition ${partitionid[i]} --time -1 --offsets 1 | tail -1) - if [ "x$cur_size" != "x${prev_offset[i]}" ]; then - all_done=0 - prev_offset[i]=$cur_size - fi - done - done - -} - -cmp_logs() { - topic=$1 - info "comparing source and target logs for topic $topic" - source_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9090 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - source_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9091 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - source_part2_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9092 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - source_part3_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9093 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - target_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9094 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - target_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9095 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - if [ "x$source_part0_size" == "x" ]; then source_part0_size=0; fi - if [ "x$source_part1_size" == "x" ]; then source_part1_size=0; fi - if [ "x$source_part2_size" == "x" ]; then source_part2_size=0; fi - if [ "x$source_part3_size" == "x" ]; then source_part3_size=0; fi - if [ "x$target_part0_size" == "x" ]; then target_part0_size=0; fi - if [ "x$target_part1_size" == "x" ]; then target_part1_size=0; fi - expected_size=$(($source_part0_size + $source_part1_size + $source_part2_size + $source_part3_size)) - actual_size=$(($target_part0_size + $target_part1_size)) - if [ "x$expected_size" != "x$actual_size" ] - then - info "source size: $expected_size target size: $actual_size" - return 1 - else - return 0 - fi -} - -take_fail_snapshot() { - snapshot_dir="$base_dir/failed-${snapshot_prefix}-${test_start_time}" - mkdir $snapshot_dir - for dir in /tmp/zookeeper_source{1..2} /tmp/zookeeper_target /tmp/kafka-source-{1..2}-{1..2}-logs /tmp/kafka-target{1..2}-logs; do - if [ -d $dir ]; then - cp -r $dir $snapshot_dir - fi - done -} - -# Usage: process_test_result -# result: last test result -# action_on_fail: (exit|wait|proceed) -# ("wait" is useful if you want to troubleshoot using zookeeper) -process_test_result() { - result=$1 - if [ $1 -eq 0 ]; then - info "test passed" - else - info "test failed" - case "$2" in - "wait") info "waiting: hit Ctrl-c to quit" - wait - ;; - "exit") shutdown_servers - take_fail_snapshot - exit $result - ;; - *) shutdown_servers - take_fail_snapshot - info "proceeding" - ;; - esac - fi -} - -test_whitelists() { - info "### Testing whitelists" - snapshot_prefix="whitelist-test" - - cleanup - start_zk - start_source_servers - start_target_servers - sleep 4 - - info "starting mirror makers" - JMX_PORT=7777 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & - pid_mirrormaker_1=$! - JMX_PORT=8888 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_2.log & - pid_mirrormaker_2=$! - - begin_timer - - start_producer whitetopic01 localhost:2181 - start_producer whitetopic01 localhost:2182 - info "waiting for whitetopic01 producers to finish producing ..." - wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0 kafka://localhost:9092 whitetopic01 0 kafka://localhost:9093 whitetopic01 0 - - start_producer whitetopic02 localhost:2181 - start_producer whitetopic03 localhost:2181 - start_producer whitetopic04 localhost:2182 - info "waiting for whitetopic02,whitetopic03,whitetopic04 producers to finish producing ..." - wait_partition_done kafka://localhost:9090 whitetopic02 0 kafka://localhost:9091 whitetopic02 0 kafka://localhost:9090 whitetopic03 0 kafka://localhost:9091 whitetopic03 0 kafka://localhost:9092 whitetopic04 0 kafka://localhost:9093 whitetopic04 0 - - start_producer blacktopic01 localhost:2182 - info "waiting for blacktopic01 producer to finish producing ..." - wait_partition_done kafka://localhost:9092 blacktopic01 0 kafka://localhost:9093 blacktopic01 0 - - info "waiting for consumer to finish consuming ..." - - wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0 kafka://localhost:9094 whitetopic02 0 kafka://localhost:9095 whitetopic02 0 kafka://localhost:9094 whitetopic03 0 kafka://localhost:9095 whitetopic03 0 kafka://localhost:9094 whitetopic04 0 kafka://localhost:9095 whitetopic04 0 - - end_timer - info "embedded consumer took $((t_end - t_begin)) seconds" - - sleep 2 - - # if [[ -d /tmp/kafka-target-1-1-logs/blacktopic01 || /tmp/kafka-target-1-2-logs/blacktopic01 ]]; then - # echo "blacktopic01 found on target cluster" - # result=1 - # else - # cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04 - # result=$? - # fi - - cmp_logs blacktopic01 - - cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04 - result=$? - - return $result -} - -test_blacklists() { - info "### Testing blacklists" - snapshot_prefix="blacklist-test" - cleanup - start_zk - start_source_servers - start_target_servers - sleep 4 - - info "starting mirror maker" - $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/blacklisttest.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --blacklist="black.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & - pid_mirrormaker_1=$! - - start_producer blacktopic01 localhost:2181 - start_producer blacktopic02 localhost:2181 - info "waiting for producer to finish producing blacktopic01,blacktopic02 ..." - wait_partition_done kafka://localhost:9090 blacktopic01 0 kafka://localhost:9091 blacktopic01 0 kafka://localhost:9090 blacktopic02 0 kafka://localhost:9091 blacktopic02 0 - - begin_timer - - start_producer whitetopic01 localhost:2181 - info "waiting for producer to finish producing whitetopic01 ..." - wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0 - - info "waiting for consumer to finish consuming ..." - wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0 - - end_timer - - info "embedded consumer took $((t_end - t_begin)) seconds" - - sleep 2 - - cmp_logs blacktopic01 || cmp_logs blacktopic02 - if [ $? -eq 0 ]; then - return 1 - fi - - cmp_logs whitetopic01 - return $? -} - -# main test begins - -echo "Test-$test_start_time" - -# Ctrl-c trap. Catches INT signal -trap "shutdown_servers; exit 0" INT - -test_whitelists -result=$? - -process_test_result $result $action_on_fail - -shutdown_servers - -sleep 2 - -test_blacklists -result=$? - -process_test_result $result $action_on_fail - -shutdown_servers - -exit $result - diff --git a/system_test/mirror_maker/config/blacklisttest.consumer.properties b/system_test/mirror_maker/config/blacklisttest.consumer.properties deleted file mode 100644 index ff1201582ff1c..0000000000000 --- a/system_test/mirror_maker/config/blacklisttest.consumer.properties +++ /dev/null @@ -1,28 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# see kafka.consumer.ConsumerConfig for more details - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -#consumer group id -group.id=group1 -shallow.iterator.enable=true - diff --git a/system_test/mirror_maker/config/mirror_producer.properties b/system_test/mirror_maker/config/mirror_producer.properties deleted file mode 100644 index aa8be6504a7a4..0000000000000 --- a/system_test/mirror_maker/config/mirror_producer.properties +++ /dev/null @@ -1,30 +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. -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2183 -# broker.list=1:localhost:9094,2:localhost:9095 - -# timeout in ms for connecting to zookeeper -# zk.connection.timeout.ms=1000000 - -producer.type=async - -# to avoid dropping events if the queue is full, wait indefinitely -queue.enqueue.timeout.ms=-1 - -num.producers.per.broker=2 - diff --git a/system_test/mirror_maker/config/server_source_1_1.properties b/system_test/mirror_maker/config/server_source_1_1.properties deleted file mode 100644 index 2f070a74e8bd6..0000000000000 --- a/system_test/mirror_maker/config/server_source_1_1.properties +++ /dev/null @@ -1,76 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=1 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9090 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-source-1-1-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=10000000 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - diff --git a/system_test/mirror_maker/config/server_source_1_2.properties b/system_test/mirror_maker/config/server_source_1_2.properties deleted file mode 100644 index f9353e8934dcb..0000000000000 --- a/system_test/mirror_maker/config/server_source_1_2.properties +++ /dev/null @@ -1,76 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=2 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9091 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-source-1-2-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - diff --git a/system_test/mirror_maker/config/server_source_2_1.properties b/system_test/mirror_maker/config/server_source_2_1.properties deleted file mode 100644 index daa01ad463ec1..0000000000000 --- a/system_test/mirror_maker/config/server_source_2_1.properties +++ /dev/null @@ -1,76 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=1 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9092 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-source-2-1-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2182 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - diff --git a/system_test/mirror_maker/config/server_source_2_2.properties b/system_test/mirror_maker/config/server_source_2_2.properties deleted file mode 100644 index be6fdfcc11dca..0000000000000 --- a/system_test/mirror_maker/config/server_source_2_2.properties +++ /dev/null @@ -1,76 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=2 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9093 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-source-2-2-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2182 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - diff --git a/system_test/mirror_maker/config/server_target_1_1.properties b/system_test/mirror_maker/config/server_target_1_1.properties deleted file mode 100644 index d37955a66dcde..0000000000000 --- a/system_test/mirror_maker/config/server_target_1_1.properties +++ /dev/null @@ -1,78 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=1 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9094 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-target-1-1-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2183 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - -# topic partition count map -# topic.partition.count.map=topic1:3, topic2:4 diff --git a/system_test/mirror_maker/config/server_target_1_2.properties b/system_test/mirror_maker/config/server_target_1_2.properties deleted file mode 100644 index aa7546cfc8f5a..0000000000000 --- a/system_test/mirror_maker/config/server_target_1_2.properties +++ /dev/null @@ -1,78 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=2 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9095 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-target-1-2-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2183 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - -# topic partition count map -# topic.partition.count.map=topic1:3, topic2:4 diff --git a/system_test/mirror_maker/config/whitelisttest_1.consumer.properties b/system_test/mirror_maker/config/whitelisttest_1.consumer.properties deleted file mode 100644 index ff1201582ff1c..0000000000000 --- a/system_test/mirror_maker/config/whitelisttest_1.consumer.properties +++ /dev/null @@ -1,28 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# see kafka.consumer.ConsumerConfig for more details - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -#consumer group id -group.id=group1 -shallow.iterator.enable=true - diff --git a/system_test/mirror_maker/config/whitelisttest_2.consumer.properties b/system_test/mirror_maker/config/whitelisttest_2.consumer.properties deleted file mode 100644 index f1a902b100ad5..0000000000000 --- a/system_test/mirror_maker/config/whitelisttest_2.consumer.properties +++ /dev/null @@ -1,28 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# see kafka.consumer.ConsumerConfig for more details - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2182 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -#consumer group id -group.id=group1 -shallow.iterator.enable=true - diff --git a/system_test/mirror_maker/config/zookeeper_source_1.properties b/system_test/mirror_maker/config/zookeeper_source_1.properties deleted file mode 100644 index f85179616ebba..0000000000000 --- a/system_test/mirror_maker/config/zookeeper_source_1.properties +++ /dev/null @@ -1,18 +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. -# the directory where the snapshot is stored. -dataDir=/tmp/zookeeper_source-1 -# the port at which the clients will connect -clientPort=2181 diff --git a/system_test/mirror_maker_testsuite/config/mirror_producer.properties b/system_test/mirror_maker_testsuite/config/mirror_producer.properties index b2bf2c23b6b39..f94bebd3f5a05 100644 --- a/system_test/mirror_maker_testsuite/config/mirror_producer.properties +++ b/system_test/mirror_maker_testsuite/config/mirror_producer.properties @@ -1,6 +1,12 @@ -producer.type=async -queue.enqueue.timeout.ms=-1 +# old producer metadata.broker.list=localhost:9094 compression.codec=0 -message.send.max.retries=3 +request.retries=3 request.required.acks=1 + +# new producer +block.on.buffer.full=true +bootstrap.servers=localhost:9094 +compression.type=none +retries=3 +acks=1 diff --git a/system_test/mirror_maker_testsuite/config/server.properties b/system_test/mirror_maker_testsuite/config/server.properties index c6284122e3dfa..9717cd6319389 100644 --- a/system_test/mirror_maker_testsuite/config/server.properties +++ b/system_test/mirror_maker_testsuite/config/server.properties @@ -128,7 +128,6 @@ log.index.size.max.bytes=10485760 log.index.interval.bytes=4096 auto.create.topics.enable=true controller.socket.timeout.ms=30000 -controller.message.queue.size=10 default.replication.factor=1 replica.lag.time.max.ms=10000 replica.lag.max.messages=4000 diff --git a/system_test/mirror_maker_testsuite/mirror_maker_test.py b/system_test/mirror_maker_testsuite/mirror_maker_test.py index dfffb4e977ef2..48f9ff6b2810f 100644 --- a/system_test/mirror_maker_testsuite/mirror_maker_test.py +++ b/system_test/mirror_maker_testsuite/mirror_maker_test.py @@ -166,7 +166,7 @@ def runTest(self): time.sleep(5) self.log_message("creating topics") - kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.create_topic_for_producer_performance(self.systemTestEnv, self.testcaseEnv) self.anonLogger.info("sleeping for 5s") time.sleep(5) @@ -248,6 +248,7 @@ def runTest(self): str(self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]) + "]", extra=self.d) if self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]: time.sleep(1) + self.testcaseEnv.lock.release() self.logger.info("all producer threads completed", extra=self.d) break time.sleep(1) @@ -314,6 +315,7 @@ def runTest(self): except Exception as e: self.log_message("Exception while running test {0}".format(e)) traceback.print_exc() + self.testcaseEnv.validationStatusDict["Test completed"] = "FAILED" finally: if not skipThisTestCase and not self.systemTestEnv.printTestDescriptionsOnly: diff --git a/system_test/mirror_maker_testsuite/testcase_15001/testcase_15001_properties.json b/system_test/mirror_maker_testsuite/testcase_15001/testcase_15001_properties.json new file mode 100644 index 0000000000000..9dd3477e70a53 --- /dev/null +++ b/system_test/mirror_maker_testsuite/testcase_15001/testcase_15001_properties.json @@ -0,0 +1,158 @@ +{ + "description": {"01":"To Test : 'Replication with Mirror Maker'", + "02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET", + "03":"Set up 2-node Zk cluster for both SOURCE & TARGET", + "04":"Produce and consume messages to a single topic - single partition.", + "05":"This test sends messages to 3 replicas", + "06":"At the end it verifies the log size and contents", + "07":"Use a consumer to verify no message loss in TARGET cluster.", + "08":"Producer dimensions : mode:sync, acks:-1, comp:0", + "09":"Log segment size : 10240" + }, + "testcase_args": { + "bounce_leader": "false", + "bounce_mirror_maker": "false", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "clientPort": "2118", + "dataDir": "/tmp/zookeeper_1", + "log_filename": "zookeeper_1.log", + "config_filename": "zookeeper_1.properties" + }, + + { + "entity_id": "2", + "clientPort": "2128", + "dataDir": "/tmp/zookeeper_2", + "log_filename": "zookeeper_2.log", + "config_filename": "zookeeper_2.properties" + }, + { + "entity_id": "3", + "clientPort": "2138", + "dataDir": "/tmp/zookeeper_3", + "log_filename": "zookeeper_3.log", + "config_filename": "zookeeper_3.properties" + }, + + { + "entity_id": "4", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_5.log", + "config_filename": "kafka_server_5.properties" + }, + { + "entity_id": "6", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_6.log", + "config_filename": "kafka_server_6.properties" + }, + { + "entity_id": "7", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_7.log", + "config_filename": "kafka_server_7.properties" + }, + { + "entity_id": "8", + "port": "9095", + "broker.id": "5", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_8.log", + "config_filename": "kafka_server_8.properties" + }, + { + "entity_id": "9", + "port": "9096", + "broker.id": "6", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_9.log", + "config_filename": "kafka_server_9.properties" + }, + + { + "entity_id": "10", + "topic": "test_1", + "threads": "5", + "compression-codec": "0", + "message-size": "500", + "message": "500", + "request-num-acks": "-1", + "sync":"true", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "11", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_11.log", + "config_filename": "console_consumer_11.properties" + }, + + { + "entity_id": "12", + "log_filename": "mirror_maker_12.log", + "mirror_consumer_config_filename": "mirror_consumer_12.properties", + "mirror_producer_config_filename": "mirror_producer_12.properties" + }, + + { + "entity_id": "13", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_13.log", + "config_filename": "console_consumer_13.properties" + } + ] +} diff --git a/system_test/mirror_maker_testsuite/testcase_15002/testcase_15002_properties.json b/system_test/mirror_maker_testsuite/testcase_15002/testcase_15002_properties.json new file mode 100644 index 0000000000000..d6495e54d6845 --- /dev/null +++ b/system_test/mirror_maker_testsuite/testcase_15002/testcase_15002_properties.json @@ -0,0 +1,158 @@ +{ + "description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker", + "02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET", + "03":"Set up 2-node Zk cluster for both SOURCE & TARGET", + "04":"Produce and consume messages to a single topic - single partition.", + "05":"This test sends messages to 3 replicas", + "06":"At the end it verifies the log size and contents", + "07":"Use a consumer to verify no message loss in TARGET cluster.", + "08":"Producer dimensions : mode:sync, acks:-1, comp:0", + "09":"Log segment size : 20480" + }, + "testcase_args": { + "bounce_leader": "false", + "bounce_mirror_maker": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "clientPort": "2118", + "dataDir": "/tmp/zookeeper_1", + "log_filename": "zookeeper_1.log", + "config_filename": "zookeeper_1.properties" + }, + + { + "entity_id": "2", + "clientPort": "2128", + "dataDir": "/tmp/zookeeper_2", + "log_filename": "zookeeper_2.log", + "config_filename": "zookeeper_2.properties" + }, + { + "entity_id": "3", + "clientPort": "2138", + "dataDir": "/tmp/zookeeper_3", + "log_filename": "zookeeper_3.log", + "config_filename": "zookeeper_3.properties" + }, + + { + "entity_id": "4", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_5.log", + "config_filename": "kafka_server_5.properties" + }, + { + "entity_id": "6", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_6.log", + "config_filename": "kafka_server_6.properties" + }, + { + "entity_id": "7", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_7.log", + "config_filename": "kafka_server_7.properties" + }, + { + "entity_id": "8", + "port": "9095", + "broker.id": "5", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_8.log", + "config_filename": "kafka_server_8.properties" + }, + { + "entity_id": "9", + "port": "9096", + "broker.id": "6", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_9.log", + "config_filename": "kafka_server_9.properties" + }, + + { + "entity_id": "10", + "topic": "test_1", + "threads": "5", + "compression-codec": "0", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "sync":"true", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "11", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_11.log", + "config_filename": "console_consumer_11.properties" + }, + + { + "entity_id": "12", + "log_filename": "mirror_maker_12.log", + "mirror_consumer_config_filename": "mirror_consumer_12.properties", + "mirror_producer_config_filename": "mirror_producer_12.properties" + }, + + { + "entity_id": "13", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_13.log", + "config_filename": "console_consumer_13.properties" + } + ] +} diff --git a/system_test/migration_tool_testsuite/testcase_9005/cluster_config.json b/system_test/mirror_maker_testsuite/testcase_15003/cluster_config.json similarity index 71% rename from system_test/migration_tool_testsuite/testcase_9005/cluster_config.json rename to system_test/mirror_maker_testsuite/testcase_15003/cluster_config.json index 9fcb3b0400bd8..f6fe86787f1c3 100644 --- a/system_test/migration_tool_testsuite/testcase_9005/cluster_config.json +++ b/system_test/mirror_maker_testsuite/testcase_15003/cluster_config.json @@ -5,68 +5,67 @@ "hostname": "localhost", "role": "zookeeper", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9900" + "jmx_port": "9100" }, { "entity_id": "1", "hostname": "localhost", "role": "zookeeper", - "cluster_name":"target", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9901" + "jmx_port": "9101" }, - { "entity_id": "2", "hostname": "localhost", - "role": "broker", - "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "role": "zookeeper", + "cluster_name":"target", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9902" + "jmx_port": "9102" }, { "entity_id": "3", "hostname": "localhost", - "role": "broker", - "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "role": "zookeeper", + "cluster_name":"target", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9903" + "jmx_port": "9103" }, + { "entity_id": "4", "hostname": "localhost", "role": "broker", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9904" + "jmx_port": "9104" }, - - { "entity_id": "5", "hostname": "localhost", "role": "broker", - "cluster_name":"target", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9905" + "jmx_port": "9105" }, { "entity_id": "6", "hostname": "localhost", "role": "broker", - "cluster_name":"target", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9906" + "jmx_port": "9106" }, + { "entity_id": "7", "hostname": "localhost", @@ -74,38 +73,35 @@ "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9907" + "jmx_port": "9107" }, - - { "entity_id": "8", "hostname": "localhost", - "role": "producer_performance", - "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", - "java_home": "/export/apps/jdk/JDK-1_6_0_27", - "jmx_port": "9908" + "role": "broker", + "cluster_name":"target", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9108" }, { "entity_id": "9", "hostname": "localhost", - "role": "producer_performance", - "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", - "java_home": "/export/apps/jdk/JDK-1_6_0_27", - "jmx_port": "9909" + "role": "broker", + "cluster_name":"target", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9109" }, - { "entity_id": "10", "hostname": "localhost", - "role": "console_consumer", - "cluster_name":"target", + "role": "producer_performance", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9910" + "jmx_port": "9110" }, { "entity_id": "11", @@ -114,28 +110,26 @@ "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9911" + "jmx_port": "9111" }, - { "entity_id": "12", "hostname": "localhost", - "role": "migration_tool", + "role": "mirror_maker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9912" + "jmx_port": "9112" }, { "entity_id": "13", "hostname": "localhost", - "role": "migration_tool", + "role": "mirror_maker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9913" + "jmx_port": "9113" } - ] } diff --git a/system_test/mirror_maker_testsuite/testcase_15003/testcase_15003_properties.json b/system_test/mirror_maker_testsuite/testcase_15003/testcase_15003_properties.json new file mode 100644 index 0000000000000..842c70eaf33fd --- /dev/null +++ b/system_test/mirror_maker_testsuite/testcase_15003/testcase_15003_properties.json @@ -0,0 +1,156 @@ +{ + "description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker", + "02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET", + "03":"Set up 2-node Zk cluster for both SOURCE & TARGET", + "04":"Produce and consume messages to a single topic - single partition.", + "05":"This test sends messages to 3 replicas", + "06":"At the end it verifies the log size and contents", + "07":"Use a consumer to verify no message loss in TARGET cluster.", + "08":"Producer dimensions : mode:async, acks:-1, comp:1", + "09":"Log segment size : 20480" + }, + "testcase_args": { + "bounce_leader": "false", + "bounce_mirror_maker": "true", + "bounced_entity_downtime_sec": "30", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "clientPort": "2118", + "dataDir": "/tmp/zookeeper_1", + "log_filename": "zookeeper_1.log", + "config_filename": "zookeeper_1.properties" + }, + + { + "entity_id": "2", + "clientPort": "2128", + "dataDir": "/tmp/zookeeper_2", + "log_filename": "zookeeper_2.log", + "config_filename": "zookeeper_2.properties" + }, + { + "entity_id": "3", + "clientPort": "2138", + "dataDir": "/tmp/zookeeper_3", + "log_filename": "zookeeper_3.log", + "config_filename": "zookeeper_3.properties" + }, + + { + "entity_id": "4", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_5.log", + "config_filename": "kafka_server_5.properties" + }, + { + "entity_id": "6", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_6.log", + "config_filename": "kafka_server_6.properties" + }, + { + "entity_id": "7", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_7.log", + "config_filename": "kafka_server_7.properties" + }, + { + "entity_id": "8", + "port": "9095", + "broker.id": "5", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_8.log", + "config_filename": "kafka_server_8.properties" + }, + { + "entity_id": "9", + "port": "9096", + "broker.id": "6", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_9.log", + "config_filename": "kafka_server_9.properties" + }, + + { + "entity_id": "10", + "topic": "test_1", + "threads": "5", + "compression-codec": "2", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "sync":"false", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "11", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_11.log", + "config_filename": "console_consumer_11.properties" + }, + + { + "entity_id": "12", + "log_filename": "mirror_maker_12.log", + "mirror_consumer_config_filename": "mirror_consumer_12.properties", + "mirror_producer_config_filename": "mirror_producer_12.properties" + }, + { + "entity_id": "13", + "log_filename": "mirror_maker_13.log", + "mirror_consumer_config_filename": "mirror_consumer_13.properties", + "mirror_producer_config_filename": "mirror_producer_13.properties" + } + ] +} diff --git a/system_test/migration_tool_testsuite/testcase_9006/cluster_config.json b/system_test/mirror_maker_testsuite/testcase_15004/cluster_config.json similarity index 71% rename from system_test/migration_tool_testsuite/testcase_9006/cluster_config.json rename to system_test/mirror_maker_testsuite/testcase_15004/cluster_config.json index 9fcb3b0400bd8..f6fe86787f1c3 100644 --- a/system_test/migration_tool_testsuite/testcase_9006/cluster_config.json +++ b/system_test/mirror_maker_testsuite/testcase_15004/cluster_config.json @@ -5,68 +5,67 @@ "hostname": "localhost", "role": "zookeeper", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9900" + "jmx_port": "9100" }, { "entity_id": "1", "hostname": "localhost", "role": "zookeeper", - "cluster_name":"target", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9901" + "jmx_port": "9101" }, - { "entity_id": "2", "hostname": "localhost", - "role": "broker", - "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "role": "zookeeper", + "cluster_name":"target", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9902" + "jmx_port": "9102" }, { "entity_id": "3", "hostname": "localhost", - "role": "broker", - "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "role": "zookeeper", + "cluster_name":"target", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9903" + "jmx_port": "9103" }, + { "entity_id": "4", "hostname": "localhost", "role": "broker", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9904" + "jmx_port": "9104" }, - - { "entity_id": "5", "hostname": "localhost", "role": "broker", - "cluster_name":"target", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9905" + "jmx_port": "9105" }, { "entity_id": "6", "hostname": "localhost", "role": "broker", - "cluster_name":"target", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9906" + "jmx_port": "9106" }, + { "entity_id": "7", "hostname": "localhost", @@ -74,38 +73,35 @@ "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9907" + "jmx_port": "9107" }, - - { "entity_id": "8", "hostname": "localhost", - "role": "producer_performance", - "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", - "java_home": "/export/apps/jdk/JDK-1_6_0_27", - "jmx_port": "9908" + "role": "broker", + "cluster_name":"target", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9108" }, { "entity_id": "9", "hostname": "localhost", - "role": "producer_performance", - "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", - "java_home": "/export/apps/jdk/JDK-1_6_0_27", - "jmx_port": "9909" + "role": "broker", + "cluster_name":"target", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9109" }, - { "entity_id": "10", "hostname": "localhost", - "role": "console_consumer", - "cluster_name":"target", + "role": "producer_performance", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9910" + "jmx_port": "9110" }, { "entity_id": "11", @@ -114,28 +110,26 @@ "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9911" + "jmx_port": "9111" }, - { "entity_id": "12", "hostname": "localhost", - "role": "migration_tool", + "role": "mirror_maker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9912" + "jmx_port": "9112" }, { "entity_id": "13", "hostname": "localhost", - "role": "migration_tool", + "role": "mirror_maker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9913" + "jmx_port": "9113" } - ] } diff --git a/system_test/mirror_maker_testsuite/testcase_15004/testcase_15004_properties.json b/system_test/mirror_maker_testsuite/testcase_15004/testcase_15004_properties.json new file mode 100644 index 0000000000000..48864e61afb9c --- /dev/null +++ b/system_test/mirror_maker_testsuite/testcase_15004/testcase_15004_properties.json @@ -0,0 +1,156 @@ +{ + "description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker", + "02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET", + "03":"Set up 2-node Zk cluster for both SOURCE & TARGET", + "04":"Produce and consume messages to a single topic - single partition.", + "05":"This test sends messages to 3 replicas", + "06":"At the end it verifies the log size and contents", + "07":"Use a consumer to verify no message loss in TARGET cluster.", + "08":"Producer dimensions : mode:async, acks:1, comp:1", + "09":"Log segment size : 20480" + }, + "testcase_args": { + "bounce_leader": "false", + "bounce_mirror_maker": "true", + "bounced_entity_downtime_sec": "30", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "clientPort": "2118", + "dataDir": "/tmp/zookeeper_1", + "log_filename": "zookeeper_1.log", + "config_filename": "zookeeper_1.properties" + }, + + { + "entity_id": "2", + "clientPort": "2128", + "dataDir": "/tmp/zookeeper_2", + "log_filename": "zookeeper_2.log", + "config_filename": "zookeeper_2.properties" + }, + { + "entity_id": "3", + "clientPort": "2138", + "dataDir": "/tmp/zookeeper_3", + "log_filename": "zookeeper_3.log", + "config_filename": "zookeeper_3.properties" + }, + + { + "entity_id": "4", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_5.log", + "config_filename": "kafka_server_5.properties" + }, + { + "entity_id": "6", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_6.log", + "config_filename": "kafka_server_6.properties" + }, + { + "entity_id": "7", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_7.log", + "config_filename": "kafka_server_7.properties" + }, + { + "entity_id": "8", + "port": "9095", + "broker.id": "5", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_8.log", + "config_filename": "kafka_server_8.properties" + }, + { + "entity_id": "9", + "port": "9096", + "broker.id": "6", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_9.log", + "config_filename": "kafka_server_9.properties" + }, + + { + "entity_id": "10", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "sync":"false", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "11", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_11.log", + "config_filename": "console_consumer_11.properties" + }, + + { + "entity_id": "12", + "log_filename": "mirror_maker_12.log", + "mirror_consumer_config_filename": "mirror_consumer_12.properties", + "mirror_producer_config_filename": "mirror_producer_12.properties" + }, + { + "entity_id": "13", + "log_filename": "mirror_maker_13.log", + "mirror_consumer_config_filename": "mirror_consumer_13.properties", + "mirror_producer_config_filename": "mirror_producer_13.properties" + } + ] +} diff --git a/system_test/migration_tool_testsuite/testcase_9003/cluster_config.json b/system_test/mirror_maker_testsuite/testcase_15005/cluster_config.json similarity index 59% rename from system_test/migration_tool_testsuite/testcase_9003/cluster_config.json rename to system_test/mirror_maker_testsuite/testcase_15005/cluster_config.json index 766a001030f69..63ba37b70e476 100644 --- a/system_test/migration_tool_testsuite/testcase_9003/cluster_config.json +++ b/system_test/mirror_maker_testsuite/testcase_15005/cluster_config.json @@ -5,108 +5,149 @@ "hostname": "localhost", "role": "zookeeper", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9990" + "jmx_port": "9100" }, { "entity_id": "1", "hostname": "localhost", - "role": "broker", + "role": "zookeeper", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9991" + "jmx_port": "9101" }, + { "entity_id": "2", "hostname": "localhost", + "role": "zookeeper", + "cluster_name":"target", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9102" + }, + { + "entity_id": "3", + "hostname": "localhost", + "role": "zookeeper", + "cluster_name":"target", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9103" + }, + + { + "entity_id": "4", + "hostname": "localhost", "role": "broker", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9992" + "jmx_port": "9104" }, { - "entity_id": "3", + "entity_id": "5", "hostname": "localhost", "role": "broker", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9993" + "jmx_port": "9105" }, { - "entity_id": "4", + "entity_id": "6", + "hostname": "localhost", + "role": "broker", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9106" + }, + + { + "entity_id": "7", "hostname": "localhost", "role": "broker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9994" + "jmx_port": "9107" }, { - "entity_id": "5", + "entity_id": "8", "hostname": "localhost", "role": "broker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9995" + "jmx_port": "9108" }, { - "entity_id": "6", + "entity_id": "9", "hostname": "localhost", "role": "broker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9996" + "jmx_port": "9109" }, + { - "entity_id": "7", + "entity_id": "10", + "hostname": "localhost", + "role": "producer_performance", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9110" + }, + { + "entity_id": "11", "hostname": "localhost", "role": "producer_performance", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", - "java_home": "/export/apps/jdk/JDK-1_6_0_27", - "jmx_port": "9997" + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9111" }, { - "entity_id": "8", + "entity_id": "12", "hostname": "localhost", "role": "console_consumer", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9998" + "jmx_port": "9112" }, { - "entity_id": "9", + "entity_id": "13", "hostname": "localhost", - "role": "zookeeper", + "role": "console_consumer", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9999" + "jmx_port": "9113" }, + { - "entity_id": "10", + "entity_id": "14", "hostname": "localhost", - "role": "migration_tool", + "role": "mirror_maker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9890" + "jmx_port": "9114" }, { - "entity_id": "11", + "entity_id": "15", "hostname": "localhost", - "role": "migration_tool", + "role": "mirror_maker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9891" + "jmx_port": "9115" } ] } diff --git a/system_test/mirror_maker_testsuite/testcase_15005/testcase_15005_properties.json b/system_test/mirror_maker_testsuite/testcase_15005/testcase_15005_properties.json new file mode 100644 index 0000000000000..92b2a6b4ffeb8 --- /dev/null +++ b/system_test/mirror_maker_testsuite/testcase_15005/testcase_15005_properties.json @@ -0,0 +1,178 @@ +{ + "description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker", + "02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET", + "03":"Set up 2-node Zk cluster for both SOURCE & TARGET", + "04":"Produce and consume messages to 2 topics - 2 partitions.", + "05":"This test sends messages to 3 replicas", + "06":"At the end it verifies the log size and contents", + "07":"Use a consumer to verify no message loss in TARGET cluster.", + "08":"Producer dimensions : mode:async, acks:-1, comp:1", + "09":"Log segment size : 20480" + }, + "testcase_args": { + "bounce_leader": "false", + "bounce_mirror_maker": "true", + "bounced_entity_downtime_sec": "30", + "replica_factor": "3", + "num_partition": "2", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "clientPort": "2118", + "dataDir": "/tmp/zookeeper_1", + "log_filename": "zookeeper_1.log", + "config_filename": "zookeeper_1.properties" + }, + + { + "entity_id": "2", + "clientPort": "2128", + "dataDir": "/tmp/zookeeper_2", + "log_filename": "zookeeper_2.log", + "config_filename": "zookeeper_2.properties" + }, + { + "entity_id": "3", + "clientPort": "2138", + "dataDir": "/tmp/zookeeper_3", + "log_filename": "zookeeper_3.log", + "config_filename": "zookeeper_3.properties" + }, + + { + "entity_id": "4", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_5.log", + "config_filename": "kafka_server_5.properties" + }, + { + "entity_id": "6", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_6.log", + "config_filename": "kafka_server_6.properties" + }, + { + "entity_id": "7", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_7.log", + "config_filename": "kafka_server_7.properties" + }, + { + "entity_id": "8", + "port": "9095", + "broker.id": "5", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_8.log", + "config_filename": "kafka_server_8.properties" + }, + { + "entity_id": "9", + "port": "9096", + "broker.id": "6", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_9.log", + "config_filename": "kafka_server_9.properties" + }, + + { + "entity_id": "10", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "sync":"false", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "11", + "topic": "test_2", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "sync":"false", + "producer-num-retries":"5", + "log_filename": "producer_performance_11.log", + "config_filename": "producer_performance_11.properties" + }, + + { + "entity_id": "12", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_12.log", + "config_filename": "console_consumer_12.properties" + }, + { + "entity_id": "13", + "topic": "test_2", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_13.log", + "config_filename": "console_consumer_13.properties" + }, + + { + "entity_id": "14", + "log_filename": "mirror_maker_14.log", + "mirror_consumer_config_filename": "mirror_consumer_14.properties", + "mirror_producer_config_filename": "mirror_producer_14.properties" + }, + { + "entity_id": "15", + "log_filename": "mirror_maker_15.log", + "mirror_consumer_config_filename": "mirror_consumer_15.properties", + "mirror_producer_config_filename": "mirror_producer_15.properties" + } + ] +} diff --git a/system_test/migration_tool_testsuite/testcase_9004/cluster_config.json b/system_test/mirror_maker_testsuite/testcase_15006/cluster_config.json similarity index 59% rename from system_test/migration_tool_testsuite/testcase_9004/cluster_config.json rename to system_test/mirror_maker_testsuite/testcase_15006/cluster_config.json index 766a001030f69..63ba37b70e476 100644 --- a/system_test/migration_tool_testsuite/testcase_9004/cluster_config.json +++ b/system_test/mirror_maker_testsuite/testcase_15006/cluster_config.json @@ -5,108 +5,149 @@ "hostname": "localhost", "role": "zookeeper", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9990" + "jmx_port": "9100" }, { "entity_id": "1", "hostname": "localhost", - "role": "broker", + "role": "zookeeper", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9991" + "jmx_port": "9101" }, + { "entity_id": "2", "hostname": "localhost", + "role": "zookeeper", + "cluster_name":"target", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9102" + }, + { + "entity_id": "3", + "hostname": "localhost", + "role": "zookeeper", + "cluster_name":"target", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9103" + }, + + { + "entity_id": "4", + "hostname": "localhost", "role": "broker", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9992" + "jmx_port": "9104" }, { - "entity_id": "3", + "entity_id": "5", "hostname": "localhost", "role": "broker", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9993" + "jmx_port": "9105" }, { - "entity_id": "4", + "entity_id": "6", + "hostname": "localhost", + "role": "broker", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9106" + }, + + { + "entity_id": "7", "hostname": "localhost", "role": "broker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9994" + "jmx_port": "9107" }, { - "entity_id": "5", + "entity_id": "8", "hostname": "localhost", "role": "broker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9995" + "jmx_port": "9108" }, { - "entity_id": "6", + "entity_id": "9", "hostname": "localhost", "role": "broker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9996" + "jmx_port": "9109" }, + { - "entity_id": "7", + "entity_id": "10", + "hostname": "localhost", + "role": "producer_performance", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9110" + }, + { + "entity_id": "11", "hostname": "localhost", "role": "producer_performance", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", - "java_home": "/export/apps/jdk/JDK-1_6_0_27", - "jmx_port": "9997" + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9111" }, { - "entity_id": "8", + "entity_id": "12", "hostname": "localhost", "role": "console_consumer", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9998" + "jmx_port": "9112" }, { - "entity_id": "9", + "entity_id": "13", "hostname": "localhost", - "role": "zookeeper", + "role": "console_consumer", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9999" + "jmx_port": "9113" }, + { - "entity_id": "10", + "entity_id": "14", "hostname": "localhost", - "role": "migration_tool", + "role": "mirror_maker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9890" + "jmx_port": "9114" }, { - "entity_id": "11", + "entity_id": "15", "hostname": "localhost", - "role": "migration_tool", + "role": "mirror_maker", "cluster_name":"target", "kafka_home": "default", "java_home": "default", - "jmx_port": "9891" + "jmx_port": "9115" } ] } diff --git a/system_test/mirror_maker_testsuite/testcase_15006/testcase_15006_properties.json b/system_test/mirror_maker_testsuite/testcase_15006/testcase_15006_properties.json new file mode 100644 index 0000000000000..7d5019c6aa7fe --- /dev/null +++ b/system_test/mirror_maker_testsuite/testcase_15006/testcase_15006_properties.json @@ -0,0 +1,178 @@ +{ + "description": {"01":"Replication with Mirror Maker => Bounce Mirror Maker", + "02":"Set up 2 clusters such as : SOURCE => MirrorMaker => TARGET", + "03":"Set up 2-node Zk cluster for both SOURCE & TARGET", + "04":"Produce and consume messages to 2 topics - 2 partitions.", + "05":"This test sends messages to 3 replicas", + "06":"At the end it verifies the log size and contents", + "07":"Use a consumer to verify no message loss in TARGET cluster.", + "08":"Producer dimensions : mode:async, acks:1, comp:1", + "09":"Log segment size : 20480" + }, + "testcase_args": { + "bounce_leader": "false", + "bounce_mirror_maker": "true", + "bounced_entity_downtime_sec": "30", + "replica_factor": "3", + "num_partition": "2", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "clientPort": "2118", + "dataDir": "/tmp/zookeeper_1", + "log_filename": "zookeeper_1.log", + "config_filename": "zookeeper_1.properties" + }, + + { + "entity_id": "2", + "clientPort": "2128", + "dataDir": "/tmp/zookeeper_2", + "log_filename": "zookeeper_2.log", + "config_filename": "zookeeper_2.properties" + }, + { + "entity_id": "3", + "clientPort": "2138", + "dataDir": "/tmp/zookeeper_3", + "log_filename": "zookeeper_3.log", + "config_filename": "zookeeper_3.properties" + }, + + { + "entity_id": "4", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_5_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_5.log", + "config_filename": "kafka_server_5.properties" + }, + { + "entity_id": "6", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_6_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_6.log", + "config_filename": "kafka_server_6.properties" + }, + { + "entity_id": "7", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_7_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_7.log", + "config_filename": "kafka_server_7.properties" + }, + { + "entity_id": "8", + "port": "9095", + "broker.id": "5", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_8_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_8.log", + "config_filename": "kafka_server_8.properties" + }, + { + "entity_id": "9", + "port": "9096", + "broker.id": "6", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_9_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_9.log", + "config_filename": "kafka_server_9.properties" + }, + + { + "entity_id": "10", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "sync":"false", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "11", + "topic": "test_2", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "sync":"false", + "producer-num-retries":"5", + "log_filename": "producer_performance_11.log", + "config_filename": "producer_performance_11.properties" + }, + + { + "entity_id": "12", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_12.log", + "config_filename": "console_consumer_12.properties" + }, + { + "entity_id": "13", + "topic": "test_2", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_13.log", + "config_filename": "console_consumer_13.properties" + }, + + { + "entity_id": "14", + "log_filename": "mirror_maker_14.log", + "mirror_consumer_config_filename": "mirror_consumer_14.properties", + "mirror_producer_config_filename": "mirror_producer_14.properties" + }, + { + "entity_id": "15", + "log_filename": "mirror_maker_15.log", + "mirror_consumer_config_filename": "mirror_consumer_15.properties", + "mirror_producer_config_filename": "mirror_producer_15.properties" + } + ] +} diff --git a/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json b/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json index 287cab9a9b6b9..0891836421909 100644 --- a/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:sync, acks:-1, comp:0", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -54,7 +54,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -65,7 +65,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -76,7 +76,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -87,7 +87,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -98,7 +98,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -109,7 +109,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -119,6 +119,7 @@ { "entity_id": "10", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -141,6 +142,7 @@ { "entity_id": "12", + "new-producer":"true", "log_filename": "mirror_maker_12.log", "mirror_consumer_config_filename": "mirror_consumer_12.properties", "mirror_producer_config_filename": "mirror_producer_12.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json b/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json index 5457eb1d26526..56e481255cbbb 100644 --- a/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:sync, acks:-1, comp:0", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -54,7 +54,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -65,7 +65,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -76,7 +76,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -87,7 +87,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -98,7 +98,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -109,7 +109,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -119,6 +119,7 @@ { "entity_id": "10", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -141,6 +142,7 @@ { "entity_id": "12", + "new-producer":"true", "log_filename": "mirror_maker_12.log", "mirror_consumer_config_filename": "mirror_consumer_12.properties", "mirror_producer_config_filename": "mirror_producer_12.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json b/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json index 98fefee5838a2..8f8c47af4e797 100644 --- a/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:-1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -120,6 +120,7 @@ { "entity_id": "10", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "2", @@ -142,12 +143,14 @@ { "entity_id": "12", + "new-producer":"true", "log_filename": "mirror_maker_12.log", "mirror_consumer_config_filename": "mirror_consumer_12.properties", "mirror_producer_config_filename": "mirror_producer_12.properties" }, { "entity_id": "13", + "new-producer":"true", "log_filename": "mirror_maker_13.log", "mirror_consumer_config_filename": "mirror_consumer_13.properties", "mirror_producer_config_filename": "mirror_producer_13.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json b/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json index 6067b1263d0a5..baa639b688888 100644 --- a/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -120,6 +120,7 @@ { "entity_id": "10", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -142,12 +143,14 @@ { "entity_id": "12", + "new-producer":"true", "log_filename": "mirror_maker_12.log", "mirror_consumer_config_filename": "mirror_consumer_12.properties", "mirror_producer_config_filename": "mirror_producer_12.properties" }, { "entity_id": "13", + "new-producer":"true", "log_filename": "mirror_maker_13.log", "mirror_consumer_config_filename": "mirror_consumer_13.properties", "mirror_producer_config_filename": "mirror_producer_13.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json b/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json index 58483ad375cd1..8c383756d88ca 100644 --- a/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:-1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -120,6 +120,7 @@ { "entity_id": "10", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -133,6 +134,7 @@ }, { "entity_id": "11", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", @@ -164,12 +166,14 @@ { "entity_id": "14", + "new-producer":"true", "log_filename": "mirror_maker_14.log", "mirror_consumer_config_filename": "mirror_consumer_14.properties", "mirror_producer_config_filename": "mirror_producer_14.properties" }, { "entity_id": "15", + "new-producer":"true", "log_filename": "mirror_maker_15.log", "mirror_consumer_config_filename": "mirror_consumer_15.properties", "mirror_producer_config_filename": "mirror_producer_15.properties" diff --git a/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json b/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json index 1d9190ce7c1b5..fb275330bd188 100644 --- a/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -120,6 +120,7 @@ { "entity_id": "10", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -133,6 +134,7 @@ }, { "entity_id": "11", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", @@ -164,12 +166,14 @@ { "entity_id": "14", + "new-producer":"true", "log_filename": "mirror_maker_14.log", "mirror_consumer_config_filename": "mirror_consumer_14.properties", "mirror_producer_config_filename": "mirror_producer_14.properties" }, { "entity_id": "15", + "new-producer":"true", "log_filename": "mirror_maker_15.log", "mirror_consumer_config_filename": "mirror_consumer_15.properties", "mirror_producer_config_filename": "mirror_producer_15.properties" diff --git a/system_test/migration_tool_testsuite/cluster_config.json b/system_test/offset_management_testsuite/cluster_config.json similarity index 64% rename from system_test/migration_tool_testsuite/cluster_config.json rename to system_test/offset_management_testsuite/cluster_config.json index 8353e56e55aab..dcca2007de4bd 100644 --- a/system_test/migration_tool_testsuite/cluster_config.json +++ b/system_test/offset_management_testsuite/cluster_config.json @@ -5,99 +5,99 @@ "hostname": "localhost", "role": "zookeeper", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9990" + "jmx_port": "9100" }, { "entity_id": "1", "hostname": "localhost", "role": "broker", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9991" + "jmx_port": "9101" }, { "entity_id": "2", "hostname": "localhost", "role": "broker", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9992" + "jmx_port": "9102" }, { "entity_id": "3", "hostname": "localhost", "role": "broker", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", + "kafka_home": "default", "java_home": "default", - "jmx_port": "9993" + "jmx_port": "9103" }, { "entity_id": "4", "hostname": "localhost", "role": "broker", - "cluster_name":"target", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9994" + "jmx_port": "9104" }, { "entity_id": "5", "hostname": "localhost", - "role": "broker", - "cluster_name":"target", + "role": "producer_performance", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9995" + "jmx_port": "9105" }, { "entity_id": "6", "hostname": "localhost", - "role": "broker", - "cluster_name":"target", + "role": "console_consumer", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9996" + "jmx_port": "9106" }, { "entity_id": "7", "hostname": "localhost", - "role": "producer_performance", + "role": "console_consumer", "cluster_name":"source", - "kafka_home": "system_test/migration_tool_testsuite/0.7", - "java_home": "/export/apps/jdk/JDK-1_6_0_27", - "jmx_port": "9997" + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9107" }, { "entity_id": "8", "hostname": "localhost", "role": "console_consumer", - "cluster_name":"target", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9998" + "jmx_port": "9108" }, { "entity_id": "9", "hostname": "localhost", - "role": "zookeeper", - "cluster_name":"target", + "role": "console_consumer", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9999" + "jmx_port": "9109" }, { "entity_id": "10", "hostname": "localhost", - "role": "migration_tool", - "cluster_name":"target", + "role": "console_consumer", + "cluster_name":"source", "kafka_home": "default", "java_home": "default", - "jmx_port": "9890" + "jmx_port": "9110" } ] } diff --git a/system_test/offset_management_testsuite/config/console_consumer.properties b/system_test/offset_management_testsuite/config/console_consumer.properties new file mode 100644 index 0000000000000..a2ab8b9c155e2 --- /dev/null +++ b/system_test/offset_management_testsuite/config/console_consumer.properties @@ -0,0 +1,2 @@ +auto.offset.reset=smallest +auto.commit.interval.ms=1000 diff --git a/system_test/offset_management_testsuite/config/producer_performance.properties b/system_test/offset_management_testsuite/config/producer_performance.properties new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/system_test/migration_tool_testsuite/config/server.properties b/system_test/offset_management_testsuite/config/server.properties similarity index 88% rename from system_test/migration_tool_testsuite/config/server.properties rename to system_test/offset_management_testsuite/config/server.properties index 54144a28a0800..b6de5289f484a 100644 --- a/system_test/migration_tool_testsuite/config/server.properties +++ b/system_test/offset_management_testsuite/config/server.properties @@ -95,11 +95,11 @@ log.retention.hours=168 # A size-based retention policy for logs. Segments are pruned from the log as long as the remaining # segments don't drop below log.retention.bytes. #log.retention.bytes=1073741824 +log.retention.bytes=-1 # The maximum size of a log segment file. When this size is reached a new log segment will be created. -#log.segment.bytes=536870912 -#log.segment.bytes=102400 -log.segment.bytes=128 +#log.segment.size=536870912 +log.segment.bytes=102400 # The interval at which log segments are checked to see if they can be deleted according # to the retention policies @@ -115,11 +115,29 @@ enable.zookeeper=true # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". # You can also append an optional chroot string to the urls to specify the # root directory for all kafka znodes. -zk.connect=localhost:2181 zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 zookeeper.connection.timeout.ms=1000000 monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +default.replication.factor=1 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + diff --git a/system_test/migration_tool_testsuite/config/zookeeper.properties b/system_test/offset_management_testsuite/config/zookeeper.properties similarity index 96% rename from system_test/migration_tool_testsuite/config/zookeeper.properties rename to system_test/offset_management_testsuite/config/zookeeper.properties index 74cbf90428f81..5474a72be1a2c 100644 --- a/system_test/migration_tool_testsuite/config/zookeeper.properties +++ b/system_test/offset_management_testsuite/config/zookeeper.properties @@ -18,3 +18,6 @@ dataDir=/tmp/zookeeper clientPort=2181 # disable the per-ip limit on the number of connections since this is a non-production config maxClientCnxns=0 +syncLimit=5 +initLimit=10 +tickTime=2000 diff --git a/system_test/migration_tool_testsuite/migration_tool_test.py b/system_test/offset_management_testsuite/offset_management_test.py similarity index 78% rename from system_test/migration_tool_testsuite/migration_tool_test.py rename to system_test/offset_management_testsuite/offset_management_test.py index ce6f4f684beaa..aa389105aa427 100644 --- a/system_test/migration_tool_testsuite/migration_tool_test.py +++ b/system_test/offset_management_testsuite/offset_management_test.py @@ -17,14 +17,11 @@ #!/usr/bin/env python # =================================== -# migration_tool_test.py +# offset_management_test.py # =================================== -import inspect -import logging import os import signal -import subprocess import sys import time import traceback @@ -41,7 +38,7 @@ import kafka_system_test_utils import metrics -class MigrationToolTest(ReplicationUtils, SetupUtils): +class OffsetManagementTest(ReplicationUtils, SetupUtils): testModuleAbsPathName = os.path.realpath(__file__) testSuiteAbsPathName = os.path.abspath(os.path.dirname(testModuleAbsPathName)) @@ -54,7 +51,7 @@ def __init__(self, systemTestEnv): # "clusterEntityConfigDictList" self.systemTestEnv = systemTestEnv - super(MigrationToolTest, self).__init__(self) + super(OffsetManagementTest, self).__init__(self) # dict to pass user-defined attributes to logger argument: "extra" d = {'name_of_class': self.__class__.__name__} @@ -118,32 +115,31 @@ def runTest(self): # ============================================================================== # # initialize self.testcaseEnv with user-defined environment variables (product specific) - self.testcaseEnv.userDefinedEnvVarDict["zkConnectStr"] = "" self.testcaseEnv.userDefinedEnvVarDict["stopBackgroundProducer"] = False self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"] = False # initialize signal handler signal.signal(signal.SIGINT, self.signal_handler) - + # TestcaseEnv.testcaseConfigsList initialized by reading testcase properties file: # system_test/_testsuite/testcase_/testcase__properties.json self.testcaseEnv.testcaseConfigsList = system_test_utils.get_json_list_data( self.testcaseEnv.testcasePropJsonPathName) - + # clean up data directories specified in zookeeper.properties and kafka_server_.properties kafka_system_test_utils.cleanup_data_at_remote_hosts(self.systemTestEnv, self.testcaseEnv) # create "LOCAL" log directories for metrics, dashboards for each entity under this testcase # for collecting logs from remote machines kafka_system_test_utils.generate_testcase_log_dirs(self.systemTestEnv, self.testcaseEnv) - + # TestcaseEnv - initialize producer & consumer config / log file pathnames kafka_system_test_utils.init_entity_props(self.systemTestEnv, self.testcaseEnv) # generate remote hosts log/config dirs if not exist kafka_system_test_utils.generate_testcase_log_dirs_in_remote_hosts(self.systemTestEnv, self.testcaseEnv) - # generate properties files for zookeeper, kafka, producer, consumer and mirror-maker: + # generate properties files for zookeeper, kafka, producer, and consumer: # 1. copy system_test/_testsuite/config/*.properties to # system_test/_testsuite/testcase_/config/ # 2. update all properties files in system_test/_testsuite/testcase_/config @@ -151,7 +147,7 @@ def runTest(self): # system_test/_testsuite/testcase_/testcase__properties.json kafka_system_test_utils.generate_overriden_props_files(self.testSuiteAbsPathName, self.testcaseEnv, self.systemTestEnv) - + # ============================================= # preparing all entities to start the test # ============================================= @@ -159,19 +155,14 @@ def runTest(self): kafka_system_test_utils.start_zookeepers(self.systemTestEnv, self.testcaseEnv) self.anonLogger.info("sleeping for 2s") time.sleep(2) - + self.log_message("starting brokers") kafka_system_test_utils.start_brokers(self.systemTestEnv, self.testcaseEnv) self.anonLogger.info("sleeping for 5s") time.sleep(5) - self.log_message("starting migration tool") - kafka_system_test_utils.start_migration_tool(self.systemTestEnv, self.testcaseEnv) - self.anonLogger.info("sleeping for 5s") - time.sleep(5) - - self.log_message("creating topics") - kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv) + self.log_message("creating offset topic") + kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 3, 2) self.anonLogger.info("sleeping for 5s") time.sleep(5) @@ -179,49 +170,65 @@ def runTest(self): # starting producer # ============================================= self.log_message("starting producer in the background") - kafka_system_test_utils.start_producer_performance(self.systemTestEnv, self.testcaseEnv, True) + kafka_system_test_utils.start_producer_performance(self.systemTestEnv, self.testcaseEnv, False) msgProducingFreeTimeSec = self.testcaseEnv.testcaseArgumentsDict["message_producing_free_time_sec"] self.anonLogger.info("sleeping for " + msgProducingFreeTimeSec + " sec to produce some messages") time.sleep(int(msgProducingFreeTimeSec)) + kafka_system_test_utils.start_console_consumers(self.systemTestEnv, self.testcaseEnv) + + kafka_system_test_utils.get_leader_for(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 0) + # ============================================= - # A while-loop to bounce leader as specified + # A while-loop to bounce consumers as specified # by "num_iterations" in testcase_n_properties.json # ============================================= i = 1 numIterations = int(self.testcaseEnv.testcaseArgumentsDict["num_iteration"]) - bouncedEntityDownTimeSec = 1 + bouncedEntityDownTimeSec = 10 try: bouncedEntityDownTimeSec = int(self.testcaseEnv.testcaseArgumentsDict["bounced_entity_downtime_sec"]) except: pass + # group1 -> offsets partition 0 // has one consumer; eid: 6 + # group2 -> offsets partition 1 // has four consumers; eid: 7, 8, 9, 10 + + offsets_0_leader_entity = kafka_system_test_utils.get_leader_for(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 0) + offsets_1_leader_entity = kafka_system_test_utils.get_leader_for(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 1) + while i <= numIterations: self.log_message("Iteration " + str(i) + " of " + str(numIterations)) + kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, offsets_0_leader_entity, self.testcaseEnv.entityBrokerParentPidDict[offsets_0_leader_entity]) + kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, offsets_1_leader_entity, self.testcaseEnv.entityBrokerParentPidDict[offsets_1_leader_entity]) # ============================================= - # Bounce Migration Tool + # Bounce consumers if specified in testcase config # ============================================= - bounceMigrationTool = self.testcaseEnv.testcaseArgumentsDict["bounce_migration_tool"] - self.log_message("bounce_migration_tool flag : " + bounceMigrationTool) - if (bounceMigrationTool.lower() == "true"): + bounceConsumers = self.testcaseEnv.testcaseArgumentsDict["bounce_consumers"] + self.log_message("bounce_consumers flag : " + bounceConsumers) + if (bounceConsumers.lower() == "true"): - clusterConfigList = self.systemTestEnv.clusterEntityConfigDictList - migrationToolEntityIdList = system_test_utils.get_data_from_list_of_dicts( - clusterConfigList, "role", "migration_tool", "entity_id") + clusterConfigList = self.systemTestEnv.clusterEntityConfigDictList + consumerEntityIdList = system_test_utils.get_data_from_list_of_dicts( clusterConfigList, "role", "console_consumer", "entity_id") - stoppedMigrationToolEntityId = migrationToolEntityIdList[0] - migrationToolPPid = self.testcaseEnv.entityMigrationToolParentPidDict[stoppedMigrationToolEntityId] + for stoppedConsumerEntityId in consumerEntityIdList: + consumerPPID = self.testcaseEnv.entityConsoleConsumerParentPidDict[stoppedConsumerEntityId] + self.log_message("stopping consumer: " + consumerPPID) + kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, stoppedConsumerEntityId, consumerPPID) - self.log_message("stopping migration tool : " + migrationToolPPid) - kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, stoppedMigrationToolEntityId, migrationToolPPid) self.anonLogger.info("sleeping for " + str(bouncedEntityDownTimeSec) + " sec") time.sleep(bouncedEntityDownTimeSec) + # leaders would have changed during the above bounce. + self.log_message("starting the previously terminated consumers.") + for stoppedConsumerEntityId in consumerEntityIdList: + # starting previously terminated consumer + kafka_system_test_utils.start_console_consumers(self.systemTestEnv, self.testcaseEnv, stoppedConsumerEntityId) - # starting previously terminated broker - self.log_message("starting the previously terminated migration tool") - kafka_system_test_utils.start_migration_tool(self.systemTestEnv, self.testcaseEnv, stoppedMigrationToolEntityId) + self.log_message("starting the previously terminated brokers") + kafka_system_test_utils.start_entity_in_background(self.systemTestEnv, self.testcaseEnv, offsets_0_leader_entity) + kafka_system_test_utils.start_entity_in_background(self.systemTestEnv, self.testcaseEnv, offsets_1_leader_entity) self.anonLogger.info("sleeping for 15s") time.sleep(15) @@ -253,17 +260,9 @@ def runTest(self): self.testcaseEnv.lock.release() time.sleep(2) - #print "\n\n#### sleeping for 30 min ...\n\n" - #time.sleep(1800) - - # ============================================= - # starting consumer - # ============================================= - self.log_message("starting consumer in the background") - kafka_system_test_utils.start_console_consumer(self.systemTestEnv, self.testcaseEnv) - self.anonLogger.info("sleeping for 20s") - time.sleep(20) - + self.anonLogger.info("sleeping for 15s") + time.sleep(15) + # ============================================= # this testcase is completed - stop all entities # ============================================= @@ -281,29 +280,17 @@ def runTest(self): # collect logs from remote hosts # ============================================= kafka_system_test_utils.collect_logs_from_remote_hosts(self.systemTestEnv, self.testcaseEnv) - + # ============================================= # validate the data matched and checksum # ============================================= self.log_message("validating data matched") - kafka_system_test_utils.validate_data_matched(self.systemTestEnv, self.testcaseEnv, replicationUtils) - kafka_system_test_utils.validate_broker_log_segment_checksum(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.validate_data_matched_in_multi_topics_from_single_consumer_producer(self.systemTestEnv, self.testcaseEnv, replicationUtils) - # ============================================= - # draw graphs - # ============================================= - metrics.draw_all_graphs(self.systemTestEnv.METRICS_PATHNAME, - self.testcaseEnv, - self.systemTestEnv.clusterEntityConfigDictList) - - # build dashboard, one for each role - metrics.build_all_dashboards(self.systemTestEnv.METRICS_PATHNAME, - self.testcaseEnv.testCaseDashboardsDir, - self.systemTestEnv.clusterEntityConfigDictList) - except Exception as e: self.log_message("Exception while running test {0}".format(e)) traceback.print_exc() + self.testcaseEnv.validationStatusDict["Test completed"] = "FAILED" finally: if not skipThisTestCase and not self.systemTestEnv.printTestDescriptionsOnly: diff --git a/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json new file mode 100644 index 0000000000000..1f0b7180a9d0a --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json @@ -0,0 +1,95 @@ +{ + "description": {"01":"To Test : 'Basic offset management test.'", + "02":"Set up a Zk and Kafka cluster.", + "03":"Produce messages to a multiple topics - various partition counts.", + "04":"Start multiple consumer groups to read various subsets of above topics.", + "05":"Bounce consumers.", + "06":"Verify that there are no duplicate messages or lost messages on any consumer group.", + "07":"Producer dimensions : mode:sync, acks:-1, comp:0" + }, + "testcase_args": { + "bounce_leaders": "false", + "bounce_consumers": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50", + "num_topics_for_auto_generated_string":"1" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_1.log", + "config_filename": "kafka_server_1.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_2.log", + "config_filename": "kafka_server_2.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_3.log", + "config_filename": "kafka_server_3.properties" + }, + { + "entity_id": "4", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "topic": "test", + "threads": "3", + "compression-codec": "0", + "message-size": "500", + "message": "1000", + "request-num-acks": "-1", + "sync":"true", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "6", + "topic": "test_0001", + "group.id": "group1", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_6.properties" + } + ] +} diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties new file mode 100644 index 0000000000000..9efbd9da7c4b2 --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties @@ -0,0 +1,147 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=1 + +# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned +# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost +# may not be what you want. +#host.name= + + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9091 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# The directory under which to store log files +log.dir=/tmp/kafka_server_1_logs + +# 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 +#topic.partition.count.map=topic1:3, topic2:4 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash. +# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency). +# 3. Throughput: The flush is generally the most expensive operation. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# 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 + +# The interval (in ms) at which logs are checked to see if they need to be flushed to disk. +log.flush.scheduler.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 +log.retention.bytes=-1 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +#log.segment.size=536870912 +log.segment.bytes=10240 + +# 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 + +############################# Zookeeper ############################# + +# Enable connecting to zookeeper +enable.zookeeper=true + +# Zk connection string (see zk docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2108 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + +monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +default.replication.factor=3 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + +kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-1/metrics +kafka.csv.metrics.reporter.enabled=true +kafka.metrics.polling.interval.secs=5 +kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties new file mode 100644 index 0000000000000..d4bf702554aa8 --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties @@ -0,0 +1,147 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=2 + +# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned +# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost +# may not be what you want. +#host.name= + + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9092 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# The directory under which to store log files +log.dir=/tmp/kafka_server_2_logs + +# 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 +#topic.partition.count.map=topic1:3, topic2:4 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash. +# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency). +# 3. Throughput: The flush is generally the most expensive operation. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# 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 + +# The interval (in ms) at which logs are checked to see if they need to be flushed to disk. +log.flush.scheduler.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 +log.retention.bytes=-1 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +#log.segment.size=536870912 +log.segment.bytes=10240 + +# 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 + +############################# Zookeeper ############################# + +# Enable connecting to zookeeper +enable.zookeeper=true + +# Zk connection string (see zk docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2108 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + +monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +default.replication.factor=3 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + +kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-2/metrics +kafka.csv.metrics.reporter.enabled=true +kafka.metrics.polling.interval.secs=5 +kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties new file mode 100644 index 0000000000000..e6e06bef378ab --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties @@ -0,0 +1,147 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=3 + +# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned +# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost +# may not be what you want. +#host.name= + + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9093 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# The directory under which to store log files +log.dir=/tmp/kafka_server_3_logs + +# 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 +#topic.partition.count.map=topic1:3, topic2:4 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash. +# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency). +# 3. Throughput: The flush is generally the most expensive operation. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# 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 + +# The interval (in ms) at which logs are checked to see if they need to be flushed to disk. +log.flush.scheduler.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 +log.retention.bytes=-1 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +#log.segment.size=536870912 +log.segment.bytes=10240 + +# 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 + +############################# Zookeeper ############################# + +# Enable connecting to zookeeper +enable.zookeeper=true + +# Zk connection string (see zk docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2108 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + +monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +default.replication.factor=3 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + +kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-3/metrics +kafka.csv.metrics.reporter.enabled=true +kafka.metrics.polling.interval.secs=5 +kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties new file mode 100644 index 0000000000000..2cb03e4fbec45 --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties @@ -0,0 +1,147 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=4 + +# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned +# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost +# may not be what you want. +#host.name= + + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9094 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# The directory under which to store log files +log.dir=/tmp/kafka_server_4_logs + +# 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 +#topic.partition.count.map=topic1:3, topic2:4 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash. +# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency). +# 3. Throughput: The flush is generally the most expensive operation. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# 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 + +# The interval (in ms) at which logs are checked to see if they need to be flushed to disk. +log.flush.scheduler.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 +log.retention.bytes=-1 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +#log.segment.size=536870912 +log.segment.bytes=10240 + +# 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 + +############################# Zookeeper ############################# + +# Enable connecting to zookeeper +enable.zookeeper=true + +# Zk connection string (see zk docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2108 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + +monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +default.replication.factor=3 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + +kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-4/metrics +kafka.csv.metrics.reporter.enabled=true +kafka.metrics.polling.interval.secs=5 +kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/system_test/mirror_maker/config/zookeeper_source_2.properties b/system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties similarity index 79% rename from system_test/mirror_maker/config/zookeeper_source_2.properties rename to system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties index d534d180daf88..97c07b9cd470d 100644 --- a/system_test/mirror_maker/config/zookeeper_source_2.properties +++ b/system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties @@ -13,6 +13,12 @@ # See the License for the specific language governing permissions and # limitations under the License. # the directory where the snapshot is stored. -dataDir=/tmp/zookeeper_source-2 +dataDir=/tmp/zookeeper_0 # the port at which the clients will connect -clientPort=2182 +clientPort=2108 +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 +syncLimit=5 +initLimit=10 +tickTime=2000 +server.1=localhost:2107:2109 diff --git a/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json b/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json new file mode 100644 index 0000000000000..c5866a2ecc42e --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json @@ -0,0 +1,127 @@ +{ + "description": {"01":"To Test : 'Basic offset management test.'", + "02":"Set up a Zk and Kafka cluster.", + "03":"Produce messages to a multiple topics - various partition counts.", + "04":"Start multiple consumer groups to read various subsets of above topics.", + "05":"Bounce consumers.", + "06":"Verify that there are no duplicate messages or lost messages on any consumer group.", + "07":"Producer dimensions : mode:sync, acks:-1, comp:0" + }, + "testcase_args": { + "bounce_leaders": "false", + "bounce_consumers": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50", + "num_topics_for_auto_generated_string":"3" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_1.log", + "config_filename": "kafka_server_1.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_2.log", + "config_filename": "kafka_server_2.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_3.log", + "config_filename": "kafka_server_3.properties" + }, + { + "entity_id": "4", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "topic": "test", + "threads": "5", + "compression-codec": "0", + "message-size": "500", + "message": "1000", + "request-num-acks": "-1", + "sync":"true", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "6", + "topic": "test_0001", + "group.id": "group1", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_6.properties" + }, + { + "entity_id": "7", + "topic": "test_0002", + "group.id": "group2", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_7.properties" + }, + { + "entity_id": "8", + "topic": "test_0002", + "group.id": "group2", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_8.properties" + }, + { + "entity_id": "9", + "topic": "test_0002", + "group.id": "group2", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_9.properties" + }, + { + "entity_id": "10", + "topic": "test_0003", + "group.id": "group2", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_10.properties" + } + ] +} diff --git a/system_test/producer_perf/bin/run-compression-test.sh b/system_test/producer_perf/bin/run-compression-test.sh index ea20f0dbd803e..5297d1f93e3fa 100755 --- a/system_test/producer_perf/bin/run-compression-test.sh +++ b/system_test/producer_perf/bin/run-compression-test.sh @@ -5,9 +5,9 @@ # 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. @@ -28,9 +28,9 @@ $base_dir/../../bin/kafka-server-start.sh $base_dir/config/server.properties 2>& sleep 4 echo "start producing $num_messages messages ..." -$base_dir/../../bin/kafka-run-class.sh kafka.perf.ProducerPerformance --brokerinfo broker.list=0:localhost:9092 --topics test01 --messages $num_messages --message-size $message_size --batch-size 200 --threads 1 --reporting-interval 100000 num_messages --async --compression-codec 1 +$base_dir/../../bin/kafka-run-class.sh kafka.tools.ProducerPerformance --brokerinfo broker.list=0:localhost:9092 --topics test01 --messages $num_messages --message-size $message_size --batch-size 200 --threads 1 --reporting-interval 100000 num_messages --async --compression-codec 1 -echo "wait for data to be persisted" +echo "wait for data to be persisted" cur_offset="-1" quit=0 while [ $quit -eq 0 ] @@ -59,4 +59,3 @@ fi ps ax | grep -i 'kafka.kafka' | grep -v grep | awk '{print $1}' | xargs kill -15 > /dev/null sleep 2 ps ax | grep -i 'QuorumPeerMain' | grep -v grep | awk '{print $1}' | xargs kill -15 > /dev/null - diff --git a/system_test/producer_perf/bin/run-test.sh b/system_test/producer_perf/bin/run-test.sh index bb60817edda2b..9a3b8858a9b27 100755 --- a/system_test/producer_perf/bin/run-test.sh +++ b/system_test/producer_perf/bin/run-test.sh @@ -5,9 +5,9 @@ # 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. @@ -28,9 +28,9 @@ $base_dir/../../bin/kafka-server-start.sh $base_dir/config/server.properties 2>& sleep 4 echo "start producing $num_messages messages ..." -$base_dir/../../bin/kafka-run-class.sh kafka.perf.ProducerPerformance --brokerinfo broker.list=0:localhost:9092 --topics test01 --messages $num_messages --message-size $message_size --batch-size 200 --threads 1 --reporting-interval 100000 num_messages --async +$base_dir/../../bin/kafka-run-class.sh kafka.tools.ProducerPerformance --brokerinfo broker.list=0:localhost:9092 --topics test01 --messages $num_messages --message-size $message_size --batch-size 200 --threads 1 --reporting-interval 100000 num_messages --async -echo "wait for data to be persisted" +echo "wait for data to be persisted" cur_offset="-1" quit=0 while [ $quit -eq 0 ] @@ -59,4 +59,3 @@ fi ps ax | grep -i 'kafka.kafka' | grep -v grep | awk '{print $1}' | xargs kill -15 > /dev/null sleep 2 ps ax | grep -i 'QuorumPeerMain' | grep -v grep | awk '{print $1}' | xargs kill -15 > /dev/null - diff --git a/system_test/producer_perf/config/server.properties b/system_test/producer_perf/config/server.properties index 9f8a633ab2f92..83a1e06794c7f 100644 --- a/system_test/producer_perf/config/server.properties +++ b/system_test/producer_perf/config/server.properties @@ -60,10 +60,10 @@ enable.zookeeper=true # zk connection string # comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 +zookeeper.connect=localhost:2181 # timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 +zookeeper.connection.timeout.ms=1000000 # time based topic flush intervals in ms #log.flush.intervals.ms.per.topic=topic:1000 diff --git a/system_test/replication_testsuite/config/server.properties b/system_test/replication_testsuite/config/server.properties index c6284122e3dfa..d1dff6865b3ee 100644 --- a/system_test/replication_testsuite/config/server.properties +++ b/system_test/replication_testsuite/config/server.properties @@ -128,7 +128,6 @@ log.index.size.max.bytes=10485760 log.index.interval.bytes=4096 auto.create.topics.enable=true controller.socket.timeout.ms=30000 -controller.message.queue.size=10 default.replication.factor=1 replica.lag.time.max.ms=10000 replica.lag.max.messages=4000 @@ -136,5 +135,5 @@ replica.socket.timeout.ms=30000 replica.socket.receive.buffer.bytes=65536 replica.fetch.max.bytes=1048576 replica.fetch.wait.max.ms=500 -replica.fetch.min.bytes=4096 +replica.fetch.min.bytes=1 num.replica.fetchers=1 diff --git a/system_test/replication_testsuite/replica_basic_test.py b/system_test/replication_testsuite/replica_basic_test.py index e20130b85aecd..16a24a407051a 100644 --- a/system_test/replication_testsuite/replica_basic_test.py +++ b/system_test/replication_testsuite/replica_basic_test.py @@ -188,7 +188,7 @@ def runTest(self): if autoCreateTopic.lower() == "false": self.log_message("creating topics") - kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.create_topic_for_producer_performance(self.systemTestEnv, self.testcaseEnv) self.anonLogger.info("sleeping for 5s") time.sleep(5) @@ -363,6 +363,7 @@ def runTest(self): str(self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]) + "]", extra=self.d) if self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]: time.sleep(1) + self.testcaseEnv.lock.release() self.logger.info("all producer threads completed", extra=self.d) break time.sleep(1) @@ -450,6 +451,8 @@ def runTest(self): except Exception as e: self.log_message("Exception while running test {0}".format(e)) traceback.print_exc() + self.testcaseEnv.validationStatusDict["Test completed"] = "FAILED" + finally: if not skipThisTestCase and not self.systemTestEnv.printTestDescriptionsOnly: diff --git a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json index 9bc164bee1dd0..7a32e8d5032fd 100644 --- a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json +++ b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:0", - "07":"Log segment size : 10240" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json index c90d7536a7564..29294149e3251 100644 --- a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json +++ b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json index b62b8aa50a38c..d9818e19f9ec8 100644 --- a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json +++ b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json index b91cef88533cb..fe42626ef8226 100644 --- a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json +++ b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:-1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json index 4b3f76a21642d..37d180a794a66 100644 --- a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json +++ b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json index b9b3485c46b61..dae8f763db83d 100644 --- a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json +++ b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json index 5c4351fad98ef..b6f513ff510b8 100644 --- a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json +++ b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:-1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json index 79cfed8bc63e4..4954752089b12 100644 --- a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json +++ b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json index a52b709579f58..0476b12120952 100644 --- a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json +++ b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json b/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json index 8d4b5fea2d1f9..e25ddb917dd1a 100644 --- a/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json +++ b/system_test/replication_testsuite/testcase_0010/testcase_0010_properties.json @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0011/testcase_0011_properties.json b/system_test/replication_testsuite/testcase_0011/testcase_0011_properties.json index b03f9cf059899..ac17570034037 100644 --- a/system_test/replication_testsuite/testcase_0011/testcase_0011_properties.json +++ b/system_test/replication_testsuite/testcase_0011/testcase_0011_properties.json @@ -61,6 +61,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json b/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json index 40c2f8df13051..f35a439782057 100644 --- a/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json +++ b/system_test/replication_testsuite/testcase_0021/testcase_0021_properties.json @@ -59,6 +59,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -72,6 +73,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json b/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json index 7a3046035329f..5a168f3f5cfab 100644 --- a/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json +++ b/system_test/replication_testsuite/testcase_0022/testcase_0022_properties.json @@ -59,6 +59,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -72,6 +73,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json b/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json index d921f01f332a4..09d81a6584a0c 100644 --- a/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json +++ b/system_test/replication_testsuite/testcase_0023/testcase_0023_properties.json @@ -59,6 +59,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -72,6 +73,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0024/testcase_0024_properties.json b/system_test/replication_testsuite/testcase_0024/testcase_0024_properties.json index 839eb1476981e..5661b88b14fdb 100644 --- a/system_test/replication_testsuite/testcase_0024/testcase_0024_properties.json +++ b/system_test/replication_testsuite/testcase_0024/testcase_0024_properties.json @@ -62,6 +62,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1,test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json b/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json index 85e4b61751b12..b9517b4c655ff 100644 --- a/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json +++ b/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:0", - "10":"Log segment size : 10240" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json b/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json index 0d2f59fbcaa30..3eb39a26f60a4 100644 --- a/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json +++ b/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json b/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json index 34acfa9d5c684..6bfc7570c88b1 100644 --- a/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json +++ b/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json b/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json index 4145345fc433a..1cfe71ca3acd1 100644 --- a/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json +++ b/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json b/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json index 2eecc768e7e97..13f3ac0eec45d 100644 --- a/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json +++ b/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json b/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json index 744174eab1c82..ccd4774f7fca6 100644 --- a/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json +++ b/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json b/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json index e881b138c1fc0..b1da75a0b2618 100644 --- a/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json +++ b/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json b/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json index 7b48fdb441ad5..359abe7ab4d36 100644 --- a/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json +++ b/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json b/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json index a98ae0390bf55..90ea44179470f 100644 --- a/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json +++ b/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0110/testcase_0110_properties.json b/system_test/replication_testsuite/testcase_0110/testcase_0110_properties.json index f51abc156ae63..f11c705d92ca3 100644 --- a/system_test/replication_testsuite/testcase_0110/testcase_0110_properties.json +++ b/system_test/replication_testsuite/testcase_0110/testcase_0110_properties.json @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json b/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json index fff0d689eba5f..cc1eae690f67a 100644 --- a/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json +++ b/system_test/replication_testsuite/testcase_0111/testcase_0111_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json b/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json index 636f02a2342d8..48a6c9d2351a5 100644 --- a/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json +++ b/system_test/replication_testsuite/testcase_0112/testcase_0112_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json b/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json index bdb885e43829e..a88b49b4363df 100644 --- a/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json +++ b/system_test/replication_testsuite/testcase_0113/testcase_0113_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json b/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json index ca51c5f5f183f..1261614396166 100644 --- a/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json +++ b/system_test/replication_testsuite/testcase_0114/testcase_0114_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json b/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json index 459becf251d7c..2d649da83b01e 100644 --- a/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json +++ b/system_test/replication_testsuite/testcase_0115/testcase_0115_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json b/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json index c9471bc62ac6a..cbad6f2b7b9f7 100644 --- a/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json +++ b/system_test/replication_testsuite/testcase_0116/testcase_0116_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json b/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json index 8159464b16c2c..0099a8f5db4e9 100644 --- a/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json +++ b/system_test/replication_testsuite/testcase_0117/testcase_0117_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json b/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json index f73f0e43a3cf7..6954d4396276a 100644 --- a/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json +++ b/system_test/replication_testsuite/testcase_0118/testcase_0118_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0119/testcase_0119_properties.json b/system_test/replication_testsuite/testcase_0119/testcase_0119_properties.json index 442a481230b15..ab1e47a87c624 100644 --- a/system_test/replication_testsuite/testcase_0119/testcase_0119_properties.json +++ b/system_test/replication_testsuite/testcase_0119/testcase_0119_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json b/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json index 606aad399dd3a..c7940c460566c 100644 --- a/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json +++ b/system_test/replication_testsuite/testcase_0121/testcase_0121_properties.json @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -76,6 +77,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json b/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json index ea475361c8c54..35daf5b4cc3d6 100644 --- a/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json +++ b/system_test/replication_testsuite/testcase_0122/testcase_0122_properties.json @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -76,6 +77,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json b/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json index f9c92733cf465..fe5e49af72d82 100644 --- a/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json +++ b/system_test/replication_testsuite/testcase_0123/testcase_0123_properties.json @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -76,6 +77,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json b/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json index 02f8506b20d34..bff5d7342fc83 100644 --- a/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json +++ b/system_test/replication_testsuite/testcase_0124/testcase_0124_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json b/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json index 00482791aa969..1f57ecc0adac1 100644 --- a/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json +++ b/system_test/replication_testsuite/testcase_0125/testcase_0125_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json b/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json index 5119e61bbfca3..ffa0fc31fc972 100644 --- a/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json +++ b/system_test/replication_testsuite/testcase_0126/testcase_0126_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json b/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json index 8b53fa7df5056..78ecd8fa3f871 100644 --- a/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json +++ b/system_test/replication_testsuite/testcase_0127/testcase_0127_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0128/testcase_0128_properties.json b/system_test/replication_testsuite/testcase_0128/testcase_0128_properties.json index e8edb9f4c7009..589eb2068d6eb 100644 --- a/system_test/replication_testsuite/testcase_0128/testcase_0128_properties.json +++ b/system_test/replication_testsuite/testcase_0128/testcase_0128_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1,test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json b/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json index a140882287b9c..0324b6f327cb7 100644 --- a/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json +++ b/system_test/replication_testsuite/testcase_0131/testcase_0131_properties.json @@ -66,6 +66,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -79,6 +80,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json b/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json index 48b30c7e014f0..83bcaaac3a6e5 100644 --- a/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json +++ b/system_test/replication_testsuite/testcase_0132/testcase_0132_properties.json @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -76,6 +77,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json b/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json index 8276aae0aa822..2a1eaa51efb58 100644 --- a/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json +++ b/system_test/replication_testsuite/testcase_0133/testcase_0133_properties.json @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -76,6 +77,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0134/testcase_0134_properties.json b/system_test/replication_testsuite/testcase_0134/testcase_0134_properties.json index 73bb859952262..0a98ce5bb4ffe 100644 --- a/system_test/replication_testsuite/testcase_0134/testcase_0134_properties.json +++ b/system_test/replication_testsuite/testcase_0134/testcase_0134_properties.json @@ -68,6 +68,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1,test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0151/testcase_0151_properties.json b/system_test/replication_testsuite/testcase_0151/testcase_0151_properties.json index eebba4db5d978..237a34388f2bc 100644 --- a/system_test/replication_testsuite/testcase_0151/testcase_0151_properties.json +++ b/system_test/replication_testsuite/testcase_0151/testcase_0151_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0152/testcase_0152_properties.json b/system_test/replication_testsuite/testcase_0152/testcase_0152_properties.json index debf544b3541a..8d57610851775 100644 --- a/system_test/replication_testsuite/testcase_0152/testcase_0152_properties.json +++ b/system_test/replication_testsuite/testcase_0152/testcase_0152_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0153/testcase_0153_properties.json b/system_test/replication_testsuite/testcase_0153/testcase_0153_properties.json index 57b7d985236c8..89b933f674357 100644 --- a/system_test/replication_testsuite/testcase_0153/testcase_0153_properties.json +++ b/system_test/replication_testsuite/testcase_0153/testcase_0153_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0154/testcase_0154_properties.json b/system_test/replication_testsuite/testcase_0154/testcase_0154_properties.json index c09fab719459d..fe3f98fb36b5d 100644 --- a/system_test/replication_testsuite/testcase_0154/testcase_0154_properties.json +++ b/system_test/replication_testsuite/testcase_0154/testcase_0154_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0155/testcase_0155_properties.json b/system_test/replication_testsuite/testcase_0155/testcase_0155_properties.json index dd5ac52f86b02..7f9ced8990704 100644 --- a/system_test/replication_testsuite/testcase_0155/testcase_0155_properties.json +++ b/system_test/replication_testsuite/testcase_0155/testcase_0155_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0156/testcase_0156_properties.json b/system_test/replication_testsuite/testcase_0156/testcase_0156_properties.json index 8236ca58994b5..ec1e83cc11dc8 100644 --- a/system_test/replication_testsuite/testcase_0156/testcase_0156_properties.json +++ b/system_test/replication_testsuite/testcase_0156/testcase_0156_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0157/testcase_0157_properties.json b/system_test/replication_testsuite/testcase_0157/testcase_0157_properties.json index a28bf81e2b79e..e96ed325d7724 100644 --- a/system_test/replication_testsuite/testcase_0157/testcase_0157_properties.json +++ b/system_test/replication_testsuite/testcase_0157/testcase_0157_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0158/testcase_0158_properties.json b/system_test/replication_testsuite/testcase_0158/testcase_0158_properties.json index 3d6edbdb16c58..7ca29427d9c58 100644 --- a/system_test/replication_testsuite/testcase_0158/testcase_0158_properties.json +++ b/system_test/replication_testsuite/testcase_0158/testcase_0158_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0159/testcase_0159_properties.json b/system_test/replication_testsuite/testcase_0159/testcase_0159_properties.json index 030c9e8f0560f..cf7ccc3911ea0 100644 --- a/system_test/replication_testsuite/testcase_0159/testcase_0159_properties.json +++ b/system_test/replication_testsuite/testcase_0159/testcase_0159_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json b/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json index c6f8a23f358c5..521592b4d235c 100644 --- a/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json +++ b/system_test/replication_testsuite/testcase_0201/testcase_0201_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json b/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json index 7b1a4c25c73fb..c2feeb870ae9a 100644 --- a/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json +++ b/system_test/replication_testsuite/testcase_0202/testcase_0202_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json b/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json index 47276a85a038e..83b4dbceb26e0 100644 --- a/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json +++ b/system_test/replication_testsuite/testcase_0203/testcase_0203_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json b/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json index 3742cfad8f251..629b7baf43fff 100644 --- a/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json +++ b/system_test/replication_testsuite/testcase_0204/testcase_0204_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json b/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json index d41672d0b2818..a9d13c380ddc1 100644 --- a/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json +++ b/system_test/replication_testsuite/testcase_0205/testcase_0205_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json b/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json index a32d8882deb14..e31666917ab32 100644 --- a/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json +++ b/system_test/replication_testsuite/testcase_0206/testcase_0206_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json b/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json index 573787099f92c..1e08f46af4578 100644 --- a/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json +++ b/system_test/replication_testsuite/testcase_0207/testcase_0207_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json b/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json index c3d1d2cb90ce2..1dd38f478a820 100644 --- a/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json +++ b/system_test/replication_testsuite/testcase_0208/testcase_0208_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0209/testcase_0209_properties.json b/system_test/replication_testsuite/testcase_0209/testcase_0209_properties.json index 8d2cebafa0fb3..ac6b4d0013b66 100644 --- a/system_test/replication_testsuite/testcase_0209/testcase_0209_properties.json +++ b/system_test/replication_testsuite/testcase_0209/testcase_0209_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json b/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json index 76ea6775657c9..9f06f3072af84 100644 --- a/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json +++ b/system_test/replication_testsuite/testcase_0251/testcase_0251_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json b/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json index 672b3ff883f27..c264fca3876d4 100644 --- a/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json +++ b/system_test/replication_testsuite/testcase_0252/testcase_0252_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json b/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json index 19bd096b83735..e5fdb2eab95d6 100644 --- a/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json +++ b/system_test/replication_testsuite/testcase_0253/testcase_0253_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json b/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json index 8d6add098da11..27ce4e97ec01b 100644 --- a/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json +++ b/system_test/replication_testsuite/testcase_0254/testcase_0254_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json b/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json index 2a7e7776214d6..1148a4546b48a 100644 --- a/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json +++ b/system_test/replication_testsuite/testcase_0255/testcase_0255_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json b/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json index 91fbc9da75e29..1b58e9bd9bf13 100644 --- a/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json +++ b/system_test/replication_testsuite/testcase_0256/testcase_0256_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json b/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json index 5594a9bc76d1c..42e33c265a769 100644 --- a/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json +++ b/system_test/replication_testsuite/testcase_0257/testcase_0257_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json b/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json index 6eabd3772dbe6..ae9ce5eb72d7c 100644 --- a/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json +++ b/system_test/replication_testsuite/testcase_0258/testcase_0258_properties.json @@ -64,6 +64,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0259/testcase_0259_properties.json b/system_test/replication_testsuite/testcase_0259/testcase_0259_properties.json index 3bbeeb10adfd8..7278327226412 100644 --- a/system_test/replication_testsuite/testcase_0259/testcase_0259_properties.json +++ b/system_test/replication_testsuite/testcase_0259/testcase_0259_properties.json @@ -65,6 +65,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json b/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json index 496340c574ec3..f9b775e04b2ec 100644 --- a/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json +++ b/system_test/replication_testsuite/testcase_0301/testcase_0301_properties.json @@ -62,6 +62,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json b/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json index beabc86e2cdba..af468c3a733bb 100644 --- a/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json +++ b/system_test/replication_testsuite/testcase_0302/testcase_0302_properties.json @@ -62,6 +62,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json b/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json index 0e8f5a45895a3..374ff9e868524 100644 --- a/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json +++ b/system_test/replication_testsuite/testcase_0303/testcase_0303_properties.json @@ -62,6 +62,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json b/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json index 98a385556c781..1b0f2eea4cc68 100644 --- a/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json +++ b/system_test/replication_testsuite/testcase_0304/testcase_0304_properties.json @@ -62,6 +62,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json b/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json index e09ac9c09b971..568de4bb3b734 100644 --- a/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json +++ b/system_test/replication_testsuite/testcase_0305/testcase_0305_properties.json @@ -62,6 +62,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json b/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json index bcfe91b08c7e2..ab933380abbfe 100644 --- a/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json +++ b/system_test/replication_testsuite/testcase_0306/testcase_0306_properties.json @@ -62,6 +62,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json b/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json index da07a9e4751b5..06b0623841149 100644 --- a/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json +++ b/system_test/replication_testsuite/testcase_0307/testcase_0307_properties.json @@ -62,6 +62,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json b/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json index 0fecafc1a49c6..0fda7c615490f 100644 --- a/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json +++ b/system_test/replication_testsuite/testcase_0308/testcase_0308_properties.json @@ -62,6 +62,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_0309/testcase_0309_properties.json b/system_test/replication_testsuite/testcase_0309/testcase_0309_properties.json index 90bd40442cc95..2879c8ff9bbcc 100644 --- a/system_test/replication_testsuite/testcase_0309/testcase_0309_properties.json +++ b/system_test/replication_testsuite/testcase_0309/testcase_0309_properties.json @@ -63,6 +63,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json index 7e1b3fbf2d22b..680213f1e9756 100644 --- a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json +++ b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:0", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -41,7 +41,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -50,13 +50,14 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_10101/testcase_10101_properties.json b/system_test/replication_testsuite/testcase_10101/testcase_10101_properties.json new file mode 100644 index 0000000000000..3f8e5870799b7 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10101/testcase_10101_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : Base Test", + "02":"Produce and consume messages to a single topic - single partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:sync, acks:-1, comp:0", + "10":"Log segment size : 20480" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "0", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "sync":"true", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10102/testcase_10102_properties.json b/system_test/replication_testsuite/testcase_10102/testcase_10102_properties.json new file mode 100644 index 0000000000000..c96352d5e18e6 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10102/testcase_10102_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : 1. comp => 1", + "02":"Produce and consume messages to a single topic - single partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:sync, acks:-1, comp:1", + "10":"Log segment size : 20480" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "sync":"true", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10103/testcase_10103_properties.json b/system_test/replication_testsuite/testcase_10103/testcase_10103_properties.json new file mode 100644 index 0000000000000..55fa39e148818 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10103/testcase_10103_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : 1. acks => 1; 2. comp => 1", + "02":"Produce and consume messages to a single topic - single partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:sync, acks:1, comp:1", + "10":"Log segment size : 20480" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "sync":"true", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10104/testcase_10104_properties.json b/system_test/replication_testsuite/testcase_10104/testcase_10104_properties.json new file mode 100644 index 0000000000000..15827eb994ae8 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10104/testcase_10104_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : 1. mode => async; 2. comp => 0", + "02":"Produce and consume messages to a single topic - single partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:async, acks:-1, comp:1", + "10":"Log segment size : 20480" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "sync":"false", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10105/testcase_10105_properties.json b/system_test/replication_testsuite/testcase_10105/testcase_10105_properties.json new file mode 100644 index 0000000000000..d1fa1ade07c53 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10105/testcase_10105_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : 1. mode => async; 2. acks => 1; 3. comp => 1", + "02":"Produce and consume messages to a single topic - single partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:async, acks:1, comp:1", + "10":"Log segment size : 20480" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "1", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "sync":"false", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10106/testcase_10106_properties.json b/system_test/replication_testsuite/testcase_10106/testcase_10106_properties.json new file mode 100644 index 0000000000000..675c76f656efa --- /dev/null +++ b/system_test/replication_testsuite/testcase_10106/testcase_10106_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : 1. comp => 1; 2. no of partion => 3", + "02":"Produce and consume messages to a single topic - 3 partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:sync, acks:-1, comp:1", + "10":"Log segment size : 20480" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "3", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "sync":"true", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10107/testcase_10107_properties.json b/system_test/replication_testsuite/testcase_10107/testcase_10107_properties.json new file mode 100644 index 0000000000000..afc221cdc8bf7 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10107/testcase_10107_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : 1. mode => async; 2. comp => 1; 3. no of partition => 3", + "02":"Produce and consume messages to a single topic - 3 partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:async, acks:-1, comp:1", + "10":"Log segment size : 20480" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "3", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "sync":"false", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10108/testcase_10108_properties.json b/system_test/replication_testsuite/testcase_10108/testcase_10108_properties.json new file mode 100644 index 0000000000000..5df72f3d52a6c --- /dev/null +++ b/system_test/replication_testsuite/testcase_10108/testcase_10108_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : 1. acks => 1; 2. comp => 1; 3. no. of partition => 3", + "02":"Produce and consume messages to a single topic - 3 partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:sync, acks:1, comp:1", + "10":"Log segment size : 20480" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "3", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "sync":"true", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10109/testcase_10109_properties.json b/system_test/replication_testsuite/testcase_10109/testcase_10109_properties.json new file mode 100644 index 0000000000000..9b156681ec485 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10109/testcase_10109_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : 1. mode => async; 2. acks => 1; 3. comp =>; 4. no. of partitions => 3", + "02":"Produce and consume messages to a single topic - 3 partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:async, acks:1, comp:1", + "10":"Log segment size : 20480" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "3", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "20480", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "sync":"false", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10110/testcase_10110_properties.json b/system_test/replication_testsuite/testcase_10110/testcase_10110_properties.json new file mode 100644 index 0000000000000..f51abc156ae63 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10110/testcase_10110_properties.json @@ -0,0 +1,86 @@ +{ + "description": {"01":"Leader Failure in Replication : 1. mode => async; 2. acks => 1; 3. comp =>; 4. no. of partitins => 3; 5. log segment size => 1M", + "02":"Produce and consume messages to a single topic - 3 partition.", + "03":"This test sends messages to 3 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:async, acks:1, comp:1", + "10":"Log segment size : 1048576 (1M)" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "3", + "num_partition": "3", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "1048576", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "1048576", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "1048576", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "3", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "sync":"false", + "log_filename": "producer_performance.log", + "config_filename": "producer_performance.properties" + }, + { + "entity_id": "5", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "zookeeper": "localhost:2188", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10131/cluster_config.json b/system_test/replication_testsuite/testcase_10131/cluster_config.json new file mode 100644 index 0000000000000..cf147eb3f2024 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10131/cluster_config.json @@ -0,0 +1,76 @@ +{ + "cluster_config": [ + { + "entity_id": "0", + "hostname": "localhost", + "role": "zookeeper", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9990" + }, + { + "entity_id": "1", + "hostname": "localhost", + "role": "broker", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9991" + }, + { + "entity_id": "2", + "hostname": "localhost", + "role": "broker", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9992" + }, + { + "entity_id": "3", + "hostname": "localhost", + "role": "broker", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9993" + }, + { + "entity_id": "4", + "hostname": "localhost", + "role": "producer_performance", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9997" + }, + { + "entity_id": "5", + "hostname": "localhost", + "role": "producer_performance", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9998" + }, + { + "entity_id": "6", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9999" + }, + { + "entity_id": "7", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9099" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10131/testcase_10131_properties.json b/system_test/replication_testsuite/testcase_10131/testcase_10131_properties.json new file mode 100644 index 0000000000000..a140882287b9c --- /dev/null +++ b/system_test/replication_testsuite/testcase_10131/testcase_10131_properties.json @@ -0,0 +1,110 @@ +{ + "description": {"01":"Leader Failure in Replication with multi topics & partitions : Base Test", + "02":"Produce and consume messages to 2 topics - 3 partitions", + "03":"This test sends messages to 2 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:sync, acks:-1, comp:0", + "10":"Log segment size : 102400" + }, + "testcase_args": { + "broker_type": "leader", + "auto_create_topic": "true", + "bounce_broker": "true", + "replica_factor": "2", + "num_partition": "3", + "num_iteration": "3", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "102400", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "3", + "log.index.interval.bytes": "10", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "102400", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "3", + "log.index.interval.bytes": "10", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "102400", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "3", + "log.index.interval.bytes": "10", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "0", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", + "sync":"true", + "log_filename": "producer_performance_4.log", + "config_filename": "producer_performance_4.properties" + }, + { + "entity_id": "5", + "topic": "test_2", + "threads": "5", + "compression-codec": "0", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "producer-retry-backoff-ms": "300", + "sync":"true", + "log_filename": "producer_performance_5.log", + "config_filename": "producer_performance_5.properties" + }, + { + "entity_id": "6", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_6.log", + "config_filename": "console_consumer_6.properties" + }, + { + "entity_id": "7", + "topic": "test_2", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_7.log", + "config_filename": "console_consumer_7.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10132/cluster_config.json b/system_test/replication_testsuite/testcase_10132/cluster_config.json new file mode 100644 index 0000000000000..cf147eb3f2024 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10132/cluster_config.json @@ -0,0 +1,76 @@ +{ + "cluster_config": [ + { + "entity_id": "0", + "hostname": "localhost", + "role": "zookeeper", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9990" + }, + { + "entity_id": "1", + "hostname": "localhost", + "role": "broker", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9991" + }, + { + "entity_id": "2", + "hostname": "localhost", + "role": "broker", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9992" + }, + { + "entity_id": "3", + "hostname": "localhost", + "role": "broker", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9993" + }, + { + "entity_id": "4", + "hostname": "localhost", + "role": "producer_performance", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9997" + }, + { + "entity_id": "5", + "hostname": "localhost", + "role": "producer_performance", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9998" + }, + { + "entity_id": "6", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9999" + }, + { + "entity_id": "7", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9099" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10132/testcase_10132_properties.json b/system_test/replication_testsuite/testcase_10132/testcase_10132_properties.json new file mode 100644 index 0000000000000..48b30c7e014f0 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10132/testcase_10132_properties.json @@ -0,0 +1,107 @@ +{ + "description": {"01":"Leader Failure in Replication with multi topics & partitions : 1. acks => 1", + "02":"Produce and consume messages to 2 topics - 3 partitions", + "03":"This test sends messages to 2 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:sync, acks:1, comp:0", + "10":"Log segment size : 512000" + }, + "testcase_args": { + "broker_type": "leader", + "auto_create_topic": "true", + "bounce_broker": "true", + "replica_factor": "2", + "num_partition": "3", + "num_iteration": "3", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "512000", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "3", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "512000", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "3", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "512000", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "3", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "0", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "producer-retry-backoff-ms": "300", + "sync":"true", + "log_filename": "producer_performance_4.log", + "config_filename": "producer_performance_4.properties" + }, + { + "entity_id": "5", + "topic": "test_2", + "threads": "5", + "compression-codec": "0", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "producer-retry-backoff-ms": "300", + "sync":"true", + "log_filename": "producer_performance_5.log", + "config_filename": "producer_performance_5.properties" + }, + { + "entity_id": "6", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_6.log", + "config_filename": "console_consumer_6.properties" + }, + { + "entity_id": "7", + "topic": "test_2", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_7.log", + "config_filename": "console_consumer_7.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10133/cluster_config.json b/system_test/replication_testsuite/testcase_10133/cluster_config.json new file mode 100644 index 0000000000000..cf147eb3f2024 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10133/cluster_config.json @@ -0,0 +1,76 @@ +{ + "cluster_config": [ + { + "entity_id": "0", + "hostname": "localhost", + "role": "zookeeper", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9990" + }, + { + "entity_id": "1", + "hostname": "localhost", + "role": "broker", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9991" + }, + { + "entity_id": "2", + "hostname": "localhost", + "role": "broker", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9992" + }, + { + "entity_id": "3", + "hostname": "localhost", + "role": "broker", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9993" + }, + { + "entity_id": "4", + "hostname": "localhost", + "role": "producer_performance", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9997" + }, + { + "entity_id": "5", + "hostname": "localhost", + "role": "producer_performance", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9998" + }, + { + "entity_id": "6", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9999" + }, + { + "entity_id": "7", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name": "source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9099" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10133/testcase_10133_properties.json b/system_test/replication_testsuite/testcase_10133/testcase_10133_properties.json new file mode 100644 index 0000000000000..8276aae0aa822 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10133/testcase_10133_properties.json @@ -0,0 +1,107 @@ +{ + "description": {"01":"Leader Failure in Replication with multi topics & partitions : 1. mode => async; 2. comp => 0", + "02":"Produce and consume messages to 2 topics - 3 partitions", + "03":"This test sends messages to 2 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:async, acks:1, comp:1", + "10":"Log segment size : 512000" + }, + "testcase_args": { + "broker_type": "leader", + "auto_create_topic": "true", + "bounce_broker": "true", + "replica_factor": "2", + "num_partition": "3", + "num_iteration": "3", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "512000", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "2", + "num.partitions": "3", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "512000", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "2", + "num.partitions": "3", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "512000", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "2", + "num.partitions": "3", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "producer-retry-backoff-ms": "300", + "sync":"false", + "log_filename": "producer_performance_4.log", + "config_filename": "producer_performance_4.properties" + }, + { + "entity_id": "5", + "topic": "test_2", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "1", + "producer-retry-backoff-ms": "300", + "sync":"false", + "log_filename": "producer_performance_5.log", + "config_filename": "producer_performance_5.properties" + }, + { + "entity_id": "6", + "topic": "test_1", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_6.log", + "config_filename": "console_consumer_6.properties" + }, + { + "entity_id": "7", + "topic": "test_2", + "group.id": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_7.log", + "config_filename": "console_consumer_7.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_10134/testcase_10134_properties.json b/system_test/replication_testsuite/testcase_10134/testcase_10134_properties.json new file mode 100644 index 0000000000000..73bb859952262 --- /dev/null +++ b/system_test/replication_testsuite/testcase_10134/testcase_10134_properties.json @@ -0,0 +1,92 @@ +{ + "description": {"01":"Leader Failure in Replication with multi topics & partitions : 1. auto_create_topic => true", + "02":"Produce and consume messages to 2 topics - 3 partitions", + "03":"This test sends messages to 2 replicas", + "04":"To trigger leader election: find the leader and terminate by controlled failure (kill -15)", + "05":"Restart the terminated broker", + "06":"Lookup brokers' log4j messages and verify that leader is re-elected successfully", + "07":"At the end it verifies the log size and contents", + "08":"Use a consumer to verify no message loss.", + "09":"Producer dimensions : mode:sync, acks:-1, comp:0", + "10":"Log segment size : 102400" + }, + "testcase_args": { + "broker_type": "leader", + "bounce_broker": "true", + "replica_factor": "2", + "num_partition": "3", + "num_iteration": "3", + "auto_create_topic": "true", + "producer_multi_topics_mode": "true", + "consumer_multi_topics_mode": "true", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2188", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_2188.log", + "config_filename": "zookeeper_2188.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "num.partitions": "3", + "default.replication.factor": "2", + "log.segment.bytes": "102400", + "log.dir": "/tmp/kafka_server_1_logs", + "log.index.interval.bytes": "10", + "log_filename": "kafka_server_9091.log", + "config_filename": "kafka_server_9091.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "num.partitions": "3", + "default.replication.factor": "2", + "log.segment.bytes": "102400", + "log.dir": "/tmp/kafka_server_2_logs", + "log.index.interval.bytes": "10", + "log_filename": "kafka_server_9092.log", + "config_filename": "kafka_server_9092.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "num.partitions": "3", + "default.replication.factor": "2", + "log.segment.bytes": "102400", + "log.dir": "/tmp/kafka_server_3_logs", + "log.index.interval.bytes": "10", + "log_filename": "kafka_server_9093.log", + "config_filename": "kafka_server_9093.properties" + }, + { + "entity_id": "4", + "topic": "test_1,test_2", + "threads": "5", + "compression-codec": "1", + "message-size": "500", + "message": "100", + "request-num-acks": "-1", + "producer-retry-backoff-ms": "3500", + "producer-num-retries": "3", + "sync":"false", + "log_filename": "producer_performance_4.log", + "config_filename": "producer_performance_4.properties" + }, + { + "entity_id": "5", + "topic": "test_1,test_2", + "groupid": "mytestgroup", + "consumer-timeout-ms": "10000", + "log_filename": "console_consumer_5.log", + "config_filename": "console_consumer_5.properties" + } + ] +} diff --git a/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json b/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json index d2ffd952416e7..2652f16b78481 100644 --- a/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json +++ b/system_test/replication_testsuite/testcase_4001/testcase_4001_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json b/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json index c86525db84ad0..87245971271bf 100644 --- a/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json +++ b/system_test/replication_testsuite/testcase_4002/testcase_4002_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json b/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json index b77e4fdf40017..4e3b6f56281d6 100644 --- a/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json +++ b/system_test/replication_testsuite/testcase_4003/testcase_4003_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json b/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json index e753327795162..f8718a648ab30 100644 --- a/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json +++ b/system_test/replication_testsuite/testcase_4004/testcase_4004_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json b/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json index 5468401c6b41d..af96c7b3bdaf6 100644 --- a/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json +++ b/system_test/replication_testsuite/testcase_4005/testcase_4005_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json b/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json index e5ab0a0e86981..e132236ff3bd2 100644 --- a/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json +++ b/system_test/replication_testsuite/testcase_4006/testcase_4006_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json b/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json index 7aa6e9a72ca74..5c4e5bbfd7da4 100644 --- a/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json +++ b/system_test/replication_testsuite/testcase_4007/testcase_4007_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json b/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json index 08aa108f4f43a..8dce9b2b3883c 100644 --- a/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json +++ b/system_test/replication_testsuite/testcase_4008/testcase_4008_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json b/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json index 512fafb949c0d..c6f1d1c15f832 100644 --- a/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json +++ b/system_test/replication_testsuite/testcase_4011/testcase_4011_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json b/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json index 9b711af013994..bc1ff6397baa5 100644 --- a/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json +++ b/system_test/replication_testsuite/testcase_4012/testcase_4012_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json b/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json index 3836366b3b19e..aa48a6861a7e0 100644 --- a/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json +++ b/system_test/replication_testsuite/testcase_4013/testcase_4013_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json b/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json index 86ab75a46a48e..7acf8b68514ea 100644 --- a/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json +++ b/system_test/replication_testsuite/testcase_4014/testcase_4014_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json b/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json index 82d51b6072c6d..7841273a3876f 100644 --- a/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json +++ b/system_test/replication_testsuite/testcase_4015/testcase_4015_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json b/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json index 31c1be0d73393..0519d273dcf34 100644 --- a/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json +++ b/system_test/replication_testsuite/testcase_4016/testcase_4016_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "0", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "0", diff --git a/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json b/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json index 72f78b074243f..c29077bfd5dc1 100644 --- a/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json +++ b/system_test/replication_testsuite/testcase_4017/testcase_4017_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json b/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json index ee459f4e35d68..ab57e5ab18703 100644 --- a/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json +++ b/system_test/replication_testsuite/testcase_4018/testcase_4018_properties.json @@ -67,6 +67,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "test_1", "threads": "5", "compression-codec": "1", @@ -80,6 +81,7 @@ }, { "entity_id": "5", + "new-producer":"true", "topic": "test_2", "threads": "5", "compression-codec": "1", diff --git a/system_test/replication_testsuite/testcase_9051/testcase_9051_properties.json b/system_test/replication_testsuite/testcase_9051/testcase_9051_properties.json index 958eef7df0be1..e959aeda4fabe 100644 --- a/system_test/replication_testsuite/testcase_9051/testcase_9051_properties.json +++ b/system_test/replication_testsuite/testcase_9051/testcase_9051_properties.json @@ -60,6 +60,7 @@ }, { "entity_id": "4", + "new-producer":"true", "topic": "t001", "threads": "5", "compression-codec": "0", diff --git a/system_test/run_all.sh b/system_test/run_all.sh new file mode 100755 index 0000000000000..0c5c02d98efa4 --- /dev/null +++ b/system_test/run_all.sh @@ -0,0 +1,7 @@ +#!/bin/bash + +cp testcase_to_run_all.json testcase_to_run.json + +python -B system_test_runner.py + + diff --git a/system_test/run_all_replica.sh b/system_test/run_all_replica.sh new file mode 100755 index 0000000000000..b3bce84381472 --- /dev/null +++ b/system_test/run_all_replica.sh @@ -0,0 +1,7 @@ +#!/bin/bash + +cp testcase_to_run_all_replica.json testcase_to_run.json + +python -B system_test_runner.py + + diff --git a/system_test/run_sanity.sh b/system_test/run_sanity.sh index 9e8042a23b935..a301b96e1a7cd 100755 --- a/system_test/run_sanity.sh +++ b/system_test/run_sanity.sh @@ -1,8 +1,5 @@ #!/bin/bash -my_ts=`date +"%s"` - -cp testcase_to_run.json testcase_to_run.json_${my_ts} cp testcase_to_run_sanity.json testcase_to_run.json python -B system_test_runner.py diff --git a/system_test/system_test_runner.py b/system_test/system_test_runner.py index ee7aa25233355..d6251b2af836e 100644 --- a/system_test/system_test_runner.py +++ b/system_test/system_test_runner.py @@ -52,6 +52,131 @@ # message to facilitate debugging. d = {'name_of_class': '(system_test_runner)'} +class report: + systemTestEnv = None + reportString = "" + reportFileName = "system_test_report.html" + systemTestReport = None + header = """ + Kafka System Test Report + + + + + + """ + footer = """ """ + + def __init__(self, systemTestEnv): + self.totalSkipped = 0 + self.totalPassed = 0 + self.totalTests = 0 + self.totalFailed = 0 + self.systemTestEnv = systemTestEnv + self.systemTestReport = open(self.reportFileName, 'w') + + def __del__(self): + self.systemTestReport.close() + self.systemTestReport = None + + def writeHtmlPage(self, body): + html = """ + + + """ + html += self.header + html += body + html += self.footer + html += """ + + """ + self.systemTestReport.write(html) + + def wrapIn(self, tag, content): + html = "\n<" + tag + ">" + html += "\n " + content + html += "\n" + return html + + def genModal(self, className, caseName, systemTestResult): + key = "validation_status" + id = className + "_" + caseName + info = self.wrapIn("h4", "Validation Status") + for validatedItem in sorted(systemTestResult[key].iterkeys()): + testItemStatus = systemTestResult[key][validatedItem] + info += validatedItem + " : " + testItemStatus + return self.wrapIn("div class=\"modal fade\" id=\"" + id + "\" tabindex=\"-1\" role=\"dialog\" aria-labelledby=\"" + id + "Label\" aria-hidden=\"true\"", + self.wrapIn("div class=\"modal-dialog\"", + self.wrapIn("div class=\"modal-content\"", + self.wrapIn("div class=\"modal-header\"", + self.wrapIn("h4 class=\"modal-title\" id=\"" + id + "Label\"", + className + " - " + caseName)) + + self.wrapIn("div class=\"modal-body\"", + info) + + self.wrapIn("div class=\"modal-footer\"", + self.wrapIn("button type=\"button\" class=\"btn btn-default\" data-dismiss=\"modal\"", "Close"))))) + + def summarize(self): + testItemsTableHeader = self.wrapIn("thead", + self.wrapIn("tr", + self.wrapIn("th", "Test Class Name") + + self.wrapIn("th", "Test Case Name") + + self.wrapIn("th", "Validation Status"))) + testItemsTableBody = "" + modals = "" + + for systemTestResult in self.systemTestEnv.systemTestResultsList: + self.totalTests += 1 + if "_test_class_name" in systemTestResult: + testClassName = systemTestResult["_test_class_name"] + else: + testClassName = "" + + if "_test_case_name" in systemTestResult: + testCaseName = systemTestResult["_test_case_name"] + else: + testCaseName = "" + + if "validation_status" in systemTestResult: + testItemStatus = "SKIPPED" + for key in systemTestResult["validation_status"].iterkeys(): + testItemStatus = systemTestResult["validation_status"][key] + if "FAILED" == testItemStatus: + break; + if "FAILED" == testItemStatus: + self.totalFailed += 1 + validationStatus = self.wrapIn("div class=\"text-danger\" data-toggle=\"modal\" data-target=\"#" + testClassName + "_" + testCaseName + "\"", "FAILED") + modals += self.genModal(testClassName, testCaseName, systemTestResult) + elif "PASSED" == testItemStatus: + self.totalPassed += 1 + validationStatus = self.wrapIn("div class=\"text-success\"", "PASSED") + else: + self.totalSkipped += 1 + validationStatus = self.wrapIn("div class=\"text-warning\"", "SKIPPED") + else: + self.reportString += "|" + + testItemsTableBody += self.wrapIn("tr", + self.wrapIn("td", testClassName) + + self.wrapIn("td", testCaseName) + + self.wrapIn("td", validationStatus)) + + testItemsTableBody = self.wrapIn("tbody", testItemsTableBody) + testItemsTable = self.wrapIn("table class=\"table table-striped\"", testItemsTableHeader + testItemsTableBody) + + statsTblBody = self.wrapIn("tr class=\"active\"", self.wrapIn("td", "Total tests") + self.wrapIn("td", str(self.totalTests))) + statsTblBody += self.wrapIn("tr class=\"success\"", self.wrapIn("td", "Total tests passed") + self.wrapIn("td", str(self.totalPassed))) + statsTblBody += self.wrapIn("tr class=\"danger\"", self.wrapIn("td", "Total tests failed") + self.wrapIn("td", str(self.totalFailed))) + statsTblBody += self.wrapIn("tr class=\"warning\"", self.wrapIn("td", "Total tests skipped") + self.wrapIn("td", str(self.totalSkipped))) + testStatsTable = self.wrapIn("table class=\"table\"", statsTblBody) + + body = self.wrapIn("div class=\"container\"", + self.wrapIn("h2", "Kafka System Test Report") + + self.wrapIn("div class=\"row\"", self.wrapIn("div class=\"col-md-4\"", testStatsTable)) + + self.wrapIn("div class=\"row\"", self.wrapIn("div class=\"col-md-6\"", testItemsTable)) + + modals) + self.writeHtmlPage(self.wrapIn("body", body)) + def main(): nLogger = logging.getLogger('namedLogger') aLogger = logging.getLogger('anonymousLogger') @@ -98,7 +223,7 @@ def main(): nLogger.info("SKIPPING : checking remote machines", extra=d) print - # get all defined names within a module: + # get all defined names within a module: definedItemList = dir(SystemTestEnv) aLogger.debug("=================================================") aLogger.debug("SystemTestEnv keys:") @@ -129,7 +254,7 @@ def main(): and not moduleFileName.startswith("__"): # found a test module file - nLogger.info("found a test module file : " + moduleFileName, extra=d) + nLogger.info("found a test module file : " + moduleFileName, extra=d) testModuleClassName = system_test_utils.sys_call("grep ^class " + testModulePathName + "/" + \ moduleFileName + " | sed 's/^class //g' | sed 's/(.*):.*//g'") @@ -138,7 +263,7 @@ def main(): # collect the test suite class data testSuiteClassDict = {} testSuiteClassDict["suite"] = dirName - extLenToRemove = systemTestEnv.SYSTEM_TEST_MODULE_EXT.__len__() * -1 + extLenToRemove = systemTestEnv.SYSTEM_TEST_MODULE_EXT.__len__() * -1 testSuiteClassDict["module"] = moduleFileName[:extLenToRemove] testSuiteClassDict["class"] = testModuleClassName testSuiteClassDictList.append(testSuiteClassDict) @@ -149,7 +274,7 @@ def main(): # add testsuite directory to sys.path such that the module can be loaded sys.path.append(systemTestEnv.SYSTEM_TEST_BASE_DIR + "/" + suiteName) - + if not systemTestEnv.printTestDescriptionsOnly: aLogger.info("=================================================") aLogger.info("Running Test for : ") @@ -165,6 +290,8 @@ def main(): instance.runTest() print + report(systemTestEnv).summarize() + if not systemTestEnv.printTestDescriptionsOnly: totalFailureCount = 0 print @@ -199,6 +326,6 @@ def main(): # main entry point # ========================= -main() +sys.exit(main()) diff --git a/system_test/testcase_to_run.json b/system_test/testcase_to_run.json deleted file mode 100644 index 8252860bcd734..0000000000000 --- a/system_test/testcase_to_run.json +++ /dev/null @@ -1,5 +0,0 @@ -{ - "ReplicaBasicTest" : [ - "testcase_0001" - ] -} diff --git a/system_test/testcase_to_run_all.json b/system_test/testcase_to_run_all.json index 182160c6ea347..3e80a1f79a63c 100644 --- a/system_test/testcase_to_run_all.json +++ b/system_test/testcase_to_run_all.json @@ -26,6 +26,17 @@ "testcase_0109", "testcase_0110", + "testcase_10101", + "testcase_10102", + "testcase_10103", + "testcase_10104", + "testcase_10105", + "testcase_10106", + "testcase_10107", + "testcase_10108", + "testcase_10109", + "testcase_10110", + "testcase_0111", "testcase_0112", "testcase_0113", @@ -46,6 +57,12 @@ "testcase_0131", "testcase_0132", "testcase_0133", + "testcase_0134", + + "testcase_10131", + "testcase_10132", + "testcase_10133", + "testcase_10134", "testcase_0151", "testcase_0152", @@ -104,20 +121,19 @@ "testcase_9051" ], - "MigrationToolTest" : [ - "testcase_9001", - "testcase_9003", - "testcase_9004", - "testcase_9005", - "testcase_9006" - ], - "MirrorMakerTest" : [ "testcase_5001", "testcase_5002", "testcase_5003", "testcase_5004", "testcase_5005", - "testcase_5006" + "testcase_5006", + + "testcase_15001", + "testcase_15002", + "testcase_15003", + "testcase_15004", + "testcase_15005", + "testcase_15006" ] } diff --git a/system_test/testcase_to_run_all_replica.json b/system_test/testcase_to_run_all_replica.json new file mode 100644 index 0000000000000..34841f55c8cb7 --- /dev/null +++ b/system_test/testcase_to_run_all_replica.json @@ -0,0 +1,123 @@ +{ + "ReplicaBasicTest" : [ + "testcase_0001", + "testcase_0002", + "testcase_0003", + "testcase_0004", + "testcase_0005", + "testcase_0006", + "testcase_0007", + "testcase_0008", + "testcase_0009", + "testcase_0010", + + "testcase_0021", + "testcase_0022", + "testcase_0023", + + "testcase_0101", + "testcase_0102", + "testcase_0103", + "testcase_0104", + "testcase_0105", + "testcase_0106", + "testcase_0107", + "testcase_0108", + "testcase_0109", + "testcase_0110", + + "testcase_10101", + "testcase_10102", + "testcase_10103", + "testcase_10104", + "testcase_10105", + "testcase_10106", + "testcase_10107", + "testcase_10108", + "testcase_10109", + "testcase_10110", + + "testcase_0111", + "testcase_0112", + "testcase_0113", + "testcase_0114", + "testcase_0115", + "testcase_0116", + "testcase_0117", + "testcase_0118", + + "testcase_0121", + "testcase_0122", + "testcase_0123", + "testcase_0124", + "testcase_0125", + "testcase_0126", + "testcase_0127", + + "testcase_0131", + "testcase_0132", + "testcase_0133", + "testcase_0134", + + "testcase_10131", + "testcase_10132", + "testcase_10133", + "testcase_10134", + + "testcase_0151", + "testcase_0152", + "testcase_0153", + "testcase_0154", + "testcase_0155", + "testcase_0156", + "testcase_0157", + "testcase_0158", + + "testcase_0201", + "testcase_0202", + "testcase_0203", + "testcase_0204", + "testcase_0205", + "testcase_0206", + "testcase_0207", + "testcase_0208", + + "testcase_0251", + "testcase_0252", + "testcase_0253", + "testcase_0254", + "testcase_0255", + "testcase_0256", + "testcase_0257", + "testcase_0258", + + "testcase_0301", + "testcase_0302", + "testcase_0303", + "testcase_0304", + "testcase_0305", + "testcase_0306", + "testcase_0307", + "testcase_0308", + + "testcase_4001", + "testcase_4002", + "testcase_4003", + "testcase_4004", + "testcase_4005", + "testcase_4006", + "testcase_4007", + "testcase_4008", + + "testcase_4011", + "testcase_4012", + "testcase_4013", + "testcase_4014", + "testcase_4015", + "testcase_4016", + "testcase_4017", + "testcase_4018", + + "testcase_9051" + ] +} diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index fb4a9c05bf6f3..a9b73f7f3831a 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -117,7 +117,7 @@ def generate_testcase_log_dirs(systemTestEnv, testcaseEnv): # create the role directory under dashboards dashboardsRoleDir = dashboardsPathName + "/" + role if not os.path.exists(dashboardsRoleDir) : os.makedirs(dashboardsRoleDir) - + def collect_logs_from_remote_hosts(systemTestEnv, testcaseEnv): anonLogger.info("================================================") @@ -212,7 +212,7 @@ def collect_logs_from_remote_hosts(systemTestEnv, testcaseEnv): logger.debug("executing command [" + cmdStr + "]", extra=d) system_test_utils.sys_call(cmdStr) - + def generate_testcase_log_dirs_in_remote_hosts(systemTestEnv, testcaseEnv): testCaseBaseDir = testcaseEnv.testCaseBaseDir @@ -432,10 +432,11 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv sys.exit(1) addedCSVConfig = {} - addedCSVConfig["kafka.csv.metrics.dir"] = get_testcase_config_log_dir_pathname(testcaseEnv, "broker", clusterCfg["entity_id"], "metrics") - addedCSVConfig["kafka.metrics.polling.interval.secs"] = "5" - addedCSVConfig["kafka.metrics.reporters"] = "kafka.metrics.KafkaCSVMetricsReporter" + addedCSVConfig["kafka.csv.metrics.dir"] = get_testcase_config_log_dir_pathname(testcaseEnv, "broker", clusterCfg["entity_id"], "metrics") + addedCSVConfig["kafka.metrics.polling.interval.secs"] = "5" + addedCSVConfig["kafka.metrics.reporters"] = "kafka.metrics.KafkaCSVMetricsReporter" addedCSVConfig["kafka.csv.metrics.reporter.enabled"] = "true" + addedCSVConfig["listeners"] = "PLAINTEXT://localhost:"+tcCfg["port"] if brokerVersion == "0.7": addedCSVConfig["brokerid"] = tcCfg["brokerid"] @@ -458,6 +459,7 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv elif ( clusterCfg["role"] == "mirror_maker"): tcCfg["metadata.broker.list"] = testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] + tcCfg["bootstrap.servers"] = testcaseEnv.userDefinedEnvVarDict["targetBrokerList"] # for new producer copy_file_with_dict_values(cfgTemplatePathname + "/mirror_producer.properties", cfgDestPathname + "/" + tcCfg["mirror_producer_config_filename"], tcCfg, None) @@ -465,7 +467,7 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv tcCfg["zookeeper.connect"] = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] copy_file_with_dict_values(cfgTemplatePathname + "/mirror_consumer.properties", cfgDestPathname + "/" + tcCfg["mirror_consumer_config_filename"], tcCfg, None) - + else: logger.debug("UNHANDLED role " + clusterCfg["role"], extra=d) @@ -494,7 +496,7 @@ def scp_file_to_remote_host(clusterEntityConfigDictList, testcaseEnv): def start_zookeepers(systemTestEnv, testcaseEnv): clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList - zkEntityIdList = system_test_utils.get_data_from_list_of_dicts( + zkEntityIdList = system_test_utils.get_data_from_list_of_dicts( clusterEntityConfigDictList, "role", "zookeeper", "entity_id") for zkEntityId in zkEntityIdList: @@ -533,12 +535,23 @@ def start_zookeepers(systemTestEnv, testcaseEnv): def start_brokers(systemTestEnv, testcaseEnv): clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList - brokerEntityIdList = system_test_utils.get_data_from_list_of_dicts( + brokerEntityIdList = system_test_utils.get_data_from_list_of_dicts( clusterEntityConfigDictList, "role", "broker", "entity_id") for brokerEntityId in brokerEntityIdList: start_entity_in_background(systemTestEnv, testcaseEnv, brokerEntityId) +def start_console_consumers(systemTestEnv, testcaseEnv, onlyThisEntityId=None): + + if onlyThisEntityId is not None: + start_entity_in_background(systemTestEnv, testcaseEnv, onlyThisEntityId) + else: + clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList + consoleConsumerEntityIdList = system_test_utils.get_data_from_list_of_dicts( + clusterEntityConfigDictList, "role", "console_consumer", "entity_id") + for entityId in consoleConsumerEntityIdList: + start_entity_in_background(systemTestEnv, testcaseEnv, entityId) + def start_mirror_makers(systemTestEnv, testcaseEnv, onlyThisEntityId=None): @@ -546,7 +559,7 @@ def start_mirror_makers(systemTestEnv, testcaseEnv, onlyThisEntityId=None): start_entity_in_background(systemTestEnv, testcaseEnv, onlyThisEntityId) else: clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList - brokerEntityIdList = system_test_utils.get_data_from_list_of_dicts( + brokerEntityIdList = system_test_utils.get_data_from_list_of_dicts( clusterEntityConfigDictList, "role", "mirror_maker", "entity_id") for brokerEntityId in brokerEntityIdList: @@ -559,17 +572,17 @@ def get_broker_shutdown_log_line(systemTestEnv, testcaseEnv, leaderAttributesDic # keep track of broker related data in this dict such as broker id, # entity id and timestamp and return it to the caller function - shutdownBrokerDict = {} + shutdownBrokerDict = {} clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList - brokerEntityIdList = system_test_utils.get_data_from_list_of_dicts( + brokerEntityIdList = system_test_utils.get_data_from_list_of_dicts( clusterEntityConfigDictList, "role", "broker", "entity_id") for brokerEntityId in brokerEntityIdList: - hostname = system_test_utils.get_data_by_lookup_keyval( + hostname = system_test_utils.get_data_by_lookup_keyval( clusterEntityConfigDictList, "entity_id", brokerEntityId, "hostname") - logFile = system_test_utils.get_data_by_lookup_keyval( + logFile = system_test_utils.get_data_by_lookup_keyval( testcaseEnv.testcaseConfigsList, "entity_id", brokerEntityId, "log_filename") logPathName = get_testcase_config_log_dir_pathname(testcaseEnv, "broker", brokerEntityId, "default") @@ -617,7 +630,7 @@ def get_leader_elected_log_line(systemTestEnv, testcaseEnv, leaderAttributesDict # keep track of leader related data in this dict such as broker id, # entity id and timestamp and return it to the caller function - leaderDict = {} + leaderDict = {} clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList brokerEntityIdList = system_test_utils.get_data_from_list_of_dicts( \ @@ -696,6 +709,7 @@ def start_entity_in_background(systemTestEnv, testcaseEnv, entityId): configFile = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "config_filename") logFile = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "log_filename") + useNewProducer = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "new-producer") mmConsumerConfigFile = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "mirror_consumer_config_filename") mmProducerConfigFile = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, @@ -722,27 +736,107 @@ def start_entity_in_background(systemTestEnv, testcaseEnv, entityId): elif role == "broker": cmdList = ["ssh " + hostname, "'JAVA_HOME=" + javaHome, - "JMX_PORT=" + jmxPort, + "JMX_PORT=" + jmxPort, + "KAFKA_LOG4J_OPTS=-Dlog4j.configuration=file:%s/config/log4j.properties" % kafkaHome, kafkaHome + "/bin/kafka-run-class.sh kafka.Kafka", configPathName + "/" + configFile + " >> ", logPathName + "/" + logFile + " & echo pid:$! > ", logPathName + "/entity_" + entityId + "_pid'"] elif role == "mirror_maker": + if useNewProducer.lower() == "true": + cmdList = ["ssh " + hostname, + "'JAVA_HOME=" + javaHome, + "JMX_PORT=" + jmxPort, + kafkaHome + "/bin/kafka-run-class.sh kafka.tools.MirrorMaker", + "--consumer.config " + configPathName + "/" + mmConsumerConfigFile, + "--producer.config " + configPathName + "/" + mmProducerConfigFile, + "--new.producer", + "--whitelist=\".*\" >> ", + logPathName + "/" + logFile + " & echo pid:$! > ", + logPathName + "/entity_" + entityId + "_pid'"] + else: + cmdList = ["ssh " + hostname, + "'JAVA_HOME=" + javaHome, + "JMX_PORT=" + jmxPort, + kafkaHome + "/bin/kafka-run-class.sh kafka.tools.MirrorMaker", + "--consumer.config " + configPathName + "/" + mmConsumerConfigFile, + "--producer.config " + configPathName + "/" + mmProducerConfigFile, + "--whitelist=\".*\" >> ", + logPathName + "/" + logFile + " & echo pid:$! > ", + logPathName + "/entity_" + entityId + "_pid'"] + + elif role == "console_consumer": + clusterToConsumeFrom = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "cluster_name") + numTopicsForAutoGenString = -1 + try: + numTopicsForAutoGenString = int(testcaseEnv.testcaseArgumentsDict["num_topics_for_auto_generated_string"]) + except: + pass + + topic = "" + if numTopicsForAutoGenString < 0: + topic = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "topic") + else: + topic = generate_topics_string("topic", numTopicsForAutoGenString) + + # update this variable and will be used by data validation functions + testcaseEnv.consumerTopicsString = topic + + # 2. consumer timeout + timeoutMs = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "consumer-timeout-ms") + + # 3. consumer formatter + formatterOption = "" + try: + formatterOption = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "formatter") + except: + pass + + # 4. consumer config + consumerProperties = {} + consumerProperties["consumer.timeout.ms"] = timeoutMs + try: + groupOption = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "group.id") + consumerProperties["group.id"] = groupOption + except: + pass + + props_file_path=write_consumer_properties(consumerProperties) + scpCmdStr = "scp "+ props_file_path +" "+ hostname + ":/tmp/" + logger.debug("executing command [" + scpCmdStr + "]", extra=d) + system_test_utils.sys_call(scpCmdStr) + + if len(formatterOption) > 0: + formatterOption = " --formatter " + formatterOption + " " + + # get zookeeper connect string + zkConnectStr = "" + if clusterName == "source": + zkConnectStr = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] + elif clusterName == "target": + zkConnectStr = testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"] + else: + logger.error("Invalid cluster name : " + clusterName, extra=d) + sys.exit(1) cmdList = ["ssh " + hostname, - "'JAVA_HOME=" + javaHome, - "JMX_PORT=" + jmxPort, - kafkaHome + "/bin/kafka-run-class.sh kafka.tools.MirrorMaker", - "--consumer.config " + configPathName + "/" + mmConsumerConfigFile, - "--producer.config " + configPathName + "/" + mmProducerConfigFile, - "--whitelist=\".*\" >> ", - logPathName + "/" + logFile + " & echo pid:$! > ", - logPathName + "/entity_" + entityId + "_pid'"] + "'JAVA_HOME=" + javaHome, + "JMX_PORT=" + jmxPort, + kafkaHome + "/bin/kafka-run-class.sh kafka.tools.ConsoleConsumer", + "--zookeeper " + zkConnectStr, + "--topic " + topic, + "--consumer.config /tmp/consumer.properties", + "--csv-reporter-enabled", + formatterOption, + "--from-beginning", + " >> " + logPathName + "/" + logFile + " & echo pid:$! > ", + logPathName + "/entity_" + entityId + "_pid'"] cmdStr = " ".join(cmdList) logger.debug("executing command: [" + cmdStr + "]", extra=d) system_test_utils.async_sys_call(cmdStr) + logger.info("sleeping for 5 seconds.", extra=d) time.sleep(5) pidCmdStr = "ssh " + hostname + " 'cat " + logPathName + "/entity_" + entityId + "_pid' 2> /dev/null" @@ -761,6 +855,8 @@ def start_entity_in_background(systemTestEnv, testcaseEnv, entityId): testcaseEnv.entityBrokerParentPidDict[entityId] = tokens[1] elif role == "mirror_maker": testcaseEnv.entityMirrorMakerParentPidDict[entityId] = tokens[1] + elif role == "console_consumer": + testcaseEnv.entityConsoleConsumerParentPidDict[entityId] = tokens[1] def start_console_consumer(systemTestEnv, testcaseEnv): @@ -771,9 +867,9 @@ def start_console_consumer(systemTestEnv, testcaseEnv): for consumerConfig in consumerConfigList: host = consumerConfig["hostname"] entityId = consumerConfig["entity_id"] - jmxPort = consumerConfig["jmx_port"] + jmxPort = consumerConfig["jmx_port"] role = consumerConfig["role"] - clusterName = consumerConfig["cluster_name"] + clusterName = consumerConfig["cluster_name"] kafkaHome = system_test_utils.get_data_by_lookup_keyval(clusterList, "entity_id", entityId, "kafka_home") javaHome = system_test_utils.get_data_by_lookup_keyval(clusterList, "entity_id", entityId, "java_home") jmxPort = system_test_utils.get_data_by_lookup_keyval(clusterList, "entity_id", entityId, "jmx_port") @@ -835,13 +931,20 @@ def start_console_consumer(systemTestEnv, testcaseEnv): logger.error("Invalid cluster name : " + clusterName, extra=d) sys.exit(1) + consumerProperties = {} + consumerProperties["consumer.timeout.ms"] = timeoutMs + props_file_path=write_consumer_properties(consumerProperties) + scpCmdStr = "scp "+ props_file_path +" "+ host + ":/tmp/" + logger.debug("executing command [" + scpCmdStr + "]", extra=d) + system_test_utils.sys_call(scpCmdStr) + cmdList = ["ssh " + host, "'JAVA_HOME=" + javaHome, "JMX_PORT=" + jmxPort, - kafkaRunClassBin + " kafka.consumer.ConsoleConsumer", + kafkaRunClassBin + " kafka.tools.ConsoleConsumer", "--zookeeper " + zkConnectStr, "--topic " + topic, - "--consumer-timeout-ms " + timeoutMs, + "--consumer.config /tmp/consumer.properties", "--csv-reporter-enabled", #"--metrics-dir " + metricsDir, formatterOption, @@ -884,14 +987,16 @@ def start_producer_performance(systemTestEnv, testcaseEnv, kafka07Client): for producerConfig in producerConfigList: host = producerConfig["hostname"] entityId = producerConfig["entity_id"] - jmxPort = producerConfig["jmx_port"] - role = producerConfig["role"] + jmxPort = producerConfig["jmx_port"] + role = producerConfig["role"] thread.start_new_thread(start_producer_in_thread, (testcaseEnv, entityConfigList, producerConfig, kafka07Client)) + logger.debug("calling testcaseEnv.lock.acquire()", extra=d) testcaseEnv.lock.acquire() testcaseEnv.numProducerThreadsRunning += 1 logger.debug("testcaseEnv.numProducerThreadsRunning : " + str(testcaseEnv.numProducerThreadsRunning), extra=d) time.sleep(1) + logger.debug("calling testcaseEnv.lock.release()", extra=d) testcaseEnv.lock.release() def generate_topics_string(topicPrefix, numOfTopics): @@ -925,7 +1030,7 @@ def generate_topics_string(topicPrefix, numOfTopics): def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafka07Client): host = producerConfig["hostname"] entityId = producerConfig["entity_id"] - jmxPort = producerConfig["jmx_port"] + jmxPort = producerConfig["jmx_port"] role = producerConfig["role"] clusterName = producerConfig["cluster_name"] kafkaHome = system_test_utils.get_data_by_lookup_keyval(entityConfigList, "entity_id", entityId, "kafka_home") @@ -933,6 +1038,9 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk jmxPort = system_test_utils.get_data_by_lookup_keyval(entityConfigList, "entity_id", entityId, "jmx_port") kafkaRunClassBin = kafkaHome + "/bin/kafka-run-class.sh" + # first keep track of its pid + testcaseEnv.producerHostParentPidDict[entityId] = os.getpid() + # get optional testcase arguments numTopicsForAutoGenString = -1 try: @@ -957,6 +1065,7 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk noMsgPerBatch = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "message") requestNumAcks = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "request-num-acks") syncMode = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "sync") + useNewProducer = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "new-producer") retryBackoffMs = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "producer-retry-backoff-ms") numOfRetries = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "producer-num-retries") @@ -995,6 +1104,8 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk boolArgumentsStr = "" if syncMode.lower() == "true": boolArgumentsStr = boolArgumentsStr + " --sync" + if useNewProducer.lower() == "true": + boolArgumentsStr = boolArgumentsStr + " --new-producer" # keep calling producer until signaled to stop by: # testcaseEnv.userDefinedEnvVarDict["stopBackgroundProducer"] @@ -1011,7 +1122,7 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk "'JAVA_HOME=" + javaHome, "JMX_PORT=" + jmxPort, "KAFKA_LOG4J_OPTS=-Dlog4j.configuration=file:%s/config/test-log4j.properties" % kafkaHome, - kafkaRunClassBin + " kafka.perf.ProducerPerformance", + kafkaRunClassBin + " kafka.tools.ProducerPerformance", "--broker-list " + brokerListStr, "--initial-message-id " + str(initMsgId), "--messages " + noMsgPerBatch, @@ -1026,7 +1137,8 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk "--metrics-dir " + metricsDir, boolArgumentsStr, " >> " + producerLogPathName, - " & echo pid:$! > " + producerLogPath + "/entity_" + entityId + "_pid'"] + " & echo $! > " + producerLogPath + "/entity_" + entityId + "_pid", + " & wait'"] if kafka07Client: cmdList[:] = [] @@ -1047,7 +1159,7 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk "'JAVA_HOME=" + javaHome, "JMX_PORT=" + jmxPort, "KAFKA_LOG4J_OPTS=-Dlog4j.configuration=file:%s/config/test-log4j.properties" % kafkaHome, - kafkaRunClassBin + " kafka.perf.ProducerPerformance", + kafkaRunClassBin + " kafka.tools.ProducerPerformance", "--brokerinfo " + brokerInfoStr, "--initial-message-id " + str(initMsgId), "--messages " + noMsgPerBatch, @@ -1057,17 +1169,20 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk "--message-size " + messageSize, "--vary-message-size --async", " >> " + producerLogPathName, - " & echo pid:$! > " + producerLogPath + "/entity_" + entityId + "_pid'"] + " & echo $! > " + producerLogPath + "/entity_" + entityId + "_pid", + " & wait'"] cmdStr = " ".join(cmdList) logger.debug("executing command: [" + cmdStr + "]", extra=d) subproc = system_test_utils.sys_call_return_subproc(cmdStr) - for line in subproc.stdout.readlines(): - pass # dummy loop to wait until producer is completed + logger.debug("waiting for producer to finish", extra=d) + subproc.communicate() + logger.debug("producer finished", extra=d) else: testcaseEnv.numProducerThreadsRunning -= 1 logger.debug("testcaseEnv.numProducerThreadsRunning : " + str(testcaseEnv.numProducerThreadsRunning), extra=d) + logger.debug("calling testcaseEnv.lock.release()", extra=d) testcaseEnv.lock.release() break @@ -1079,24 +1194,30 @@ def start_producer_in_thread(testcaseEnv, entityConfigList, producerConfig, kafk # wait until other producer threads also stops and # let the main testcase know all producers have stopped while 1: + logger.debug("calling testcaseEnv.lock.acquire()", extra=d) testcaseEnv.lock.acquire() time.sleep(1) if testcaseEnv.numProducerThreadsRunning == 0: testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"] = True + logger.debug("calling testcaseEnv.lock.release()", extra=d) testcaseEnv.lock.release() break else: logger.debug("waiting for TRUE of testcaseEnv.userDefinedEnvVarDict['backgroundProducerStopped']", extra=d) + logger.debug("calling testcaseEnv.lock.release()", extra=d) testcaseEnv.lock.release() time.sleep(1) + # finally remove itself from the tracking pids + del testcaseEnv.producerHostParentPidDict[entityId] + def stop_remote_entity(systemTestEnv, entityId, parentPid, signalType="SIGTERM"): clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList hostname = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "entity_id", entityId, "hostname") pidStack = system_test_utils.get_remote_child_processes(hostname, parentPid) - logger.debug("terminating (" + signalType + ") process id: " + parentPid + " in host: " + hostname, extra=d) + logger.info("terminating (" + signalType + ") process id: " + parentPid + " in host: " + hostname, extra=d) if signalType.lower() == "sigterm": system_test_utils.sigterm_remote_process(hostname, pidStack) @@ -1117,7 +1238,7 @@ def force_stop_remote_entity(systemTestEnv, entityId, parentPid): system_test_utils.sigkill_remote_process(hostname, pidStack) -def create_topic(systemTestEnv, testcaseEnv): +def create_topic_for_producer_performance(systemTestEnv, testcaseEnv): clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList prodPerfCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "producer_performance") @@ -1149,7 +1270,7 @@ def create_topic(systemTestEnv, testcaseEnv): testcaseBaseDir = replace_kafka_home(testcaseBaseDir, kafkaHome) for topic in topicsList: - logger.info("creating topic: [" + topic + "] at: [" + zkConnectStr + "]", extra=d) + logger.info("creating topic: [" + topic + "] at: [" + zkConnectStr + "]", extra=d) cmdList = ["ssh " + zkHost, "'JAVA_HOME=" + javaHome, createTopicBin, @@ -1158,11 +1279,45 @@ def create_topic(systemTestEnv, testcaseEnv): " --replication-factor " + testcaseEnv.testcaseArgumentsDict["replica_factor"], " --partitions " + testcaseEnv.testcaseArgumentsDict["num_partition"] + " >> ", testcaseBaseDir + "/logs/create_source_cluster_topic.log'"] - + cmdStr = " ".join(cmdList) logger.debug("executing command: [" + cmdStr + "]", extra=d) subproc = system_test_utils.sys_call_return_subproc(cmdStr) +def create_topic(systemTestEnv, testcaseEnv, topic, replication_factor, num_partitions): + clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList + zkEntityId = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "role", "zookeeper", "entity_id") + 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-topics.sh --create" + zkConnectStr = "" + zkHost = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "role", "zookeeper", "hostname") + if len(testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"]) > 0: + zkConnectStr = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] + elif len(testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"]) > 0: + zkConnectStr = testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"] + else: + raise Exception("Empty zkConnectStr found") + + testcaseBaseDir = testcaseEnv.testCaseBaseDir + + testcaseBaseDir = replace_kafka_home(testcaseBaseDir, kafkaHome) + + logger.debug("creating topic: [" + topic + "] at: [" + zkConnectStr + "]", extra=d) + cmdList = ["ssh " + zkHost, + "'JAVA_HOME=" + javaHome, + createTopicBin, + " --topic " + topic, + " --zookeeper " + zkConnectStr, + " --replication-factor " + str(replication_factor), + " --partitions " + str(num_partitions) + " >> ", + testcaseBaseDir + "/logs/create_source_cluster_topic.log'"] + + cmdStr = " ".join(cmdList) + logger.info("executing command: [" + cmdStr + "]", extra=d) + subproc = system_test_utils.sys_call_return_subproc(cmdStr) + + def get_message_id(logPathName, topic=""): logLines = open(logPathName, "r").readlines() @@ -1200,7 +1355,7 @@ def get_message_checksum(logPathName): def validate_data_matched(systemTestEnv, testcaseEnv, replicationUtils): - logger.debug("#### Inside validate_data_matched", extra=d) + logger.info("#### Inside validate_data_matched", extra=d) validationStatusDict = testcaseEnv.validationStatusDict clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList @@ -1224,11 +1379,11 @@ def validate_data_matched(systemTestEnv, testcaseEnv, replicationUtils): consumerTopic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", consumerEntityId, "topic") if consumerTopic in topic: matchingConsumerEntityId = consumerEntityId - logger.debug("matching consumer entity id found", extra=d) + logger.info("matching consumer entity id found", extra=d) break if matchingConsumerEntityId is None: - logger.debug("matching consumer entity id NOT found", extra=d) + logger.info("matching consumer entity id NOT found", extra=d) break msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( \ @@ -1316,7 +1471,7 @@ def cleanup_data_at_remote_hosts(systemTestEnv, testcaseEnv): logger.info("cleaning up test case dir: [" + testCaseBaseDir + "]", extra=d) if "system_test" not in testCaseBaseDir: - logger.warn("possible destructive command [" + cmdStr + "]", extra=d) + # logger.warn("possible destructive command [" + cmdStr + "]", extra=d) logger.warn("check config file: system_test/cluster_config.properties", extra=d) logger.warn("aborting test...", extra=d) sys.exit(1) @@ -1416,7 +1571,7 @@ def ps_grep_terminate_running_entity(systemTestEnv): cmdStr = " ".join(cmdList) logger.debug("executing command [" + cmdStr + "]", extra=d) - system_test_utils.sys_call(cmdStr) + system_test_utils.sys_call(cmdStr) def get_reelection_latency(systemTestEnv, testcaseEnv, leaderDict, leaderAttributesDict): leaderEntityId = None @@ -1473,7 +1628,7 @@ def get_reelection_latency(systemTestEnv, testcaseEnv, leaderDict, leaderAttribu if shutdownTimestamp > 0: leaderReElectionLatency = float(leaderDict2["timestamp"]) - float(shutdownTimestamp) logger.info("leader Re-election Latency: " + str(leaderReElectionLatency) + " sec", extra=d) - + return leaderReElectionLatency @@ -1481,9 +1636,36 @@ def stop_all_remote_running_processes(systemTestEnv, testcaseEnv): entityConfigs = systemTestEnv.clusterEntityConfigDictList - for hostname, producerPPid in testcaseEnv.producerHostParentPidDict.items(): - producerEntityId = system_test_utils.get_data_by_lookup_keyval(entityConfigs, "hostname", hostname, "entity_id") - stop_remote_entity(systemTestEnv, producerEntityId, producerPPid) + # If there are any alive local threads that keep starting remote producer performance, we need to kill them; + # note we do not need to stop remote processes since they will terminate themselves eventually. + if len(testcaseEnv.producerHostParentPidDict) != 0: + # ============================================= + # tell producer to stop + # ============================================= + logger.debug("calling testcaseEnv.lock.acquire()", extra=d) + testcaseEnv.lock.acquire() + testcaseEnv.userDefinedEnvVarDict["stopBackgroundProducer"] = True + logger.debug("calling testcaseEnv.lock.release()", extra=d) + testcaseEnv.lock.release() + + # ============================================= + # wait for producer thread's update of + # "backgroundProducerStopped" to be "True" + # ============================================= + while 1: + logger.debug("calling testcaseEnv.lock.acquire()", extra=d) + testcaseEnv.lock.acquire() + logger.info("status of backgroundProducerStopped : [" + \ + str(testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]) + "]", extra=d) + if testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]: + logger.debug("calling testcaseEnv.lock.release()", extra=d) + testcaseEnv.lock.release() + logger.info("all producer threads completed", extra=d) + break + logger.debug("calling testcaseEnv.lock.release()", extra=d) + testcaseEnv.lock.release() + + testcaseEnv.producerHostParentPidDict.clear() for hostname, consumerPPid in testcaseEnv.consumerHostParentPidDict.items(): consumerEntityId = system_test_utils.get_data_by_lookup_keyval(entityConfigs, "hostname", hostname, "entity_id") @@ -1496,6 +1678,9 @@ def stop_all_remote_running_processes(systemTestEnv, testcaseEnv): for entityId, mirrorMakerParentPid in testcaseEnv.entityMirrorMakerParentPidDict.items(): stop_remote_entity(systemTestEnv, entityId, mirrorMakerParentPid) + for entityId, consumerParentPid in testcaseEnv.entityConsoleConsumerParentPidDict.items(): + stop_remote_entity(systemTestEnv, entityId, consumerParentPid) + for entityId, brokerParentPid in testcaseEnv.entityBrokerParentPidDict.items(): stop_remote_entity(systemTestEnv, entityId, brokerParentPid) @@ -1514,8 +1699,8 @@ def start_migration_tool(systemTestEnv, testcaseEnv, onlyThisEntityId=None): if onlyThisEntityId is None or entityId == onlyThisEntityId: host = migrationToolConfig["hostname"] - jmxPort = migrationToolConfig["jmx_port"] - role = migrationToolConfig["role"] + jmxPort = migrationToolConfig["jmx_port"] + role = migrationToolConfig["role"] kafkaHome = system_test_utils.get_data_by_lookup_keyval(clusterConfigList, "entity_id", entityId, "kafka_home") javaHome = system_test_utils.get_data_by_lookup_keyval(clusterConfigList, "entity_id", entityId, "java_home") jmxPort = system_test_utils.get_data_by_lookup_keyval(clusterConfigList, "entity_id", entityId, "jmx_port") @@ -1581,7 +1766,7 @@ def validate_07_08_migrated_data_matched(systemTestEnv, testcaseEnv): producerEntityId = prodPerfCfg["entity_id"] topic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", producerEntityId, "topic") - consumerEntityIdList = system_test_utils.get_data_from_list_of_dicts( + consumerEntityIdList = system_test_utils.get_data_from_list_of_dicts( clusterEntityConfigDictList, "role", "console_consumer", "entity_id") matchingConsumerEntityId = None @@ -1595,7 +1780,7 @@ def validate_07_08_migrated_data_matched(systemTestEnv, testcaseEnv): if matchingConsumerEntityId is None: break - msgChecksumMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( + msgChecksumMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( testcaseEnv, "console_consumer", matchingConsumerEntityId, "default") \ + "/msg_checksum_missing_in_consumer.log" producerLogPath = get_testcase_config_log_dir_pathname(testcaseEnv, "producer_performance", producerEntityId, "default") @@ -1680,7 +1865,7 @@ def validate_broker_log_segment_checksum(systemTestEnv, testcaseEnv, clusterName # |- 00000000000000000020.log # |- . . . - # loop through all topicPartition directories such as : test_1-0, test_1-1, ... + # loop through all topicPartition directories such as : test_1-0, test_1-1, ... for topicPartition in os.listdir(localLogSegmentPath): # found a topic-partition directory if os.path.isdir(localLogSegmentPath + "/" + topicPartition): @@ -1733,7 +1918,7 @@ def validate_broker_log_segment_checksum(systemTestEnv, testcaseEnv, clusterName # 'test_2-0' : ['d41d8cd98f00b204e9800998ecf8427e','d41d8cd98f00b204e9800998ecf8427e'], # 'test_2-1' : ['d41d8cd98f00b204e9800998ecf8427e','d41d8cd98f00b204e9800998ecf8427e'] # } - + for brokerTopicPartitionKey, md5Checksum in brokerLogCksumDict.items(): tokens = brokerTopicPartitionKey.split(":") brokerKey = tokens[0] @@ -1759,7 +1944,7 @@ def validate_broker_log_segment_checksum(systemTestEnv, testcaseEnv, clusterName logger.debug("merged log segment checksum in " + topicPartition + " matched", extra=d) else: logger.error("unexpected error in " + topicPartition, extra=d) - + if failureCount == 0: validationStatusDict["Validate for merged log segment checksum in cluster [" + clusterName + "]"] = "PASSED" else: @@ -1772,8 +1957,8 @@ def start_simple_consumer(systemTestEnv, testcaseEnv, minStartingOffsetDict=None for consumerConfig in consumerConfigList: host = consumerConfig["hostname"] entityId = consumerConfig["entity_id"] - jmxPort = consumerConfig["jmx_port"] - clusterName = consumerConfig["cluster_name"] + jmxPort = consumerConfig["jmx_port"] + clusterName = consumerConfig["cluster_name"] kafkaHome = system_test_utils.get_data_by_lookup_keyval(clusterList, "entity_id", entityId, "kafka_home") javaHome = system_test_utils.get_data_by_lookup_keyval(clusterList, "entity_id", entityId, "java_home") kafkaRunClassBin = kafkaHome + "/bin/kafka-run-class.sh" @@ -1837,16 +2022,16 @@ def start_simple_consumer(systemTestEnv, testcaseEnv, minStartingOffsetDict=None "--no-wait-at-logend ", " > " + outputFilePathName, " & echo pid:$! > " + consumerLogPath + "/entity_" + entityId + "_pid'"] - + cmdStr = " ".join(cmdList) - + logger.debug("executing command: [" + cmdStr + "]", extra=d) subproc_1 = system_test_utils.sys_call_return_subproc(cmdStr) # dummy for-loop to wait until the process is completed for line in subproc_1.stdout.readlines(): - pass + pass time.sleep(1) - + partitionId += 1 replicaIndex += 1 @@ -1855,7 +2040,7 @@ def get_controller_attributes(systemTestEnv, testcaseEnv): logger.info("Querying Zookeeper for Controller info ...", extra=d) # keep track of controller data in this dict such as broker id & entity id - controllerDict = {} + controllerDict = {} clusterConfigsList = systemTestEnv.clusterEntityConfigDictList tcConfigsList = testcaseEnv.testcaseConfigsList @@ -1871,7 +2056,7 @@ def get_controller_attributes(systemTestEnv, testcaseEnv): cmdStrList = ["ssh " + hostname, "\"JAVA_HOME=" + javaHome, - kafkaRunClassBin + " org.apache.zookeeper.ZooKeeperMain", + kafkaRunClassBin + " kafka.tools.ZooKeeperMainWrapper ", "-server " + testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"], "get /controller 2> /dev/null | tail -1\""] @@ -1910,7 +2095,7 @@ def getMinCommonStartingOffset(systemTestEnv, testcaseEnv, clusterName="source") logPathName = get_testcase_config_log_dir_pathname(testcaseEnv, "broker", brokerEntityId, "default") localLogSegmentPath = logPathName + "/" + remoteLogSegmentDir - # loop through all topicPartition directories such as : test_1-0, test_1-1, ... + # loop through all topicPartition directories such as : test_1-0, test_1-1, ... for topicPartition in sorted(os.listdir(localLogSegmentPath)): # found a topic-partition directory if os.path.isdir(localLogSegmentPath + "/" + topicPartition): @@ -1949,7 +2134,7 @@ def getMinCommonStartingOffset(systemTestEnv, testcaseEnv, clusterName="source") # u'3:test_2-0': '0', # u'3:test_2-1': '0'} - # loop through brokerLogStartOffsetDict to get the min common starting offset for each topic-partition + # loop through brokerLogStartOffsetDict to get the min common starting offset for each topic-partition for brokerTopicPartition in sorted(brokerLogStartOffsetDict.iterkeys()): topicPartition = brokerTopicPartition.split(':')[1] @@ -2076,7 +2261,6 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList prodPerfCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "producer_performance") - consumerCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "console_consumer") for prodPerfCfg in prodPerfCfgList: producerEntityId = prodPerfCfg["entity_id"] @@ -2104,7 +2288,7 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe topicList = topicStr.split(',') for topic in topicList: consumerDuplicateCount = 0 - msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( + msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( testcaseEnv, "console_consumer", matchingConsumerEntityId, "default") \ + "/msg_id_missing_in_consumer_" + topic + ".log" producerMsgIdList = get_message_id(producerLogPathName, topic) @@ -2120,8 +2304,11 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe outfile.write(id + "\n") outfile.close() + logger.info("Producer entity id " + producerEntityId, extra=d) + logger.info("Consumer entity id " + matchingConsumerEntityId, extra=d) logger.info("no. of unique messages on topic [" + topic + "] sent from publisher : " + str(len(producerMsgIdSet)), extra=d) logger.info("no. of unique messages on topic [" + topic + "] received by consumer : " + str(len(consumerMsgIdSet)), extra=d) + logger.info("no. of duplicate messages on topic [" + topic + "] received by consumer: " + str(consumerDuplicateCount), extra=d) validationStatusDict["Unique messages from producer on [" + topic + "]"] = str(len(producerMsgIdSet)) validationStatusDict["Unique messages from consumer on [" + topic + "]"] = str(len(consumerMsgIdSet)) @@ -2214,18 +2401,58 @@ def validate_index_log(systemTestEnv, testcaseEnv, clusterName="source"): logger.debug("#### error found [" + line + "]", extra=d) failureCount += 1 showMismatchedIndexOffset = True + if subproc.wait() != 0: + logger.debug("#### error found [DumpLogSegments exited abnormally]", extra=d) + failureCount += 1 if failureCount == 0: validationStatusDict["Validate index log in cluster [" + clusterName + "]"] = "PASSED" else: validationStatusDict["Validate index log in cluster [" + clusterName + "]"] = "FAILED" +def get_leader_for(systemTestEnv, testcaseEnv, topic, partition): + logger.info("Querying Zookeeper for leader info for topic " + topic, extra=d) + clusterConfigsList = systemTestEnv.clusterEntityConfigDictList + tcConfigsList = testcaseEnv.testcaseConfigsList + + zkDictList = system_test_utils.get_dict_from_list_of_dicts(clusterConfigsList, "role", "zookeeper") + firstZkDict = zkDictList[0] + hostname = firstZkDict["hostname"] + zkEntityId = firstZkDict["entity_id"] + clientPort = system_test_utils.get_data_by_lookup_keyval(tcConfigsList, "entity_id", zkEntityId, "clientPort") + kafkaHome = system_test_utils.get_data_by_lookup_keyval(clusterConfigsList, "entity_id", zkEntityId, "kafka_home") + javaHome = system_test_utils.get_data_by_lookup_keyval(clusterConfigsList, "entity_id", zkEntityId, "java_home") + kafkaRunClassBin = kafkaHome + "/bin/kafka-run-class.sh" + + zkQueryStr = "get /brokers/topics/" + topic + "/partitions/" + str(partition) + "/state" + brokerid = '' + leaderEntityId = '' + + cmdStrList = ["ssh " + hostname, + "\"JAVA_HOME=" + javaHome, + kafkaRunClassBin + " kafka.tools.ZooKeeperMainWrapper ", + "-server " + testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"], + zkQueryStr + " 2> /dev/null | tail -1\""] + cmdStr = " ".join(cmdStrList) + logger.info("executing command [" + cmdStr + "]", extra=d) + subproc = system_test_utils.sys_call_return_subproc(cmdStr) + for line in subproc.stdout.readlines(): + if "\"leader\"" in line: + line = line.rstrip('\n') + json_data = json.loads(line) + for key,val in json_data.items(): + if key == 'leader': + brokerid = str(val) + leaderEntityId = system_test_utils.get_data_by_lookup_keyval(tcConfigsList, "broker.id", brokerid, "entity_id") + break + return leaderEntityId + def get_leader_attributes(systemTestEnv, testcaseEnv): logger.info("Querying Zookeeper for leader info ...", extra=d) # keep track of leader data in this dict such as broker id & entity id - leaderDict = {} + leaderDict = {} clusterConfigsList = systemTestEnv.clusterEntityConfigDictList tcConfigsList = testcaseEnv.testcaseConfigsList @@ -2247,15 +2474,14 @@ def get_leader_attributes(systemTestEnv, testcaseEnv): cmdStrList = ["ssh " + hostname, "\"JAVA_HOME=" + javaHome, - kafkaRunClassBin + " org.apache.zookeeper.ZooKeeperMain", + kafkaRunClassBin + " kafka.tools.ZooKeeperMainWrapper ", "-server " + testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"], zkQueryStr + " 2> /dev/null | tail -1\""] cmdStr = " ".join(cmdStrList) - logger.debug("executing command [" + cmdStr + "]", extra=d) + logger.info("executing command [" + cmdStr + "]", extra=d) subproc = system_test_utils.sys_call_return_subproc(cmdStr) for line in subproc.stdout.readlines(): - logger.debug("zk returned : " + line, extra=d) if "\"leader\"" in line: line = line.rstrip('\n') json_data = json.loads(line) @@ -2275,5 +2501,12 @@ def get_leader_attributes(systemTestEnv, testcaseEnv): print leaderDict return leaderDict - +def write_consumer_properties(consumerProperties): + import tempfile + props_file_path = tempfile.gettempdir() + "/consumer.properties" + consumer_props_file=open(props_file_path,"w") + for key,value in consumerProperties.iteritems(): + consumer_props_file.write(key+"="+value+"\n") + consumer_props_file.close() + return props_file_path diff --git a/system_test/utils/metrics.py b/system_test/utils/metrics.py index d98d3cdeab00b..3e663483202a1 100644 --- a/system_test/utils/metrics.py +++ b/system_test/utils/metrics.py @@ -106,6 +106,8 @@ def ensure_valid_headers(headers, attributes): " headers: {0}".format(",".join(headers))) def plot_graphs(inputCsvFiles, labels, title, xLabel, yLabel, attribute, outputGraphFile): + if not inputCsvFiles: return + # create empty plot fig=plt.figure() fig.subplots_adjust(bottom=0.2) diff --git a/system_test/utils/system_test_utils.py b/system_test/utils/system_test_utils.py index 50340f067774e..e8529cd31f920 100644 --- a/system_test/utils/system_test_utils.py +++ b/system_test/utils/system_test_utils.py @@ -360,19 +360,23 @@ def setup_remote_hosts(systemTestEnv): clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList localKafkaHome = os.path.abspath(systemTestEnv.SYSTEM_TEST_BASE_DIR + "/..") - localJavaBin = "" - localJavaHome = "" - subproc = sys_call_return_subproc("which java") - for line in subproc.stdout.readlines(): - if line.startswith("which: no "): - logger.error("No Java binary found in local host", extra=d) - return False - else: - line = line.rstrip('\n') - localJavaBin = line - matchObj = re.match("(.*)\/bin\/java$", line) - localJavaHome = matchObj.group(1) + # when configuring "default" java_home, use JAVA_HOME environment variable, if exists + # otherwise, use the directory with the java binary + localJavaHome = os.environ.get('JAVA_HOME') + if localJavaHome is not None: + localJavaBin = localJavaHome + '/bin/java' + else: + subproc = sys_call_return_subproc("which java") + for line in subproc.stdout.readlines(): + if line.startswith("which: no "): + logger.error("No Java binary found in local host", extra=d) + return False + else: + line = line.rstrip('\n') + localJavaBin = line + matchObj = re.match("(.*)\/bin\/java$", line) + localJavaHome = matchObj.group(1) listIndex = -1 for clusterEntityConfigDict in clusterEntityConfigDictList: diff --git a/system_test/utils/testcase_env.py b/system_test/utils/testcase_env.py index bee87166510b7..1d2fb5762f531 100644 --- a/system_test/utils/testcase_env.py +++ b/system_test/utils/testcase_env.py @@ -28,62 +28,67 @@ import system_test_utils class TestcaseEnv(): + def __init__(self, systemTestEnv, classInstance): + self.systemTestEnv = systemTestEnv - # ================================ - # Generic testcase environment - # ================================ - - # dictionary of entity_id to ppid for Zookeeper entities - # key: entity_id - # val: ppid of Zookeeper associated to that entity_id - # { 0: 12345, 1: 12389, ... } - entityZkParentPidDict = {} - - # dictionary of entity_id to ppid for broker entities - # key: entity_id - # val: ppid of broker associated to that entity_id - # { 0: 12345, 1: 12389, ... } - entityBrokerParentPidDict = {} - - # dictionary of entity_id to ppid for mirror-maker entities - # key: entity_id - # val: ppid of broker associated to that entity_id - # { 0: 12345, 1: 12389, ... } - entityMirrorMakerParentPidDict = {} - - # dictionary of entity_id to ppid for migration tool entities - # key: entity_id - # val: ppid of broker associated to that entity_id - # { 0: 12345, 1: 12389, ... } - entityMigrationToolParentPidDict = {} - - # dictionary of entity_id to list of JMX ppid - # key: entity_id - # val: list of JMX ppid associated to that entity_id - # { 1: [1234, 1235, 1236], 2: [2234, 2235, 2236], ... } - entityJmxParentPidDict = {} - - # dictionary of hostname-topic-ppid for consumer - # key: hostname - # val: dict of topic-ppid - # { host1: { test1 : 12345 }, host1: { test2 : 12389 }, ... } - consumerHostParentPidDict = {} - - # dictionary of hostname-topic-ppid for producer - # key: hostname - # val: dict of topic-ppid - # { host1: { test1 : 12345 }, host1: { test2 : 12389 }, ... } - producerHostParentPidDict = {} - - # list of testcase configs - testcaseConfigsList = [] - - # dictionary to keep track of testcase arguments such as replica_factor, num_partition - testcaseArgumentsDict = {} + # ================================ + # Generic testcase environment + # ================================ + # dictionary of entity_id to ppid for Zookeeper entities + # key: entity_id + # val: ppid of Zookeeper associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityZkParentPidDict = {} + + # dictionary of entity_id to ppid for broker entities + # key: entity_id + # val: ppid of broker associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityBrokerParentPidDict = {} + + # dictionary of entity_id to ppid for mirror-maker entities + # key: entity_id + # val: ppid of broker associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityMirrorMakerParentPidDict = {} + + # dictionary of entity_id to ppid for console-consumer entities + # key: entity_id + # val: ppid of console consumer associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityConsoleConsumerParentPidDict = {} + + # dictionary of entity_id to ppid for migration tool entities + # key: entity_id + # val: ppid of broker associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityMigrationToolParentPidDict = {} + + # dictionary of entity_id to list of JMX ppid + # key: entity_id + # val: list of JMX ppid associated to that entity_id + # { 1: [1234, 1235, 1236], 2: [2234, 2235, 2236], ... } + self.entityJmxParentPidDict = {} + + # dictionary of hostname-topic-ppid for consumer + # key: hostname + # val: dict of topic-ppid + # { host1: { test1 : 12345 }, host1: { test2 : 12389 }, ... } + self.consumerHostParentPidDict = {} + + # dictionary of hostname-topic-ppid for producer + # key: hostname + # val: dict of topic-ppid + # { host1: { test1 : 12345 }, host1: { test2 : 12389 }, ... } + self.producerHostParentPidDict = {} + + # list of testcase configs + self.testcaseConfigsList = [] + + # dictionary to keep track of testcase arguments such as replica_factor, num_partition + self.testcaseArgumentsDict = {} - def __init__(self, systemTestEnv, classInstance): - self.systemTestEnv = systemTestEnv # gather the test case related info and add to an SystemTestEnv object self.testcaseResultsDict = {} diff --git a/vagrant/README.md b/vagrant/README.md new file mode 100644 index 0000000000000..73cf0390bc4c7 --- /dev/null +++ b/vagrant/README.md @@ -0,0 +1,126 @@ +# Apache Kafka # + +Using Vagrant to get up and running. + +1) Install Virtual Box [https://www.virtualbox.org/](https://www.virtualbox.org/) +2) Install Vagrant >= 1.6.4 [http://www.vagrantup.com/](http://www.vagrantup.com/) +3) Install Vagrant Plugins: + + # Required + $ vagrant plugin install vagrant-hostmanager + # Optional + $ vagrant plugin install vagrant-cachier # Caches & shares package downloads across VMs + +In the main Kafka folder, do a normal Kafka build: + + $ gradle + $ ./gradlew jar + +You can override default settings in `Vagrantfile.local`, which is a Ruby file +that is ignored by git and imported into the Vagrantfile. +One setting you likely want to enable +in `Vagrantfile.local` is `enable_dns = true` to put hostnames in the host's +/etc/hosts file. You probably want this to avoid having to use IP addresses when +addressing the cluster from outside the VMs, e.g. if you run a client on the +host. It's disabled by default since it requires `sudo` access, mucks with your +system state, and breaks with naming conflicts if you try to run multiple +clusters concurrently. + +Now bring up the cluster: + + $ vagrant up --no-provision && vagrant provision + +We separate out the two steps (bringing up the base VMs and configuring them) +due to current limitations in ZooKeeper (ZOOKEEPER-1506) that require us to +collect IPs for all nodes before starting ZooKeeper nodes. + +Once this completes: + +* Zookeeper will be running on 192.168.50.11 (and `zk1` if you used enable_dns) +* Broker 1 on 192.168.50.51 (and `broker1` if you used enable_dns) +* Broker 2 on 192.168.50.52 (and `broker2` if you used enable_dns) +* Broker 3 on 192.168.50.53 (and `broker3` if you used enable_dns) + +To log into one of the machines: + + vagrant ssh + +You can access the brokers and zookeeper by their IP or hostname, e.g. + + # Specify ZooKeeper node 1 by it's IP: 192.168.50.11 + bin/kafka-topics.sh --create --zookeeper 192.168.50.11:2181 --replication-factor 3 --partitions 1 --topic sandbox + + # Specify brokers by their hostnames: broker1, broker2, broker3 + bin/kafka-console-producer.sh --broker-list broker1:9092,broker2:9092,broker3:9092 --topic sandbox + + # Specify ZooKeeper node by its hostname: zk1 + bin/kafka-console-consumer.sh --zookeeper zk1:2181 --topic sandbox --from-beginning + +If you need to update the running cluster, you can re-run the provisioner (the +step that installs software and configures services): + + vagrant provision + +Note that this doesn't currently ensure a fresh start -- old cluster state will +still remain intact after everything restarts. This can be useful for updating +the cluster to your most recent development version. + +Finally, you can clean up the cluster by destroying all the VMs: + + vagrant destroy + +## Configuration ## + +You can override some default settings by specifying the values in +`Vagrantfile.local`. It is interpreted as a Ruby file, although you'll probably +only ever need to change a few simple configuration variables. Some values you +might want to override: + +* `enable_dns` - Register each VM with a hostname in /etc/hosts on the + hosts. Hostnames are always set in the /etc/hosts in the VMs, so this is only + necessary if you want to address them conveniently from the host for tasks + that aren't provided by Vagrant. +* `num_zookeepers` - Size of zookeeper cluster +* `num_brokers` - Number of broker instances to run + + +## Using Other Providers ## + +### EC2 ### + +Install the `vagrant-aws` plugin to provide EC2 support: + + $ vagrant plugin install vagrant-aws + +Next, configure parameters in `Vagrantfile.local`. A few are *required*: +`enable_dns`, `ec2_access_key`, `ec2_secret_key`, `ec2_keypair_name`, `ec2_keypair_file`, and +`ec2_security_groups`. A couple of important notes: + +1. You definitely want to use `enable_dns` if you plan to run clients outside of + the cluster (e.g. from your local host). If you don't, you'll need to go + lookup `vagrant ssh-config`. + +2. You'll have to setup a reasonable security group yourself. You'll need to + open ports for Zookeeper (2888 & 3888 between ZK nodes, 2181 for clients) and + Kafka (9092). Beware that opening these ports to all sources (e.g. so you can + run producers/consumers locally) will allow anyone to access your Kafka + cluster. All other settings have reasonable defaults for setting up an + Ubuntu-based cluster, but you may want to customize instance type, region, + AMI, etc. + +3. `ec2_access_key` and `ec2_secret_key` will use the environment variables + `AWS_ACCESS_KEY` and `AWS_SECRET_KEY` respectively if they are set and not + overridden in `Vagrantfile.local`. + +4. If you're launching into a VPC, you must specify `ec2_subnet_id` (the subnet + in which to launch the nodes) and `ec2_security_groups` must be a list of + security group IDs instead of names, e.g. `sg-34fd3551` instead of + `kafka-test-cluster`. + +Now start things up, but specify the aws provider: + + $ vagrant up --provider=aws --no-parallel --no-provision && vagrant provision + +Your instances should get tagged with a name including your hostname to make +them identifiable and make it easier to track instances in the AWS management +console. diff --git a/vagrant/base.sh b/vagrant/base.sh new file mode 100644 index 0000000000000..6f28dfed67877 --- /dev/null +++ b/vagrant/base.sh @@ -0,0 +1,43 @@ +# 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. + +#!/bin/bash + +set -e + +if [ -z `which javac` ]; then + apt-get -y update + apt-get install -y software-properties-common python-software-properties + add-apt-repository -y ppa:webupd8team/java + apt-get -y update + + # Try to share cache. See Vagrantfile for details + mkdir -p /var/cache/oracle-jdk7-installer + if [ -e "/tmp/oracle-jdk7-installer-cache/" ]; then + find /tmp/oracle-jdk7-installer-cache/ -not -empty -exec cp '{}' /var/cache/oracle-jdk7-installer/ \; + fi + + /bin/echo debconf shared/accepted-oracle-license-v1-1 select true | /usr/bin/debconf-set-selections + apt-get -y install oracle-java7-installer oracle-java7-set-default + + if [ -e "/tmp/oracle-jdk7-installer-cache/" ]; then + cp -R /var/cache/oracle-jdk7-installer/* /tmp/oracle-jdk7-installer-cache + fi +fi + +chmod a+rw /opt +if [ ! -e /opt/kafka ]; then + ln -s /vagrant /opt/kafka +fi diff --git a/vagrant/broker.sh b/vagrant/broker.sh new file mode 100644 index 0000000000000..bc040c92e3302 --- /dev/null +++ b/vagrant/broker.sh @@ -0,0 +1,43 @@ +# 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. + +#!/bin/bash + +# Usage: brokers.sh + +set -e + +BROKER_ID=$1 +PUBLIC_ADDRESS=$2 +PUBLIC_ZOOKEEPER_ADDRESSES=$3 +JMX_PORT=$4 + +cd /opt/kafka + +sed \ + -e 's/broker.id=0/'broker.id=$BROKER_ID'/' \ + -e 's/#advertised.host.name=/'advertised.host.name=$PUBLIC_ADDRESS'/' \ + -e 's/zookeeper.connect=localhost:2181/'zookeeper.connect=$PUBLIC_ZOOKEEPER_ADDRESSES'/' \ + /opt/kafka/config/server.properties > /opt/kafka/config/server-$BROKER_ID.properties + +echo "Killing server" +bin/kafka-server-stop.sh || true +sleep 5 # Because kafka-server-stop.sh doesn't actually wait +echo "Starting server" +if [[ -n $JMX_PORT ]]; then + export JMX_PORT=$JMX_PORT + export KAFKA_JMX_OPTS="-Djava.rmi.server.hostname=$PUBLIC_ADDRESS -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " +fi +bin/kafka-server-start.sh /opt/kafka/config/server-$BROKER_ID.properties 1>> /tmp/broker.log 2>> /tmp/broker.log & diff --git a/vagrant/zk.sh b/vagrant/zk.sh new file mode 100644 index 0000000000000..6fc4b7cfe8fed --- /dev/null +++ b/vagrant/zk.sh @@ -0,0 +1,47 @@ +# 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. + +#!/bin/bash + +# Usage: zk.sh + +set -e + +ZKID=$1 +NUM_ZK=$2 +JMX_PORT=$3 + +cd /opt/kafka + +cp /opt/kafka/config/zookeeper.properties /opt/kafka/config/zookeeper-$ZKID.properties +echo "initLimit=5" >> /opt/kafka/config/zookeeper-$ZKID.properties +echo "syncLimit=2" >> /opt/kafka/config/zookeeper-$ZKID.properties +echo "quorumListenOnAllIPs=true" >> /opt/kafka/config/zookeeper-$ZKID.properties +for i in `seq 1 $NUM_ZK`; do + echo "server.${i}=zk${i}:2888:3888" >> /opt/kafka/config/zookeeper-$ZKID.properties +done + +mkdir -p /tmp/zookeeper +echo "$ZKID" > /tmp/zookeeper/myid + +echo "Killing ZooKeeper" +bin/zookeeper-server-stop.sh || true +sleep 5 # Because zookeeper-server-stop.sh doesn't actually wait +echo "Starting ZooKeeper" +if [[ -n $JMX_PORT ]]; then + export JMX_PORT=$JMX_PORT + export KAFKA_JMX_OPTS="-Djava.rmi.server.hostname=zk$ZKID -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " +fi +bin/zookeeper-server-start.sh config/zookeeper-$ZKID.properties 1>> /tmp/zk.log 2>> /tmp/zk.log & diff --git a/wrapper.gradle b/wrapper.gradle new file mode 100644 index 0000000000000..bc6350632e2a6 --- /dev/null +++ b/wrapper.gradle @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +defaultTasks 'downloadWrapper' + +task downloadWrapper(type: Wrapper) { + description = "Download the gradle wrapper and requisite files. Overwrites existing wrapper files." + gradleVersion = project.gradleVersion +} \ No newline at end of file